diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R
index 31b986c326d0c..cdb59093781fb 100644
--- a/R/pkg/R/sparkR.R
+++ b/R/pkg/R/sparkR.R
@@ -266,11 +266,12 @@ sparkR.sparkContext <- function(
#' df <- read.json(path)
#'
#' sparkR.session("local[2]", "SparkR", "/home/spark")
-#' sparkR.session("yarn-client", "SparkR", "/home/spark",
-#' list(spark.executor.memory="4g"),
+#' sparkR.session("yarn", "SparkR", "/home/spark",
+#' list(spark.executor.memory="4g", spark.submit.deployMode="client"),
#' c("one.jar", "two.jar", "three.jar"),
#' c("com.databricks:spark-avro_2.12:2.0.1"))
-#' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g")
+#' sparkR.session(spark.master = "yarn", spark.submit.deployMode = "client",
+# spark.executor.memory = "4g")
#'}
#' @note sparkR.session since 2.0.0
sparkR.session <- function(
diff --git a/R/pkg/tests/fulltests/test_sparkR.R b/R/pkg/tests/fulltests/test_sparkR.R
index f73fc6baeccef..4232f5ec430f6 100644
--- a/R/pkg/tests/fulltests/test_sparkR.R
+++ b/R/pkg/tests/fulltests/test_sparkR.R
@@ -36,8 +36,8 @@ test_that("sparkCheckInstall", {
# "yarn-client, mesos-client" mode, SPARK_HOME was not set
sparkHome <- ""
- master <- "yarn-client"
- deployMode <- ""
+ master <- "yarn"
+ deployMode <- "client"
expect_error(sparkCheckInstall(sparkHome, master, deployMode))
sparkHome <- ""
master <- ""
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 24be54ec91828..72c9000a0a386 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -548,23 +548,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria
}
}
- if (contains("spark.master") && get("spark.master").startsWith("yarn-")) {
- val warning = s"spark.master ${get("spark.master")} is deprecated in Spark 2.0+, please " +
- "instead use \"yarn\" with specified deploy mode."
-
- get("spark.master") match {
- case "yarn-cluster" =>
- logWarning(warning)
- set("spark.master", "yarn")
- set(SUBMIT_DEPLOY_MODE, "cluster")
- case "yarn-client" =>
- logWarning(warning)
- set("spark.master", "yarn")
- set(SUBMIT_DEPLOY_MODE, "client")
- case _ => // Any other unexpected master will be checked when creating scheduler backend.
- }
- }
-
if (contains(SUBMIT_DEPLOY_MODE)) {
get(SUBMIT_DEPLOY_MODE) match {
case "cluster" | "client" =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 12a8473b22025..ebc843d9758ee 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -229,10 +229,6 @@ private[spark] class SparkSubmit extends Logging {
// Set the cluster manager
val clusterManager: Int = args.master match {
case "yarn" => YARN
- case "yarn-client" | "yarn-cluster" =>
- logWarning(s"Master ${args.master} is deprecated since 2.0." +
- " Please use master \"yarn\" with specified deploy mode instead.")
- YARN
case m if m.startsWith("spark") => STANDALONE
case m if m.startsWith("mesos") => MESOS
case m if m.startsWith("k8s") => KUBERNETES
@@ -251,22 +247,7 @@ private[spark] class SparkSubmit extends Logging {
-1
}
- // Because the deprecated way of specifying "yarn-cluster" and "yarn-client" encapsulate both
- // the master and deploy mode, we have some logic to infer the master and deploy mode
- // from each other if only one is specified, or exit early if they are at odds.
if (clusterManager == YARN) {
- (args.master, args.deployMode) match {
- case ("yarn-cluster", null) =>
- deployMode = CLUSTER
- args.master = "yarn"
- case ("yarn-cluster", "client") =>
- error("Client deploy mode is not compatible with master \"yarn-cluster\"")
- case ("yarn-client", "cluster") =>
- error("Cluster deploy mode is not compatible with master \"yarn-client\"")
- case (_, mode) =>
- args.master = "yarn"
- }
-
// Make sure YARN is included in our build if we're trying to use it
if (!Utils.classIsLoadable(YARN_CLUSTER_SUBMIT_CLASS) && !Utils.isTesting) {
error(
diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala
index 1f19884bc24d3..44f38e7043dcd 100644
--- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala
@@ -22,14 +22,15 @@ import org.apache.hadoop.fs.Path
import org.scalatest.Matchers
import org.apache.spark.{SparkConf, SparkFunSuite}
-import org.apache.spark.internal.config.STAGING_DIR
+import org.apache.spark.internal.config.{STAGING_DIR, SUBMIT_DEPLOY_MODE}
class HadoopFSDelegationTokenProviderSuite extends SparkFunSuite with Matchers {
test("hadoopFSsToAccess should return defaultFS even if not configured") {
val sparkConf = new SparkConf()
val defaultFS = "hdfs://localhost:8020"
val statingDir = "hdfs://localhost:8021"
- sparkConf.set("spark.master", "yarn-client")
+ sparkConf.setMaster("yarn")
+ sparkConf.set(SUBMIT_DEPLOY_MODE, "client")
sparkConf.set(STAGING_DIR, statingDir)
val hadoopConf = new Configuration()
hadoopConf.set("fs.defaultFS", defaultFS)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index 93a88cc30a20c..6b0928a0d2e68 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -437,7 +437,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
}
test("check blacklist configuration invariants") {
- val conf = new SparkConf().setMaster("yarn-cluster")
+ val conf = new SparkConf().setMaster("yarn").set(config.SUBMIT_DEPLOY_MODE, "cluster")
Seq(
(2, 2),
(2, 3)
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index 0f6dbf2f99a97..d4e6b6749010a 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -362,7 +362,6 @@ def __hash__(self):
"pyspark.sql.window",
"pyspark.sql.avro.functions",
# unittests
- "pyspark.sql.tests.test_appsubmit",
"pyspark.sql.tests.test_arrow",
"pyspark.sql.tests.test_catalog",
"pyspark.sql.tests.test_column",
diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md
index f931fa32ea541..fb2883de6810a 100644
--- a/docs/mllib-evaluation-metrics.md
+++ b/docs/mllib-evaluation-metrics.md
@@ -577,31 +577,3 @@ variable from a number of independent variables.
-
-**Examples**
-
-
-The following code snippets illustrate how to load a sample dataset, train a linear regression algorithm on the data,
-and evaluate the performance of the algorithm by several regression metrics.
-
-
-Refer to the [`RegressionMetrics` Scala docs](api/scala/index.html#org.apache.spark.mllib.evaluation.RegressionMetrics) for details on the API.
-
-{% include_example scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala %}
-
-
-
-
-Refer to the [`RegressionMetrics` Java docs](api/java/org/apache/spark/mllib/evaluation/RegressionMetrics.html) for details on the API.
-
-{% include_example java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java %}
-
-
-
-
-Refer to the [`RegressionMetrics` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.evaluation.RegressionMetrics) for more details on the API.
-
-{% include_example python/mllib/regression_metrics_example.py %}
-
-
-
diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md
index b7f8ae9d07b0a..33a223ad486af 100644
--- a/docs/mllib-feature-extraction.md
+++ b/docs/mllib-feature-extraction.md
@@ -348,17 +348,3 @@ Refer to the [`ElementwiseProduct` Python docs](api/python/pyspark.mllib.html#py
A feature transformer that projects vectors to a low-dimensional space using PCA.
Details you can read at [dimensionality reduction](mllib-dimensionality-reduction.html).
-
-### Example
-
-The following code demonstrates how to compute principal components on a `Vector`
-and use them to project the vectors into a low-dimensional space while keeping associated labels
-for calculation a [Linear Regression](mllib-linear-methods.html)
-
-
-
-Refer to the [`PCA` Scala docs](api/scala/index.html#org.apache.spark.mllib.feature.PCA) for details on the API.
-
-{% include_example scala/org/apache/spark/examples/mllib/PCAExample.scala %}
-
-
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md
index 2d3ec4ca24443..801876dbffa79 100644
--- a/docs/mllib-linear-methods.md
+++ b/docs/mllib-linear-methods.md
@@ -360,57 +360,6 @@ regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) u
regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is
known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error).
-**Examples**
-
-
-
-
-The following example demonstrates how to load training data, parse it as an RDD of LabeledPoint.
-The example then uses LinearRegressionWithSGD to build a simple linear model to predict label
-values. We compute the mean squared error at the end to evaluate
-[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
-
-Refer to the [`LinearRegressionWithSGD` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Scala docs](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionModel) for details on the API.
-
-{% include_example scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala %}
-
-[`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD)
-and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) can be used in a similar fashion as `LinearRegressionWithSGD`.
-
-
-
-
-All of MLlib's methods use Java-friendly types, so you can import and call them there the same
-way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the
-Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by
-calling `.rdd()` on your `JavaRDD` object. The corresponding Java example to
-the Scala snippet provided, is presented below:
-
-Refer to the [`LinearRegressionWithSGD` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionWithSGD.html) and [`LinearRegressionModel` Java docs](api/java/org/apache/spark/mllib/regression/LinearRegressionModel.html) for details on the API.
-
-{% include_example java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java %}
-
-
-
-The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint.
-The example then uses LinearRegressionWithSGD to build a simple linear model to predict label
-values. We compute the mean squared error at the end to evaluate
-[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
-
-Note that the Python API does not yet support model save/load but will in the future.
-
-Refer to the [`LinearRegressionWithSGD` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionWithSGD) and [`LinearRegressionModel` Python docs](api/python/pyspark.mllib.html#pyspark.mllib.regression.LinearRegressionModel) for more details on the API.
-
-{% include_example python/mllib/linear_regression_with_sgd_example.py %}
-
-
-
-In order to run the above application, follow the instructions
-provided in the [Self-Contained Applications](quick-start.html#self-contained-applications)
-section of the Spark
-quick-start guide. Be sure to also include *spark-mllib* to your build file as
-a dependency.
-
### Streaming linear regression
When data arrive in a streaming fashion, it is useful to fit regression models online,
diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md
index cc3ef1e757756..5add2fb93937d 100644
--- a/docs/sql-migration-guide-upgrade.md
+++ b/docs/sql-migration-guide-upgrade.md
@@ -23,6 +23,11 @@ license: |
{:toc}
## Upgrading From Spark SQL 2.4 to 3.0
+
+ - In Spark 3.0, the deprecated methods `SQLContext.createExternalTable` and `SparkSession.createExternalTable` have been removed in favor of its replacement, `createTable`.
+
+ - In Spark 3.0, the deprecated `HiveContext` class has been removed. Use `SparkSession.builder.enableHiveSupport()` instead.
+
- Since Spark 3.0, configuration `spark.sql.crossJoin.enabled` become internal configuration, and is true by default, so by default spark won't raise exception on sql with implicit cross join.
- Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases.
diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md
index 26a83e52de2b1..3389d453c2cbd 100644
--- a/docs/streaming-kinesis-integration.md
+++ b/docs/streaming-kinesis-integration.md
@@ -81,7 +81,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m
.storageLevel(StorageLevel.MEMORY_AND_DISK_2)
.build();
- See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html)
+ See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisInputDStream.html)
and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the [Running the Example](#running-the-example) subsection for instructions to run the example.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index f5abed74bff20..f6b579fbf74d1 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -2488,13 +2488,13 @@ additional effort may be necessary to achieve exactly-once semantics. There are
* [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and
[DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream)
* [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$),
- [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$),
+ [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisInputDStream),
- Java docs
* [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html),
[JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and
[JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html)
* [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html),
- [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html)
+ [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisInputDStream.html)
- Python docs
* [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream)
* [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils)
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java
deleted file mode 100644
index 324a781c1a44a..0000000000000
--- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaLinearRegressionWithSGDExample.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.examples.mllib;
-
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaSparkContext;
-
-// $example on$
-import scala.Tuple2;
-
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.mllib.linalg.Vectors;
-import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.mllib.regression.LinearRegressionModel;
-import org.apache.spark.mllib.regression.LinearRegressionWithSGD;
-// $example off$
-
-/**
- * Example for LinearRegressionWithSGD.
- */
-public class JavaLinearRegressionWithSGDExample {
- public static void main(String[] args) {
- SparkConf conf = new SparkConf().setAppName("JavaLinearRegressionWithSGDExample");
- JavaSparkContext sc = new JavaSparkContext(conf);
-
- // $example on$
- // Load and parse the data
- String path = "data/mllib/ridge-data/lpsa.data";
- JavaRDD data = sc.textFile(path);
- JavaRDD parsedData = data.map(line -> {
- String[] parts = line.split(",");
- String[] features = parts[1].split(" ");
- double[] v = new double[features.length];
- for (int i = 0; i < features.length - 1; i++) {
- v[i] = Double.parseDouble(features[i]);
- }
- return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v));
- });
- parsedData.cache();
-
- // Building the model
- int numIterations = 100;
- double stepSize = 0.00000001;
- LinearRegressionModel model =
- LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, stepSize);
-
- // Evaluate model on training examples and compute training error
- JavaPairRDD valuesAndPreds = parsedData.mapToPair(point ->
- new Tuple2<>(model.predict(point.features()), point.label()));
-
- double MSE = valuesAndPreds.mapToDouble(pair -> {
- double diff = pair._1() - pair._2();
- return diff * diff;
- }).mean();
- System.out.println("training Mean Squared Error = " + MSE);
-
- // Save and load model
- model.save(sc.sc(), "target/tmp/javaLinearRegressionWithSGDModel");
- LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(),
- "target/tmp/javaLinearRegressionWithSGDModel");
- // $example off$
-
- sc.stop();
- }
-}
diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java
deleted file mode 100644
index 00033b5730a3d..0000000000000
--- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRegressionMetricsExample.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.examples.mllib;
-
-// $example on$
-import scala.Tuple2;
-
-import org.apache.spark.api.java.*;
-import org.apache.spark.mllib.linalg.Vectors;
-import org.apache.spark.mllib.regression.LabeledPoint;
-import org.apache.spark.mllib.regression.LinearRegressionModel;
-import org.apache.spark.mllib.regression.LinearRegressionWithSGD;
-import org.apache.spark.mllib.evaluation.RegressionMetrics;
-import org.apache.spark.SparkConf;
-// $example off$
-
-public class JavaRegressionMetricsExample {
- public static void main(String[] args) {
- SparkConf conf = new SparkConf().setAppName("Java Regression Metrics Example");
- JavaSparkContext sc = new JavaSparkContext(conf);
- // $example on$
- // Load and parse the data
- String path = "data/mllib/sample_linear_regression_data.txt";
- JavaRDD data = sc.textFile(path);
- JavaRDD parsedData = data.map(line -> {
- String[] parts = line.split(" ");
- double[] v = new double[parts.length - 1];
- for (int i = 1; i < parts.length; i++) {
- v[i - 1] = Double.parseDouble(parts[i].split(":")[1]);
- }
- return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v));
- });
- parsedData.cache();
-
- // Building the model
- int numIterations = 100;
- LinearRegressionModel model = LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData),
- numIterations);
-
- // Evaluate model on training examples and compute training error
- JavaPairRDD valuesAndPreds = parsedData.mapToPair(point ->
- new Tuple2<>(model.predict(point.features()), point.label()));
-
- // Instantiate metrics object
- RegressionMetrics metrics = new RegressionMetrics(valuesAndPreds.rdd());
-
- // Squared error
- System.out.format("MSE = %f\n", metrics.meanSquaredError());
- System.out.format("RMSE = %f\n", metrics.rootMeanSquaredError());
-
- // R-squared
- System.out.format("R Squared = %f\n", metrics.r2());
-
- // Mean absolute error
- System.out.format("MAE = %f\n", metrics.meanAbsoluteError());
-
- // Explained variance
- System.out.format("Explained Variance = %f\n", metrics.explainedVariance());
-
- // Save and load model
- model.save(sc.sc(), "target/tmp/LogisticRegressionModel");
- LinearRegressionModel sameModel = LinearRegressionModel.load(sc.sc(),
- "target/tmp/LogisticRegressionModel");
- // $example off$
-
- sc.stop();
- }
-}
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
deleted file mode 100644
index 03222b13ad27d..0000000000000
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-// scalastyle:off println
-package org.apache.spark.examples.mllib
-
-import org.apache.log4j.{Level, Logger}
-import scopt.OptionParser
-
-import org.apache.spark.{SparkConf, SparkContext}
-import org.apache.spark.mllib.optimization.{L1Updater, SimpleUpdater, SquaredL2Updater}
-import org.apache.spark.mllib.regression.LinearRegressionWithSGD
-import org.apache.spark.mllib.util.MLUtils
-
-/**
- * An example app for linear regression. Run with
- * {{{
- * bin/run-example org.apache.spark.examples.mllib.LinearRegression
- * }}}
- * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`.
- * If you use it as a template to create your own app, please use `spark-submit` to submit your app.
- */
-@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0")
-object LinearRegression {
-
- object RegType extends Enumeration {
- type RegType = Value
- val NONE, L1, L2 = Value
- }
-
- import RegType._
-
- case class Params(
- input: String = null,
- numIterations: Int = 100,
- stepSize: Double = 1.0,
- regType: RegType = L2,
- regParam: Double = 0.01) extends AbstractParams[Params]
-
- def main(args: Array[String]) {
- val defaultParams = Params()
-
- val parser = new OptionParser[Params]("LinearRegression") {
- head("LinearRegression: an example app for linear regression.")
- opt[Int]("numIterations")
- .text("number of iterations")
- .action((x, c) => c.copy(numIterations = x))
- opt[Double]("stepSize")
- .text(s"initial step size, default: ${defaultParams.stepSize}")
- .action((x, c) => c.copy(stepSize = x))
- opt[String]("regType")
- .text(s"regularization type (${RegType.values.mkString(",")}), " +
- s"default: ${defaultParams.regType}")
- .action((x, c) => c.copy(regType = RegType.withName(x)))
- opt[Double]("regParam")
- .text(s"regularization parameter, default: ${defaultParams.regParam}")
- arg[String](" ")
- .required()
- .text("input paths to labeled examples in LIBSVM format")
- .action((x, c) => c.copy(input = x))
- note(
- """
- |For example, the following command runs this app on a synthetic dataset:
- |
- | bin/spark-submit --class org.apache.spark.examples.mllib.LinearRegression \
- | examples/target/scala-*/spark-examples-*.jar \
- | data/mllib/sample_linear_regression_data.txt
- """.stripMargin)
- }
-
- parser.parse(args, defaultParams) match {
- case Some(params) => run(params)
- case _ => sys.exit(1)
- }
- }
-
- def run(params: Params): Unit = {
- val conf = new SparkConf().setAppName(s"LinearRegression with $params")
- val sc = new SparkContext(conf)
-
- Logger.getRootLogger.setLevel(Level.WARN)
-
- val examples = MLUtils.loadLibSVMFile(sc, params.input).cache()
-
- val splits = examples.randomSplit(Array(0.8, 0.2))
- val training = splits(0).cache()
- val test = splits(1).cache()
-
- val numTraining = training.count()
- val numTest = test.count()
- println(s"Training: $numTraining, test: $numTest.")
-
- examples.unpersist()
-
- val updater = params.regType match {
- case NONE => new SimpleUpdater()
- case L1 => new L1Updater()
- case L2 => new SquaredL2Updater()
- }
-
- val algorithm = new LinearRegressionWithSGD()
- algorithm.optimizer
- .setNumIterations(params.numIterations)
- .setStepSize(params.stepSize)
- .setUpdater(updater)
- .setRegParam(params.regParam)
-
- val model = algorithm.run(training)
-
- val prediction = model.predict(test.map(_.features))
- val predictionAndLabel = prediction.zip(test.map(_.label))
-
- val loss = predictionAndLabel.map { case (p, l) =>
- val err = p - l
- err * err
- }.reduce(_ + _)
- val rmse = math.sqrt(loss / numTest)
-
- println(s"Test RMSE = $rmse.")
-
- sc.stop()
- }
-}
-// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala
deleted file mode 100644
index 449b725d1d173..0000000000000
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegressionWithSGDExample.scala
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-// scalastyle:off println
-package org.apache.spark.examples.mllib
-
-import org.apache.spark.{SparkConf, SparkContext}
-// $example on$
-import org.apache.spark.mllib.linalg.Vectors
-import org.apache.spark.mllib.regression.LabeledPoint
-import org.apache.spark.mllib.regression.LinearRegressionModel
-import org.apache.spark.mllib.regression.LinearRegressionWithSGD
-// $example off$
-
-@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0")
-object LinearRegressionWithSGDExample {
-
- def main(args: Array[String]): Unit = {
- val conf = new SparkConf().setAppName("LinearRegressionWithSGDExample")
- val sc = new SparkContext(conf)
-
- // $example on$
- // Load and parse the data
- val data = sc.textFile("data/mllib/ridge-data/lpsa.data")
- val parsedData = data.map { line =>
- val parts = line.split(',')
- LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble)))
- }.cache()
-
- // Building the model
- val numIterations = 100
- val stepSize = 0.00000001
- val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize)
-
- // Evaluate model on training examples and compute training error
- val valuesAndPreds = parsedData.map { point =>
- val prediction = model.predict(point.features)
- (point.label, prediction)
- }
- val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2) }.mean()
- println(s"training Mean Squared Error $MSE")
-
- // Save and load model
- model.save(sc, "target/tmp/scalaLinearRegressionWithSGDModel")
- val sameModel = LinearRegressionModel.load(sc, "target/tmp/scalaLinearRegressionWithSGDModel")
- // $example off$
-
- sc.stop()
- }
-}
-// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala
deleted file mode 100644
index eff2393cc3abe..0000000000000
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/PCAExample.scala
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-// scalastyle:off println
-package org.apache.spark.examples.mllib
-
-import org.apache.spark.SparkConf
-import org.apache.spark.SparkContext
-// $example on$
-import org.apache.spark.mllib.feature.PCA
-import org.apache.spark.mllib.linalg.Vectors
-import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD}
-// $example off$
-
-@deprecated("Deprecated since LinearRegressionWithSGD is deprecated. Use ml.feature.PCA", "2.0.0")
-object PCAExample {
-
- def main(args: Array[String]): Unit = {
-
- val conf = new SparkConf().setAppName("PCAExample")
- val sc = new SparkContext(conf)
-
- // $example on$
- val data = sc.textFile("data/mllib/ridge-data/lpsa.data").map { line =>
- val parts = line.split(',')
- LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble)))
- }.cache()
-
- val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L)
- val training = splits(0).cache()
- val test = splits(1)
-
- val pca = new PCA(training.first().features.size / 2).fit(data.map(_.features))
- val training_pca = training.map(p => p.copy(features = pca.transform(p.features)))
- val test_pca = test.map(p => p.copy(features = pca.transform(p.features)))
-
- val numIterations = 100
- val model = LinearRegressionWithSGD.train(training, numIterations)
- val model_pca = LinearRegressionWithSGD.train(training_pca, numIterations)
-
- val valuesAndPreds = test.map { point =>
- val score = model.predict(point.features)
- (score, point.label)
- }
-
- val valuesAndPreds_pca = test_pca.map { point =>
- val score = model_pca.predict(point.features)
- (score, point.label)
- }
-
- val MSE = valuesAndPreds.map { case (v, p) => math.pow((v - p), 2) }.mean()
- val MSE_pca = valuesAndPreds_pca.map { case (v, p) => math.pow((v - p), 2) }.mean()
-
- println(s"Mean Squared Error = $MSE")
- println(s"PCA Mean Squared Error = $MSE_pca")
- // $example off$
-
- sc.stop()
- }
-}
-// scalastyle:on println
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala
deleted file mode 100644
index 76cfb804e18f3..0000000000000
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/RegressionMetricsExample.scala
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-// scalastyle:off println
-
-package org.apache.spark.examples.mllib
-
-// $example on$
-import org.apache.spark.mllib.evaluation.RegressionMetrics
-import org.apache.spark.mllib.linalg.Vector
-import org.apache.spark.mllib.regression.{LabeledPoint, LinearRegressionWithSGD}
-// $example off$
-import org.apache.spark.sql.SparkSession
-
-@deprecated("Use ml.regression.LinearRegression and the resulting model summary for metrics",
- "2.0.0")
-object RegressionMetricsExample {
- def main(args: Array[String]): Unit = {
- val spark = SparkSession
- .builder
- .appName("RegressionMetricsExample")
- .getOrCreate()
- // $example on$
- // Load the data
- val data = spark
- .read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt")
- .rdd.map(row => LabeledPoint(row.getDouble(0), row.get(1).asInstanceOf[Vector]))
- .cache()
-
- // Build the model
- val numIterations = 100
- val model = LinearRegressionWithSGD.train(data, numIterations)
-
- // Get predictions
- val valuesAndPreds = data.map{ point =>
- val prediction = model.predict(point.features)
- (prediction, point.label)
- }
-
- // Instantiate metrics object
- val metrics = new RegressionMetrics(valuesAndPreds)
-
- // Squared error
- println(s"MSE = ${metrics.meanSquaredError}")
- println(s"RMSE = ${metrics.rootMeanSquaredError}")
-
- // R-squared
- println(s"R-squared = ${metrics.r2}")
-
- // Mean absolute error
- println(s"MAE = ${metrics.meanAbsoluteError}")
-
- // Explained variance
- println(s"Explained variance = ${metrics.explainedVariance}")
- // $example off$
-
- spark.stop()
- }
-}
-// scalastyle:on println
-
diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java
index 86c42df9e8435..31ca2fe5c95ff 100644
--- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java
+++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java
@@ -32,13 +32,14 @@
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.kinesis.KinesisUtils;
+import org.apache.spark.streaming.kinesis.KinesisInitialPositions;
+import org.apache.spark.streaming.kinesis.KinesisInputDStream;
import scala.Tuple2;
+import scala.reflect.ClassTag$;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.services.kinesis.AmazonKinesisClient;
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
/**
* Consumes messages from a Amazon Kinesis streams and does wordcount.
@@ -135,11 +136,19 @@ public static void main(String[] args) throws Exception {
// Create the Kinesis DStreams
List> streamsList = new ArrayList<>(numStreams);
for (int i = 0; i < numStreams; i++) {
- streamsList.add(
- KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName,
- InitialPositionInStream.LATEST, kinesisCheckpointInterval,
- StorageLevel.MEMORY_AND_DISK_2())
- );
+ streamsList.add(JavaDStream.fromDStream(
+ KinesisInputDStream.builder()
+ .streamingContext(jssc)
+ .checkpointAppName(kinesisAppName)
+ .streamName(streamName)
+ .endpointUrl(endpointUrl)
+ .regionName(regionName)
+ .initialPosition(new KinesisInitialPositions.Latest())
+ .checkpointInterval(kinesisCheckpointInterval)
+ .storageLevel(StorageLevel.MEMORY_AND_DISK_2())
+ .build(),
+ ClassTag$.MODULE$.apply(byte[].class)
+ ));
}
// Union all the streams if there is more than 1 stream
diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala
deleted file mode 100644
index 4e796b2caec87..0000000000000
--- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala
+++ /dev/null
@@ -1,642 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.streaming.kinesis
-
-import scala.reflect.ClassTag
-
-import com.amazonaws.regions.RegionUtils
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
-import com.amazonaws.services.kinesis.model.Record
-
-import org.apache.spark.api.java.function.{Function => JFunction}
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Duration, StreamingContext}
-import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext}
-import org.apache.spark.streaming.dstream.ReceiverInputDStream
-
-object KinesisUtils {
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param ssc StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- *
- * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
- * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
- * gets the AWS credentials.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T: ClassTag](
- ssc: StreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: Record => T): ReceiverInputDStream[T] = {
- val cleanedHandler = ssc.sc.clean(messageHandler)
- // Setting scope to override receiver stream's scope of "receiver stream"
- ssc.withNamedScope("kinesis stream") {
- new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
- KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream),
- kinesisAppName, checkpointInterval, storageLevel,
- cleanedHandler, DefaultCredentials, None, None,
- KinesisInputDStream.DEFAULT_METRICS_LEVEL,
- KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS)
- }
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param ssc StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- // scalastyle:off
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T: ClassTag](
- ssc: StreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: Record => T,
- awsAccessKeyId: String,
- awsSecretKey: String): ReceiverInputDStream[T] = {
- // scalastyle:on
- val cleanedHandler = ssc.sc.clean(messageHandler)
- ssc.withNamedScope("kinesis stream") {
- val kinesisCredsProvider = BasicCredentials(
- awsAccessKeyId = awsAccessKeyId,
- awsSecretKey = awsSecretKey)
- new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
- KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream),
- kinesisAppName, checkpointInterval, storageLevel,
- cleanedHandler, kinesisCredsProvider, None, None,
- KinesisInputDStream.DEFAULT_METRICS_LEVEL,
- KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS)
- }
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param ssc StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from
- * Kinesis stream.
- * @param stsSessionName Name to uniquely identify STS sessions if multiple principals assume
- * the same role.
- * @param stsExternalId External ID that can be used to validate against the assumed IAM role's
- * trust policy.
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- // scalastyle:off
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T: ClassTag](
- ssc: StreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: Record => T,
- awsAccessKeyId: String,
- awsSecretKey: String,
- stsAssumeRoleArn: String,
- stsSessionName: String,
- stsExternalId: String): ReceiverInputDStream[T] = {
- // scalastyle:on
- val cleanedHandler = ssc.sc.clean(messageHandler)
- ssc.withNamedScope("kinesis stream") {
- val kinesisCredsProvider = STSCredentials(
- stsRoleArn = stsAssumeRoleArn,
- stsSessionName = stsSessionName,
- stsExternalId = Option(stsExternalId),
- longLivedCreds = BasicCredentials(
- awsAccessKeyId = awsAccessKeyId,
- awsSecretKey = awsSecretKey))
- new KinesisInputDStream[T](ssc, streamName, endpointUrl, validateRegion(regionName),
- KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream),
- kinesisAppName, checkpointInterval, storageLevel,
- cleanedHandler, kinesisCredsProvider, None, None,
- KinesisInputDStream.DEFAULT_METRICS_LEVEL,
- KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS)
- }
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param ssc StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- *
- * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
- * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
- * gets the AWS credentials.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream(
- ssc: StreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = {
- // Setting scope to override receiver stream's scope of "receiver stream"
- ssc.withNamedScope("kinesis stream") {
- new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName),
- KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream),
- kinesisAppName, checkpointInterval, storageLevel,
- KinesisInputDStream.defaultMessageHandler, DefaultCredentials, None, None,
- KinesisInputDStream.DEFAULT_METRICS_LEVEL,
- KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS)
- }
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param ssc StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream(
- ssc: StreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- awsAccessKeyId: String,
- awsSecretKey: String): ReceiverInputDStream[Array[Byte]] = {
- ssc.withNamedScope("kinesis stream") {
- val kinesisCredsProvider = BasicCredentials(
- awsAccessKeyId = awsAccessKeyId,
- awsSecretKey = awsSecretKey)
- new KinesisInputDStream[Array[Byte]](ssc, streamName, endpointUrl, validateRegion(regionName),
- KinesisInitialPositions.fromKinesisInitialPosition(initialPositionInStream),
- kinesisAppName, checkpointInterval, storageLevel,
- KinesisInputDStream.defaultMessageHandler, kinesisCredsProvider, None, None,
- KinesisInputDStream.DEFAULT_METRICS_LEVEL,
- KinesisInputDStream.DEFAULT_METRICS_ENABLED_DIMENSIONS)
- }
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param jssc Java StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- * @param recordClass Class of the records in DStream
- *
- * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
- * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
- * gets the AWS credentials.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T](
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: JFunction[Record, T],
- recordClass: Class[T]): JavaReceiverInputDStream[T] = {
- implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
- val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
- createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler)
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param jssc Java StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- * @param recordClass Class of the records in DStream
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- // scalastyle:off
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T](
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: JFunction[Record, T],
- recordClass: Class[T],
- awsAccessKeyId: String,
- awsSecretKey: String): JavaReceiverInputDStream[T] = {
- // scalastyle:on
- implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
- val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
- createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler,
- awsAccessKeyId, awsSecretKey)
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param jssc Java StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param messageHandler A custom message handler that can generate a generic output from a
- * Kinesis `Record`, which contains both message data, and metadata.
- * @param recordClass Class of the records in DStream
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- * @param stsAssumeRoleArn ARN of IAM role to assume when using STS sessions to read from
- * Kinesis stream.
- * @param stsSessionName Name to uniquely identify STS sessions if multiple princpals assume
- * the same role.
- * @param stsExternalId External ID that can be used to validate against the assumed IAM role's
- * trust policy.
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- // scalastyle:off
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream[T](
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- messageHandler: JFunction[Record, T],
- recordClass: Class[T],
- awsAccessKeyId: String,
- awsSecretKey: String,
- stsAssumeRoleArn: String,
- stsSessionName: String,
- stsExternalId: String): JavaReceiverInputDStream[T] = {
- // scalastyle:on
- implicit val recordCmt: ClassTag[T] = ClassTag(recordClass)
- val cleanedHandler = jssc.sparkContext.clean(messageHandler.call(_))
- createStream[T](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- initialPositionInStream, checkpointInterval, storageLevel, cleanedHandler,
- awsAccessKeyId, awsSecretKey, stsAssumeRoleArn, stsSessionName, stsExternalId)
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param jssc Java StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- *
- * @note The AWS credentials will be discovered using the DefaultAWSCredentialsProviderChain
- * on the workers. See AWS documentation to understand how DefaultAWSCredentialsProviderChain
- * gets the AWS credentials.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream(
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel
- ): JavaReceiverInputDStream[Array[Byte]] = {
- createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- initialPositionInStream, checkpointInterval, storageLevel,
- KinesisInputDStream.defaultMessageHandler(_))
- }
-
- /**
- * Create an input stream that pulls messages from a Kinesis stream.
- * This uses the Kinesis Client Library (KCL) to pull messages from Kinesis.
- *
- * @param jssc Java StreamingContext object
- * @param kinesisAppName Kinesis application name used by the Kinesis Client Library
- * (KCL) to update DynamoDB
- * @param streamName Kinesis stream name
- * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com)
- * @param regionName Name of region used by the Kinesis Client Library (KCL) to update
- * DynamoDB (lease coordination and checkpointing) and CloudWatch (metrics)
- * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the
- * worker's initial starting position in the stream.
- * The values are either the beginning of the stream
- * per Kinesis' limit of 24 hours
- * (InitialPositionInStream.TRIM_HORIZON) or
- * the tip of the stream (InitialPositionInStream.LATEST).
- * @param checkpointInterval Checkpoint interval for Kinesis checkpointing.
- * See the Kinesis Spark Streaming documentation for more
- * details on the different types of checkpoints.
- * @param storageLevel Storage level to use for storing the received objects.
- * StorageLevel.MEMORY_AND_DISK_2 is recommended.
- * @param awsAccessKeyId AWS AccessKeyId (if null, will use DefaultAWSCredentialsProviderChain)
- * @param awsSecretKey AWS SecretKey (if null, will use DefaultAWSCredentialsProviderChain)
- *
- * @note The given AWS credentials will get saved in DStream checkpoints if checkpointing
- * is enabled. Make sure that your checkpoint directory is secure.
- */
- @deprecated("Use KinesisInputDStream.builder instead", "2.2.0")
- def createStream(
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: InitialPositionInStream,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- awsAccessKeyId: String,
- awsSecretKey: String): JavaReceiverInputDStream[Array[Byte]] = {
- createStream[Array[Byte]](jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- initialPositionInStream, checkpointInterval, storageLevel,
- KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey)
- }
-
- private def validateRegion(regionName: String): String = {
- Option(RegionUtils.getRegion(regionName)).map { _.getName }.getOrElse {
- throw new IllegalArgumentException(s"Region name '$regionName' is not valid")
- }
- }
-}
-
-/**
- * This is a helper class that wraps the methods in KinesisUtils into more Python-friendly class and
- * function so that it can be easily instantiated and called from Python's KinesisUtils.
- */
-private class KinesisUtilsPythonHelper {
-
- def getInitialPositionInStream(initialPositionInStream: Int): InitialPositionInStream = {
- initialPositionInStream match {
- case 0 => InitialPositionInStream.LATEST
- case 1 => InitialPositionInStream.TRIM_HORIZON
- case _ => throw new IllegalArgumentException(
- "Illegal InitialPositionInStream. Please use " +
- "InitialPositionInStream.LATEST or InitialPositionInStream.TRIM_HORIZON")
- }
- }
-
- // scalastyle:off
- def createStream(
- jssc: JavaStreamingContext,
- kinesisAppName: String,
- streamName: String,
- endpointUrl: String,
- regionName: String,
- initialPositionInStream: Int,
- checkpointInterval: Duration,
- storageLevel: StorageLevel,
- awsAccessKeyId: String,
- awsSecretKey: String,
- stsAssumeRoleArn: String,
- stsSessionName: String,
- stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = {
- // scalastyle:on
- if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null)
- && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) {
- throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " +
- "must all be defined or all be null")
- }
-
- if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) {
- validateAwsCreds(awsAccessKeyId, awsSecretKey)
- KinesisUtils.createStream(jssc.ssc, kinesisAppName, streamName, endpointUrl, regionName,
- getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel,
- KinesisInputDStream.defaultMessageHandler(_), awsAccessKeyId, awsSecretKey,
- stsAssumeRoleArn, stsSessionName, stsExternalId)
- } else {
- validateAwsCreds(awsAccessKeyId, awsSecretKey)
- if (awsAccessKeyId == null && awsSecretKey == null) {
- KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName,
- getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel)
- } else {
- KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName,
- getInitialPositionInStream(initialPositionInStream), checkpointInterval, storageLevel,
- awsAccessKeyId, awsSecretKey)
- }
- }
- }
-
- // Throw IllegalArgumentException unless both values are null or neither are.
- private def validateAwsCreds(awsAccessKeyId: String, awsSecretKey: String) {
- if (awsAccessKeyId == null && awsSecretKey != null) {
- throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null")
- }
- if (awsAccessKeyId != null && awsSecretKey == null) {
- throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null")
- }
- }
-}
diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala
new file mode 100644
index 0000000000000..c89dedd3366d1
--- /dev/null
+++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtilsPythonHelper.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.streaming.kinesis
+
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.Duration
+import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext}
+
+/**
+ * This is a helper class that wraps the methods in KinesisUtils into more Python-friendly class and
+ * function so that it can be easily instantiated and called from Python's KinesisUtils.
+ */
+private class KinesisUtilsPythonHelper {
+
+ // scalastyle:off
+ def createStream(
+ jssc: JavaStreamingContext,
+ kinesisAppName: String,
+ streamName: String,
+ endpointUrl: String,
+ regionName: String,
+ initialPositionInStream: Int,
+ checkpointInterval: Duration,
+ storageLevel: StorageLevel,
+ awsAccessKeyId: String,
+ awsSecretKey: String,
+ stsAssumeRoleArn: String,
+ stsSessionName: String,
+ stsExternalId: String): JavaReceiverInputDStream[Array[Byte]] = {
+ // scalastyle:on
+ if (!(stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null)
+ && !(stsAssumeRoleArn == null && stsSessionName == null && stsExternalId == null)) {
+ throw new IllegalArgumentException("stsAssumeRoleArn, stsSessionName, and stsExtenalId " +
+ "must all be defined or all be null")
+ }
+ if (awsAccessKeyId == null && awsSecretKey != null) {
+ throw new IllegalArgumentException("awsSecretKey is set but awsAccessKeyId is null")
+ }
+ if (awsAccessKeyId != null && awsSecretKey == null) {
+ throw new IllegalArgumentException("awsAccessKeyId is set but awsSecretKey is null")
+ }
+
+ val kinesisInitialPosition = initialPositionInStream match {
+ case 0 => InitialPositionInStream.LATEST
+ case 1 => InitialPositionInStream.TRIM_HORIZON
+ case _ => throw new IllegalArgumentException(
+ "Illegal InitialPositionInStream. Please use " +
+ "InitialPositionInStream.LATEST or InitialPositionInStream.TRIM_HORIZON")
+ }
+
+ val builder = KinesisInputDStream.builder.
+ streamingContext(jssc).
+ checkpointAppName(kinesisAppName).
+ streamName(streamName).
+ endpointUrl(endpointUrl).
+ regionName(regionName).
+ initialPosition(KinesisInitialPositions.fromKinesisInitialPosition(kinesisInitialPosition)).
+ checkpointInterval(checkpointInterval).
+ storageLevel(storageLevel)
+
+ if (stsAssumeRoleArn != null && stsSessionName != null && stsExternalId != null) {
+ val kinesisCredsProvider = STSCredentials(
+ stsAssumeRoleArn, stsSessionName, Option(stsExternalId),
+ BasicCredentials(awsAccessKeyId, awsSecretKey))
+ builder.
+ kinesisCredentials(kinesisCredsProvider).
+ buildWithMessageHandler(KinesisInputDStream.defaultMessageHandler)
+ } else {
+ if (awsAccessKeyId == null && awsSecretKey == null) {
+ builder.build()
+ } else {
+ builder.kinesisCredentials(BasicCredentials(awsAccessKeyId, awsSecretKey)).build()
+ }
+ }
+ }
+
+}
diff --git a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java
deleted file mode 100644
index b37b087467926..0000000000000
--- a/external/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming.kinesis;
-
-import com.amazonaws.services.kinesis.model.Record;
-import org.junit.Test;
-
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.streaming.Duration;
-import org.apache.spark.streaming.LocalJavaStreamingContext;
-import org.apache.spark.streaming.api.java.JavaDStream;
-
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream;
-
-/**
- * Demonstrate the use of the KinesisUtils Java API
- */
-public class JavaKinesisStreamSuite extends LocalJavaStreamingContext {
- @Test
- public void testKinesisStream() {
- String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl();
- String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl);
-
- // Tests the API, does not actually test data receiving
- JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
- dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000),
- StorageLevel.MEMORY_AND_DISK_2());
- ssc.stop();
- }
-
- @Test
- public void testAwsCreds() {
- String dummyEndpointUrl = KinesisTestUtils.defaultEndpointUrl();
- String dummyRegionName = KinesisTestUtils.getRegionNameByEndpoint(dummyEndpointUrl);
-
- // Tests the API, does not actually test data receiving
- JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
- dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, new Duration(2000),
- StorageLevel.MEMORY_AND_DISK_2(), "fakeAccessKey", "fakeSecretKey");
- ssc.stop();
- }
-
- private static Function handler = new Function() {
- @Override
- public String call(Record record) {
- return record.getPartitionKey() + "-" + record.getSequenceNumber();
- }
- };
-
- @Test
- public void testCustomHandler() {
- // Tests the API, does not actually test data receiving
- JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream",
- "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST,
- new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class);
-
- ssc.stop();
- }
-
- @Test
- public void testCustomHandlerAwsCreds() {
- // Tests the API, does not actually test data receiving
- JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream",
- "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST,
- new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class,
- "fakeAccessKey", "fakeSecretKey");
-
- ssc.stop();
- }
-
- @Test
- public void testCustomHandlerAwsStsCreds() {
- // Tests the API, does not actually test data receiving
- JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "testApp", "mySparkStream",
- "https://kinesis.us-west-2.amazonaws.com", "us-west-2", InitialPositionInStream.LATEST,
- new Duration(2000), StorageLevel.MEMORY_AND_DISK_2(), handler, String.class,
- "fakeAccessKey", "fakeSecretKey", "fakeSTSRoleArn", "fakeSTSSessionName",
- "fakeSTSExternalId");
-
- ssc.stop();
- }
-}
diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala
index 51ee7fd213de5..ee53fbadf8887 100644
--- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala
+++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala
@@ -21,7 +21,6 @@ import scala.collection.mutable
import scala.concurrent.duration._
import scala.util.Random
-import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
import com.amazonaws.services.kinesis.model.Record
import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll}
import org.scalatest.Matchers._
@@ -102,20 +101,17 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun
}
}
- test("KinesisUtils API") {
- val kinesisStream1 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
- dummyEndpointUrl, dummyRegionName,
- InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2)
- val kinesisStream2 = KinesisUtils.createStream(ssc, "myAppName", "mySparkStream",
- dummyEndpointUrl, dummyRegionName,
- InitialPositionInStream.LATEST, Seconds(2), StorageLevel.MEMORY_AND_DISK_2,
- dummyAWSAccessKey, dummyAWSSecretKey)
- }
-
test("RDD generation") {
- val inputStream = KinesisUtils.createStream(ssc, appName, "dummyStream",
- dummyEndpointUrl, dummyRegionName, InitialPositionInStream.LATEST, Seconds(2),
- StorageLevel.MEMORY_AND_DISK_2, dummyAWSAccessKey, dummyAWSSecretKey)
+ val inputStream = KinesisInputDStream.builder.
+ streamingContext(ssc).
+ checkpointAppName(appName).
+ streamName("dummyStream").
+ endpointUrl(dummyEndpointUrl).
+ regionName(dummyRegionName).initialPosition(new Latest()).
+ checkpointInterval(Seconds(2)).
+ storageLevel(StorageLevel.MEMORY_AND_DISK_2).
+ kinesisCredentials(BasicCredentials(dummyAWSAccessKey, dummyAWSSecretKey)).
+ build()
assert(inputStream.isInstanceOf[KinesisInputDStream[Array[Byte]]])
val kinesisStream = inputStream.asInstanceOf[KinesisInputDStream[Array[Byte]]]
diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
index 3479e0c3422bd..f3805f70ea30d 100644
--- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
+++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java
@@ -392,7 +392,7 @@ boolean isClientMode(Map userProps) {
// Default master is "local[*]", so assume client mode in that case
return userMaster == null ||
"client".equals(userDeployMode) ||
- (!userMaster.equals("yarn-cluster") && userDeployMode == null);
+ (!userMaster.equals("yarn") && userDeployMode == null);
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index 4617073f9decd..f6e51ab7f84f8 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -347,7 +347,6 @@ private[python] class PythonMLLibAPI extends Serializable {
data: JavaRDD[Vector],
k: Int,
maxIterations: Int,
- runs: Int,
initializationMode: String,
seed: java.lang.Long,
initializationSteps: Int,
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index d86aa01c9195a..df888bc3d5d51 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -224,117 +224,11 @@ class LogisticRegressionWithSGD private[mllib] (
.setMiniBatchFraction(miniBatchFraction)
override protected val validators = List(DataValidators.binaryLabelValidator)
- /**
- * Construct a LogisticRegression object with default parameters: {stepSize: 1.0,
- * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}.
- */
- @Since("0.8.0")
- @deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0")
- def this() = this(1.0, 100, 0.01, 1.0)
-
override protected[mllib] def createModel(weights: Vector, intercept: Double) = {
new LogisticRegressionModel(weights, intercept)
}
}
-/**
- * Top-level methods for calling Logistic Regression using Stochastic Gradient Descent.
- *
- * @note Labels used in Logistic Regression should be {0, 1}
- */
-@Since("0.8.0")
-@deprecated("Use ml.classification.LogisticRegression or LogisticRegressionWithLBFGS", "2.0.0")
-object LogisticRegressionWithSGD {
- // NOTE(shivaram): We use multiple train methods instead of default arguments to support
- // Java programs.
-
- /**
- * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed
- * number of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in
- * gradient descent are initialized using the initial weights provided.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
- * the number of features in the data.
- *
- * @note Labels used in Logistic Regression should be {0, 1}
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- miniBatchFraction: Double,
- initialWeights: Vector): LogisticRegressionModel = {
- new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction)
- .run(input, initialWeights)
- }
-
- /**
- * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed
- * number of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate the gradient.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- *
- * @note Labels used in Logistic Regression should be {0, 1}
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- miniBatchFraction: Double): LogisticRegressionModel = {
- new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction)
- .run(input)
- }
-
- /**
- * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed
- * number of iterations of gradient descent using the specified step size. We use the entire data
- * set to update the gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param stepSize Step size to be used for each iteration of Gradient Descent.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LogisticRegressionModel which has the weights and offset from training.
- *
- * @note Labels used in Logistic Regression should be {0, 1}
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double): LogisticRegressionModel = {
- train(input, numIterations, stepSize, 1.0)
- }
-
- /**
- * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed
- * number of iterations of gradient descent using a step size of 1.0. We use the entire data set
- * to update the gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LogisticRegressionModel which has the weights and offset from training.
- *
- * @note Labels used in Logistic Regression should be {0, 1}
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int): LogisticRegressionModel = {
- train(input, numIterations, 1.0, 1.0)
- }
-}
-
/**
* Train a classification model for Multinomial/Binary Logistic Regression using
* Limited-memory BFGS. Standard feature scaling and L2 regularization are used by default.
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
index 4bb79bc69eef4..278d61d916735 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
@@ -479,58 +479,6 @@ object KMeans {
.run(data)
}
- /**
- * Trains a k-means model using the given set of parameters.
- *
- * @param data Training points as an `RDD` of `Vector` types.
- * @param k Number of clusters to create.
- * @param maxIterations Maximum number of iterations allowed.
- * @param runs This param has no effect since Spark 2.0.0.
- * @param initializationMode The initialization algorithm. This can either be "random" or
- * "k-means||". (default: "k-means||")
- * @param seed Random seed for cluster initialization. Default is to generate seed based
- * on system time.
- */
- @Since("1.3.0")
- @deprecated("Use train method without 'runs'", "2.1.0")
- def train(
- data: RDD[Vector],
- k: Int,
- maxIterations: Int,
- runs: Int,
- initializationMode: String,
- seed: Long): KMeansModel = {
- new KMeans().setK(k)
- .setMaxIterations(maxIterations)
- .setInitializationMode(initializationMode)
- .setSeed(seed)
- .run(data)
- }
-
- /**
- * Trains a k-means model using the given set of parameters.
- *
- * @param data Training points as an `RDD` of `Vector` types.
- * @param k Number of clusters to create.
- * @param maxIterations Maximum number of iterations allowed.
- * @param runs This param has no effect since Spark 2.0.0.
- * @param initializationMode The initialization algorithm. This can either be "random" or
- * "k-means||". (default: "k-means||")
- */
- @Since("0.8.0")
- @deprecated("Use train method without 'runs'", "2.1.0")
- def train(
- data: RDD[Vector],
- k: Int,
- maxIterations: Int,
- runs: Int,
- initializationMode: String): KMeansModel = {
- new KMeans().setK(k)
- .setMaxIterations(maxIterations)
- .setInitializationMode(initializationMode)
- .run(data)
- }
-
/**
* Trains a k-means model using specified parameters and the default values for unspecified.
*/
@@ -544,21 +492,6 @@ object KMeans {
.run(data)
}
- /**
- * Trains a k-means model using specified parameters and the default values for unspecified.
- */
- @Since("0.8.0")
- @deprecated("Use train method without 'runs'", "2.1.0")
- def train(
- data: RDD[Vector],
- k: Int,
- maxIterations: Int,
- runs: Int): KMeansModel = {
- new KMeans().setK(k)
- .setMaxIterations(maxIterations)
- .run(data)
- }
-
private[spark] def validateInitMode(initMode: String): Boolean = {
initMode match {
case KMeans.RANDOM => true
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala
index 82f5b279846ba..b771e077b02ac 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala
@@ -44,17 +44,6 @@ class ChiSqSelectorModel @Since("1.3.0") (
private val filterIndices = selectedFeatures.sorted
- @deprecated("not intended for subclasses to use", "2.1.0")
- protected def isSorted(array: Array[Int]): Boolean = {
- var i = 1
- val len = array.length
- while (i < len) {
- if (array(i) < array(i-1)) return false
- i += 1
- }
- true
- }
-
/**
* Applies transformation on a vector.
*
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index ead9f5b300375..47bb1fa9127a6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -24,7 +24,6 @@ import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.pmml.PMMLExportable
import org.apache.spark.mllib.regression.impl.GLMRegressionModel
import org.apache.spark.mllib.util.{Loader, Saveable}
-import org.apache.spark.rdd.RDD
/**
* Regression model trained using Lasso.
@@ -99,117 +98,7 @@ class LassoWithSGD private[mllib] (
.setRegParam(regParam)
.setMiniBatchFraction(miniBatchFraction)
- /**
- * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100,
- * regParam: 0.01, miniBatchFraction: 1.0}.
- */
- @Since("0.8.0")
- @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " +
- "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0")
- def this() = this(1.0, 100, 0.01, 1.0)
-
override protected def createModel(weights: Vector, intercept: Double) = {
new LassoModel(weights, intercept)
}
}
-
-/**
- * Top-level methods for calling Lasso.
- *
- */
-@Since("0.8.0")
-@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 1.0. Note the default " +
- "regParam is 0.01 for LassoWithSGD, but is 0.0 for LinearRegression.", "2.0.0")
-object LassoWithSGD {
-
- /**
- * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used
- * in gradient descent are initialized using the initial weights provided.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size scaling to be used for the iterations of gradient descent.
- * @param regParam Regularization parameter.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
- * the number of features in the data.
- *
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double,
- miniBatchFraction: Double,
- initialWeights: Vector): LassoModel = {
- new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction)
- .run(input, initialWeights)
- }
-
- /**
- * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param regParam Regularization parameter.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double,
- miniBatchFraction: Double): LassoModel = {
- new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
- }
-
- /**
- * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. We use the entire data set to
- * update the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param stepSize Step size to be used for each iteration of Gradient Descent.
- * @param regParam Regularization parameter.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LassoModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double): LassoModel = {
- train(input, numIterations, stepSize, regParam, 1.0)
- }
-
- /**
- * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using a step size of 1.0. We use the entire data set to
- * compute the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LassoModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int): LassoModel = {
- train(input, numIterations, 1.0, 0.01, 1.0)
- }
-}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index cb08216fbf690..f68ebc17e294d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -24,7 +24,6 @@ import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.pmml.PMMLExportable
import org.apache.spark.mllib.regression.impl.GLMRegressionModel
import org.apache.spark.mllib.util.{Loader, Saveable}
-import org.apache.spark.rdd.RDD
/**
* Regression model trained using LinearRegression.
@@ -100,109 +99,8 @@ class LinearRegressionWithSGD private[mllib] (
.setRegParam(regParam)
.setMiniBatchFraction(miniBatchFraction)
- /**
- * Construct a LinearRegression object with default parameters: {stepSize: 1.0,
- * numIterations: 100, miniBatchFraction: 1.0}.
- */
- @Since("0.8.0")
- @deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0")
- def this() = this(1.0, 100, 0.0, 1.0)
-
override protected[mllib] def createModel(weights: Vector, intercept: Double) = {
new LinearRegressionModel(weights, intercept)
}
}
-/**
- * Top-level methods for calling LinearRegression.
- *
- */
-@Since("0.8.0")
-@deprecated("Use ml.regression.LinearRegression or LBFGS", "2.0.0")
-object LinearRegressionWithSGD {
-
- /**
- * Train a Linear Regression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used
- * in gradient descent are initialized using the initial weights provided.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
- * the number of features in the data.
- *
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- miniBatchFraction: Double,
- initialWeights: Vector): LinearRegressionModel = {
- new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction)
- .run(input, initialWeights)
- }
-
- /**
- * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- miniBatchFraction: Double): LinearRegressionModel = {
- new LinearRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run(input)
- }
-
- /**
- * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. We use the entire data set to
- * compute the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param stepSize Step size to be used for each iteration of Gradient Descent.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LinearRegressionModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double): LinearRegressionModel = {
- train(input, numIterations, stepSize, 1.0)
- }
-
- /**
- * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using a step size of 1.0. We use the entire data set to
- * compute the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs. Each pair describes a row of the data
- * matrix A as well as the corresponding right hand side label y
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a LinearRegressionModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int): LinearRegressionModel = {
- train(input, numIterations, 1.0, 1.0)
- }
-}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index 43c3154dd053b..1c3bdceab1d14 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -24,8 +24,6 @@ import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.pmml.PMMLExportable
import org.apache.spark.mllib.regression.impl.GLMRegressionModel
import org.apache.spark.mllib.util.{Loader, Saveable}
-import org.apache.spark.rdd.RDD
-
/**
* Regression model trained using RidgeRegression.
@@ -100,113 +98,7 @@ class RidgeRegressionWithSGD private[mllib] (
.setRegParam(regParam)
.setMiniBatchFraction(miniBatchFraction)
- /**
- * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100,
- * regParam: 0.01, miniBatchFraction: 1.0}.
- */
- @Since("0.8.0")
- @deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " +
- "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0")
- def this() = this(1.0, 100, 0.01, 1.0)
-
override protected def createModel(weights: Vector, intercept: Double) = {
new RidgeRegressionModel(weights, intercept)
}
}
-
-/**
- * Top-level methods for calling RidgeRegression.
- *
- */
-@Since("0.8.0")
-@deprecated("Use ml.regression.LinearRegression with elasticNetParam = 0.0. Note the default " +
- "regParam is 0.01 for RidgeRegressionWithSGD, but is 0.0 for LinearRegression.", "2.0.0")
-object RidgeRegressionWithSGD {
-
- /**
- * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient. The weights used
- * in gradient descent are initialized using the initial weights provided.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param regParam Regularization parameter.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
- * the number of features in the data.
- *
- */
- @Since("1.0.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double,
- miniBatchFraction: Double,
- initialWeights: Vector): RidgeRegressionModel = {
- new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(
- input, initialWeights)
- }
-
- /**
- * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. Each iteration uses
- * `miniBatchFraction` fraction of the data to calculate a stochastic gradient.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @param stepSize Step size to be used for each iteration of gradient descent.
- * @param regParam Regularization parameter.
- * @param miniBatchFraction Fraction of data to be used per iteration.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double,
- miniBatchFraction: Double): RidgeRegressionModel = {
- new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
- }
-
- /**
- * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using the specified step size. We use the entire data set to
- * compute the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param stepSize Step size to be used for each iteration of Gradient Descent.
- * @param regParam Regularization parameter.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a RidgeRegressionModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int,
- stepSize: Double,
- regParam: Double): RidgeRegressionModel = {
- train(input, numIterations, stepSize, regParam, 1.0)
- }
-
- /**
- * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number
- * of iterations of gradient descent using a step size of 1.0. We use the entire data set to
- * compute the true gradient in each iteration.
- *
- * @param input RDD of (label, array of features) pairs.
- * @param numIterations Number of iterations of gradient descent to run.
- * @return a RidgeRegressionModel which has the weights and offset from training.
- *
- */
- @Since("0.8.0")
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int): RidgeRegressionModel = {
- train(input, numIterations, 1.0, 0.01, 1.0)
- }
-}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
index c04e2e69541ba..208a5aaa2bb15 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
@@ -50,11 +50,8 @@ public void runLRUsingConstructor() {
List validationData =
LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17);
- LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD();
+ LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(1.0, 100, 1.0, 1.0);
lrImpl.setIntercept(true);
- lrImpl.optimizer().setStepSize(1.0)
- .setRegParam(1.0)
- .setNumIterations(100);
LogisticRegressionModel model = lrImpl.run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
@@ -72,8 +69,8 @@ public void runLRUsingStaticMethods() {
List validationData =
LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17);
- LogisticRegressionModel model = LogisticRegressionWithSGD.train(
- testRDD.rdd(), 100, 1.0, 1.0);
+ LogisticRegressionModel model = new LogisticRegressionWithSGD(1.0, 100, 0.01, 1.0)
+ .run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0);
diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
index 270e636f82117..a9a8b7f2b88d6 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
@@ -42,11 +42,11 @@ public void runKMeansUsingStaticMethods() {
Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0);
JavaRDD data = jsc.parallelize(points, 2);
- KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL());
+ KMeansModel model = KMeans.train(data.rdd(), 1, 1, KMeans.K_MEANS_PARALLEL());
assertEquals(1, model.clusterCenters().length);
assertEquals(expectedCenter, model.clusterCenters()[0]);
- model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM());
+ model = KMeans.train(data.rdd(), 1, 1, KMeans.RANDOM());
assertEquals(expectedCenter, model.clusterCenters()[0]);
}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
index 1458cc72bc17f..35ad24bc2a84f 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
@@ -51,10 +51,7 @@ public void runLassoUsingConstructor() {
List validationData =
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1);
- LassoWithSGD lassoSGDImpl = new LassoWithSGD();
- lassoSGDImpl.optimizer().setStepSize(1.0)
- .setRegParam(0.01)
- .setNumIterations(20);
+ LassoWithSGD lassoSGDImpl = new LassoWithSGD(1.0, 20, 0.01, 1.0);
LassoModel model = lassoSGDImpl.run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
@@ -72,7 +69,7 @@ public void runLassoUsingStaticMethods() {
List validationData =
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1);
- LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0);
+ LassoModel model = new LassoWithSGD(1.0, 100, 0.01, 1.0).run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0);
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
index 86c723aa00746..7e87588c4f0f6 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
@@ -33,7 +33,7 @@ private static int validatePrediction(
List validationData, LinearRegressionModel model) {
int numAccurate = 0;
for (LabeledPoint point : validationData) {
- Double prediction = model.predict(point.features());
+ double prediction = model.predict(point.features());
// A prediction is off if the prediction is more than 0.5 away from expected value.
if (Math.abs(prediction - point.label()) <= 0.5) {
numAccurate++;
@@ -53,7 +53,7 @@ public void runLinearRegressionUsingConstructor() {
List validationData =
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1);
- LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD();
+ LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0);
linSGDImpl.setIntercept(true);
LinearRegressionModel model = linSGDImpl.run(testRDD.rdd());
@@ -72,7 +72,8 @@ public void runLinearRegressionUsingStaticMethods() {
List validationData =
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1);
- LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100);
+ LinearRegressionModel model = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0)
+ .run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0);
@@ -85,7 +86,7 @@ public void testPredictJavaRDD() {
double[] weights = {10, 10};
JavaRDD testRDD = jsc.parallelize(
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache();
- LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD();
+ LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0);
LinearRegressionModel model = linSGDImpl.run(testRDD.rdd());
JavaRDD vectors = testRDD.map(LabeledPoint::features);
JavaRDD predictions = model.predict(vectors);
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
index 5a9389c424b44..63441950cd18f 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
@@ -34,7 +34,7 @@ private static double predictionError(List validationData,
RidgeRegressionModel model) {
double errorSum = 0;
for (LabeledPoint point : validationData) {
- Double prediction = model.predict(point.features());
+ double prediction = model.predict(point.features());
errorSum += (prediction - point.label()) * (prediction - point.label());
}
return errorSum / validationData.size();
@@ -60,11 +60,7 @@ public void runRidgeRegressionUsingConstructor() {
new ArrayList<>(data.subList(0, numExamples)));
List validationData = data.subList(numExamples, 2 * numExamples);
- RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD();
- ridgeSGDImpl.optimizer()
- .setStepSize(1.0)
- .setRegParam(0.0)
- .setNumIterations(200);
+ RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(1.0, 200, 0.0, 1.0);
RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd());
double unRegularizedErr = predictionError(validationData, model);
@@ -85,10 +81,12 @@ public void runRidgeRegressionUsingStaticMethods() {
new ArrayList<>(data.subList(0, numExamples)));
List validationData = data.subList(numExamples, 2 * numExamples);
- RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0);
+ RidgeRegressionModel model = new RidgeRegressionWithSGD(1.0, 200, 0.0, 1.0)
+ .run(testRDD.rdd());
double unRegularizedErr = predictionError(validationData, model);
- model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.1);
+ model = new RidgeRegressionWithSGD(1.0, 200, 0.1, 1.0)
+ .run(testRDD.rdd());
double regularizedErr = predictionError(validationData, model);
Assert.assertTrue(regularizedErr < unRegularizedErr);
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index 5cf4377768516..126f0a23bfcb9 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -224,12 +224,8 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
- val lr = new LogisticRegressionWithSGD().setIntercept(true)
- lr.optimizer
- .setStepSize(10.0)
- .setRegParam(0.0)
- .setNumIterations(20)
- .setConvergenceTol(0.0005)
+ val lr = new LogisticRegressionWithSGD(10.0, 20, 0.0, 1.0).setIntercept(true)
+ lr.optimizer.setConvergenceTol(0.0005)
val model = lr.run(testRDD)
@@ -300,11 +296,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w
testRDD.cache()
// Use half as many iterations as the previous test.
- val lr = new LogisticRegressionWithSGD().setIntercept(true)
- lr.optimizer
- .setStepSize(10.0)
- .setRegParam(0.0)
- .setNumIterations(10)
+ val lr = new LogisticRegressionWithSGD(10.0, 10, 0.0, 1.0).setIntercept(true)
val model = lr.run(testRDD, initialWeights)
@@ -335,11 +327,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w
testRDD.cache()
// Use half as many iterations as the previous test.
- val lr = new LogisticRegressionWithSGD().setIntercept(true)
- lr.optimizer.
- setStepSize(1.0).
- setNumIterations(10).
- setRegParam(1.0)
+ val lr = new LogisticRegressionWithSGD(1.0, 10, 1.0, 1.0).setIntercept(true)
val model = lr.run(testRDD, initialWeights)
@@ -916,7 +904,7 @@ class LogisticRegressionClusterSuite extends SparkFunSuite with LocalClusterSpar
}.cache()
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val model = LogisticRegressionWithSGD.train(points, 2)
+ val model = new LogisticRegressionWithSGD(1.0, 2, 0.0, 1.0).run(points)
val predictions = model.predict(points.map(_.features))
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
index c4bf5b27187f6..149a525a58ff6 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
@@ -367,7 +367,7 @@ class KMeansClusterSuite extends SparkFunSuite with LocalClusterSparkContext {
for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) {
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val model = KMeans.train(points, 2, 2, 1, initMode)
+ val model = KMeans.train(points, 2, 2, initMode)
val predictions = model.predict(points).collect()
val cost = model.computeCost(points)
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
index d96103d01e4ab..b4281d66ec1f8 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
@@ -55,8 +55,7 @@ class LassoSuite extends SparkFunSuite with MLlibTestSparkContext {
}
val testRDD = sc.parallelize(testData, 2).cache()
- val ls = new LassoWithSGD()
- ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40)
+ val ls = new LassoWithSGD(1.0, 40, 0.01, 1.0)
val model = ls.run(testRDD)
val weight0 = model.weights(0)
@@ -99,8 +98,8 @@ class LassoSuite extends SparkFunSuite with MLlibTestSparkContext {
val testRDD = sc.parallelize(testData, 2).cache()
- val ls = new LassoWithSGD()
- ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40).setConvergenceTol(0.0005)
+ val ls = new LassoWithSGD(1.0, 40, 0.01, 1.0)
+ ls.optimizer.setConvergenceTol(0.0005)
val model = ls.run(testRDD, initialWeights)
val weight0 = model.weights(0)
@@ -153,7 +152,7 @@ class LassoClusterSuite extends SparkFunSuite with LocalClusterSparkContext {
}.cache()
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val model = LassoWithSGD.train(points, 2)
+ val model = new LassoWithSGD(1.0, 2, 0.01, 1.0).run(points)
val predictions = model.predict(points.map(_.features))
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index 0694079b9df9e..a9765b007b450 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
@@ -46,7 +46,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
test("linear regression") {
val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(
3.0, Array(10.0, 10.0), 100, 42), 2).cache()
- val linReg = new LinearRegressionWithSGD().setIntercept(true)
+ val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(true)
linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
val model = linReg.run(testRDD)
@@ -72,7 +72,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
test("linear regression without intercept") {
val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(
0.0, Array(10.0, 10.0), 100, 42), 2).cache()
- val linReg = new LinearRegressionWithSGD().setIntercept(false)
+ val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(false)
linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
val model = linReg.run(testRDD)
@@ -103,7 +103,7 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1))))
LabeledPoint(label, sv)
}.cache()
- val linReg = new LinearRegressionWithSGD().setIntercept(false)
+ val linReg = new LinearRegressionWithSGD(1.0, 100, 0.0, 1.0).setIntercept(false)
linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
val model = linReg.run(sparseRDD)
@@ -160,7 +160,7 @@ class LinearRegressionClusterSuite extends SparkFunSuite with LocalClusterSparkC
}.cache()
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val model = LinearRegressionWithSGD.train(points, 2)
+ val model = new LinearRegressionWithSGD(1.0, 2, 0.0, 1.0).run(points)
val predictions = model.predict(points.map(_.features))
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index 815be32d2e510..2d6aec184ad9d 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -60,18 +60,13 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
val validationRDD = sc.parallelize(validationData, 2).cache()
// First run without regularization.
- val linearReg = new LinearRegressionWithSGD()
- linearReg.optimizer.setNumIterations(200)
- .setStepSize(1.0)
+ val linearReg = new LinearRegressionWithSGD(1.0, 200, 0.0, 1.0)
val linearModel = linearReg.run(testRDD)
val linearErr = predictionError(
linearModel.predict(validationRDD.map(_.features)).collect(), validationData)
- val ridgeReg = new RidgeRegressionWithSGD()
- ridgeReg.optimizer.setNumIterations(200)
- .setRegParam(0.1)
- .setStepSize(1.0)
+ val ridgeReg = new RidgeRegressionWithSGD(1.0, 200, 0.1, 1.0)
val ridgeModel = ridgeReg.run(testRDD)
val ridgeErr = predictionError(
ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData)
@@ -110,7 +105,7 @@ class RidgeRegressionClusterSuite extends SparkFunSuite with LocalClusterSparkCo
}.cache()
// If we serialize data directly in the task closure, the size of the serialized task would be
// greater than 1MB and hence Spark would throw an error.
- val model = RidgeRegressionWithSGD.train(points, 2)
+ val model = new RidgeRegressionWithSGD(1.0, 2, 0.01, 1.0).run(points)
val predictions = model.predict(points.map(_.features))
}
}
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 04bec14c42c6f..097f1d2c2a6e1 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -36,6 +36,20 @@ object MimaExcludes {
// Exclude rules for 3.0.x
lazy val v30excludes = v24excludes ++ Seq(
+ // [SPARK-][SQL][CORE][MLLIB] Remove more old deprecated items in Spark 3
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SQLContext.createExternalTable"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.createExternalTable"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.KMeans.train"),
+ ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.clustering.KMeans.train"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.classification.LogisticRegressionWithSGD$"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.classification.LogisticRegressionWithSGD.this"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.ChiSqSelectorModel.isSorted"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.RidgeRegressionWithSGD$"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.RidgeRegressionWithSGD.this"),
+ ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.regression.LassoWithSGD.this"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LassoWithSGD$"),
+ ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.regression.LinearRegressionWithSGD$"),
+
// [SPARK-28486][CORE][PYTHON] Map PythonBroadcast's data file to a BroadcastBlock to avoid delete by GC
ProblemFilters.exclude[InaccessibleMethodProblem]("java.lang.Object.finalize"),
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 70c0b27a6aa33..76a5bd0d645ba 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -113,7 +113,7 @@ def wrapper(self, *args, **kwargs):
# for back compatibility
-from pyspark.sql import SQLContext, HiveContext, Row
+from pyspark.sql import SQLContext, Row
__all__ = [
"SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast",
diff --git a/python/pyspark/ml/tests/test_image.py b/python/pyspark/ml/tests/test_image.py
index 0008b0b670d34..5cc2a815eaa14 100644
--- a/python/pyspark/ml/tests/test_image.py
+++ b/python/pyspark/ml/tests/test_image.py
@@ -20,7 +20,7 @@
from pyspark.ml.image import ImageSchema
from pyspark.testing.mlutils import PySparkTestCase, SparkSessionTestCase
-from pyspark.sql import HiveContext, Row
+from pyspark.sql import Row
from pyspark.testing.utils import QuietTest
@@ -67,47 +67,6 @@ def test_read_images(self):
lambda: ImageSchema.toImage("a"))
-class ImageFileFormatOnHiveContextTest(PySparkTestCase):
-
- @classmethod
- def setUpClass(cls):
- super(ImageFileFormatOnHiveContextTest, cls).setUpClass()
- cls.hive_available = True
- # Note that here we enable Hive's support.
- cls.spark = None
- try:
- cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
- except py4j.protocol.Py4JError:
- cls.tearDownClass()
- cls.hive_available = False
- except TypeError:
- cls.tearDownClass()
- cls.hive_available = False
- if cls.hive_available:
- cls.spark = HiveContext._createForTesting(cls.sc)
-
- def setUp(self):
- if not self.hive_available:
- self.skipTest("Hive is not available.")
-
- @classmethod
- def tearDownClass(cls):
- super(ImageFileFormatOnHiveContextTest, cls).tearDownClass()
- if cls.spark is not None:
- cls.spark.sparkSession.stop()
- cls.spark = None
-
- def test_read_images_multiple_times(self):
- # This test case is to check if ImageFileFormat tries to
- # initiate Hive client multiple times. See SPARK-22651.
- data_path = 'data/mllib/images/origin/kittens'
- for i in range(2):
- self.spark.read.format("image") \
- .option("dropInvalid", True) \
- .option("recursiveFileLookup", True) \
- .load(data_path)
-
-
if __name__ == "__main__":
from pyspark.ml.tests.test_image import *
diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
index f220911993096..e41e5c9cc8e89 100644
--- a/python/pyspark/mllib/clustering.py
+++ b/python/pyspark/mllib/clustering.py
@@ -304,7 +304,7 @@ class KMeans(object):
@classmethod
@since('0.9.0')
- def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||",
+ def train(cls, rdd, k, maxIterations=100, initializationMode="k-means||",
seed=None, initializationSteps=2, epsilon=1e-4, initialModel=None):
"""
Train a k-means clustering model.
@@ -317,8 +317,6 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"
:param maxIterations:
Maximum number of iterations allowed.
(default: 100)
- :param runs:
- This param has no effect since Spark 2.0.0.
:param initializationMode:
The initialization algorithm. This can be either "random" or
"k-means||".
@@ -342,8 +340,6 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"
rather than using the random or k-means|| initializationModel.
(default: None)
"""
- if runs != 1:
- warnings.warn("The param `runs` has no effect since Spark 2.0.0.")
clusterInitialModel = []
if initialModel is not None:
if not isinstance(initialModel, KMeansModel):
@@ -351,7 +347,7 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"
"to be of ")
clusterInitialModel = [_convert_to_vector(c) for c in initialModel.clusterCenters]
model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations,
- runs, initializationMode, seed, initializationSteps, epsilon,
+ initializationMode, seed, initializationSteps, epsilon,
clusterInitialModel)
centers = callJavaFunc(rdd.context, model.clusterCenters)
return KMeansModel([c.toArray() for c in centers])
diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py
index c3c06c8124362..9c760e3527be4 100644
--- a/python/pyspark/sql/__init__.py
+++ b/python/pyspark/sql/__init__.py
@@ -43,7 +43,7 @@
from pyspark.sql.types import Row
-from pyspark.sql.context import SQLContext, HiveContext, UDFRegistration
+from pyspark.sql.context import SQLContext, UDFRegistration
from pyspark.sql.session import SparkSession
from pyspark.sql.column import Column
from pyspark.sql.catalog import Catalog
@@ -54,7 +54,7 @@
__all__ = [
- 'SparkSession', 'SQLContext', 'HiveContext', 'UDFRegistration',
+ 'SparkSession', 'SQLContext', 'UDFRegistration',
'DataFrame', 'GroupedData', 'Column', 'Catalog', 'Row',
'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec',
'DataFrameReader', 'DataFrameWriter'
diff --git a/python/pyspark/sql/catalog.py b/python/pyspark/sql/catalog.py
index 974251f63b37a..08cf6ee330785 100644
--- a/python/pyspark/sql/catalog.py
+++ b/python/pyspark/sql/catalog.py
@@ -138,26 +138,6 @@ def listColumns(self, tableName, dbName=None):
isBucket=jcolumn.isBucket()))
return columns
- @since(2.0)
- def createExternalTable(self, tableName, path=None, source=None, schema=None, **options):
- """Creates a table based on the dataset in a data source.
-
- It returns the DataFrame associated with the external table.
-
- The data source is specified by the ``source`` and a set of ``options``.
- If ``source`` is not specified, the default data source configured by
- ``spark.sql.sources.default`` will be used.
-
- Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and
- created external table.
-
- :return: :class:`DataFrame`
- """
- warnings.warn(
- "createExternalTable is deprecated since Spark 2.2, please use createTable instead.",
- DeprecationWarning)
- return self.createTable(tableName, path, source, schema, **options)
-
@since(2.2)
def createTable(self, tableName, path=None, source=None, schema=None, **options):
"""Creates a table based on the dataset in a data source.
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 48a49c583f9c7..e6bd2eb31a844 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -32,7 +32,7 @@
from pyspark.sql.udf import UDFRegistration
from pyspark.sql.utils import install_exception_handler
-__all__ = ["SQLContext", "HiveContext"]
+__all__ = ["SQLContext"]
class SQLContext(object):
@@ -325,24 +325,6 @@ def dropTempTable(self, tableName):
"""
self.sparkSession.catalog.dropTempView(tableName)
- @since(1.3)
- def createExternalTable(self, tableName, path=None, source=None, schema=None, **options):
- """Creates an external table based on the dataset in a data source.
-
- It returns the DataFrame associated with the external table.
-
- The data source is specified by the ``source`` and a set of ``options``.
- If ``source`` is not specified, the default data source configured by
- ``spark.sql.sources.default`` will be used.
-
- Optionally, a schema can be provided as the schema of the returned :class:`DataFrame` and
- created external table.
-
- :return: :class:`DataFrame`
- """
- return self.sparkSession.catalog.createExternalTable(
- tableName, path, source, schema, **options)
-
@ignore_unicode_prefix
@since(1.0)
def sql(self, sqlQuery):
@@ -466,53 +448,6 @@ def streams(self):
return StreamingQueryManager(self._ssql_ctx.streams())
-class HiveContext(SQLContext):
- """A variant of Spark SQL that integrates with data stored in Hive.
-
- Configuration for Hive is read from ``hive-site.xml`` on the classpath.
- It supports running both SQL and HiveQL commands.
-
- :param sparkContext: The SparkContext to wrap.
- :param jhiveContext: An optional JVM Scala HiveContext. If set, we do not instantiate a new
- :class:`HiveContext` in the JVM, instead we make all calls to this object.
-
- .. note:: Deprecated in 2.0.0. Use SparkSession.builder.enableHiveSupport().getOrCreate().
- """
-
- def __init__(self, sparkContext, jhiveContext=None):
- warnings.warn(
- "HiveContext is deprecated in Spark 2.0.0. Please use " +
- "SparkSession.builder.enableHiveSupport().getOrCreate() instead.",
- DeprecationWarning)
- if jhiveContext is None:
- sparkContext._conf.set("spark.sql.catalogImplementation", "hive")
- sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate()
- else:
- sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession())
- SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext)
-
- @classmethod
- def _createForTesting(cls, sparkContext):
- """(Internal use only) Create a new HiveContext for testing.
-
- All test code that touches HiveContext *must* go through this method. Otherwise,
- you may end up launching multiple derby instances and encounter with incredibly
- confusing error messages.
- """
- jsc = sparkContext._jsc.sc()
- jtestHive = sparkContext._jvm.org.apache.spark.sql.hive.test.TestHiveContext(jsc, False)
- return cls(sparkContext, jtestHive)
-
- def refreshTable(self, tableName):
- """Invalidate and refresh all the cached the metadata of the given
- table. For performance reasons, Spark SQL or the external data source
- library it uses might cache certain metadata about a table, such as the
- location of blocks. When those change outside of Spark SQL, users should
- call this function to invalidate the cache.
- """
- self._ssql_ctx.refreshTable(tableName)
-
-
def _test():
import os
import doctest
diff --git a/python/pyspark/sql/tests/test_appsubmit.py b/python/pyspark/sql/tests/test_appsubmit.py
deleted file mode 100644
index 99c0317cc45df..0000000000000
--- a/python/pyspark/sql/tests/test_appsubmit.py
+++ /dev/null
@@ -1,97 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-import os
-import subprocess
-import tempfile
-
-import py4j
-
-from pyspark import SparkContext
-from pyspark.tests.test_appsubmit import SparkSubmitTests
-
-
-class HiveSparkSubmitTests(SparkSubmitTests):
-
- @classmethod
- def setUpClass(cls):
- # get a SparkContext to check for availability of Hive
- sc = SparkContext('local[4]', cls.__name__)
- cls.hive_available = True
- try:
- sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
- except py4j.protocol.Py4JError:
- cls.hive_available = False
- except TypeError:
- cls.hive_available = False
- finally:
- # we don't need this SparkContext for the test
- sc.stop()
-
- def setUp(self):
- super(HiveSparkSubmitTests, self).setUp()
- if not self.hive_available:
- self.skipTest("Hive is not available.")
-
- def test_hivecontext(self):
- # This test checks that HiveContext is using Hive metastore (SPARK-16224).
- # It sets a metastore url and checks if there is a derby dir created by
- # Hive metastore. If this derby dir exists, HiveContext is using
- # Hive metastore.
- metastore_path = os.path.join(tempfile.mkdtemp(), "spark16224_metastore_db")
- metastore_URL = "jdbc:derby:;databaseName=" + metastore_path + ";create=true"
- hive_site_dir = os.path.join(self.programDir, "conf")
- hive_site_file = self.createTempFile("hive-site.xml", ("""
- |
- |
- | javax.jdo.option.ConnectionURL
- | %s
- |
- |
- """ % metastore_URL).lstrip(), "conf")
- script = self.createTempFile("test.py", """
- |import os
- |
- |from pyspark.conf import SparkConf
- |from pyspark.context import SparkContext
- |from pyspark.sql import HiveContext
- |
- |conf = SparkConf()
- |sc = SparkContext(conf=conf)
- |hive_context = HiveContext(sc)
- |print(hive_context.sql("show databases").collect())
- """)
- proc = subprocess.Popen(
- self.sparkSubmit + ["--master", "local-cluster[1,1,1024]",
- "--driver-class-path", hive_site_dir, script],
- stdout=subprocess.PIPE)
- out, err = proc.communicate()
- self.assertEqual(0, proc.returncode)
- self.assertIn("default", out.decode('utf-8'))
- self.assertTrue(os.path.exists(metastore_path))
-
-
-if __name__ == "__main__":
- import unittest
- from pyspark.sql.tests.test_appsubmit import *
-
- try:
- import xmlrunner
- testRunner = xmlrunner.XMLTestRunner(output='target/test-reports', verbosity=2)
- except ImportError:
- testRunner = None
- unittest.main(testRunner=testRunner, verbosity=2)
diff --git a/python/pyspark/sql/tests/test_context.py b/python/pyspark/sql/tests/test_context.py
index b2df3921e62a9..d57ebc48cf5e6 100644
--- a/python/pyspark/sql/tests/test_context.py
+++ b/python/pyspark/sql/tests/test_context.py
@@ -27,7 +27,7 @@
import py4j
-from pyspark import HiveContext, Row
+from pyspark.sql import Row, SparkSession
from pyspark.sql.types import *
from pyspark.sql.window import Window
from pyspark.testing.utils import ReusedPySparkTestCase
@@ -40,15 +40,20 @@ def setUpClass(cls):
ReusedPySparkTestCase.setUpClass()
cls.tempdir = tempfile.NamedTemporaryFile(delete=False)
cls.hive_available = True
+ cls.spark = None
try:
cls.sc._jvm.org.apache.hadoop.hive.conf.HiveConf()
except py4j.protocol.Py4JError:
+ cls.tearDownClass()
cls.hive_available = False
except TypeError:
+ cls.tearDownClass()
cls.hive_available = False
+ if cls.hive_available:
+ cls.spark = SparkSession.builder.enableHiveSupport().getOrCreate()
+
os.unlink(cls.tempdir.name)
if cls.hive_available:
- cls.spark = HiveContext._createForTesting(cls.sc)
cls.testData = [Row(key=i, value=str(i)) for i in range(100)]
cls.df = cls.sc.parallelize(cls.testData).toDF()
@@ -60,13 +65,16 @@ def setUp(self):
def tearDownClass(cls):
ReusedPySparkTestCase.tearDownClass()
shutil.rmtree(cls.tempdir.name, ignore_errors=True)
+ if cls.spark is not None:
+ cls.spark.stop()
+ cls.spark = None
def test_save_and_load_table(self):
df = self.df
tmpPath = tempfile.mkdtemp()
shutil.rmtree(tmpPath)
df.write.saveAsTable("savedJsonTable", "json", "append", path=tmpPath)
- actual = self.spark.createExternalTable("externalJsonTable", tmpPath, "json")
+ actual = self.spark.catalog.createTable("externalJsonTable", tmpPath, "json")
self.assertEqual(sorted(df.collect()),
sorted(self.spark.sql("SELECT * FROM savedJsonTable").collect()))
self.assertEqual(sorted(df.collect()),
@@ -76,7 +84,7 @@ def test_save_and_load_table(self):
df.write.saveAsTable("savedJsonTable", "json", "overwrite", path=tmpPath)
schema = StructType([StructField("value", StringType(), True)])
- actual = self.spark.createExternalTable("externalJsonTable", source="json",
+ actual = self.spark.catalog.createTable("externalJsonTable", source="json",
schema=schema, path=tmpPath,
noUse="this options will not be used")
self.assertEqual(sorted(df.collect()),
@@ -87,11 +95,11 @@ def test_save_and_load_table(self):
self.spark.sql("DROP TABLE savedJsonTable")
self.spark.sql("DROP TABLE externalJsonTable")
- defaultDataSourceName = self.spark.getConf("spark.sql.sources.default",
- "org.apache.spark.sql.parquet")
+ defaultDataSourceName = self.spark.conf.get("spark.sql.sources.default",
+ "org.apache.spark.sql.parquet")
self.spark.sql("SET spark.sql.sources.default=org.apache.spark.sql.json")
df.write.saveAsTable("savedJsonTable", path=tmpPath, mode="overwrite")
- actual = self.spark.createExternalTable("externalJsonTable", path=tmpPath)
+ actual = self.spark.catalog.createTable("externalJsonTable", path=tmpPath)
self.assertEqual(sorted(df.collect()),
sorted(self.spark.sql("SELECT * FROM savedJsonTable").collect()))
self.assertEqual(sorted(df.collect()),
diff --git a/python/pyspark/streaming/kinesis.py b/python/pyspark/streaming/kinesis.py
index 4ed9f2a40c3a7..729ec97505aad 100644
--- a/python/pyspark/streaming/kinesis.py
+++ b/python/pyspark/streaming/kinesis.py
@@ -80,7 +80,6 @@ def createStream(ssc, kinesisAppName, streamName, endpointUrl, regionName,
jduration = ssc._jduration(checkpointInterval)
try:
- # Use KinesisUtilsPythonHelper to access Scala's KinesisUtils
helper = ssc._jvm.org.apache.spark.streaming.kinesis.KinesisUtilsPythonHelper()
except TypeError as e:
if str(e) == "'JavaPackage' object is not callable":
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 61c027ec4483a..45627e9d3f1f8 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -74,7 +74,7 @@ private[spark] class ClientArguments(args: Array[String]) {
s"""
|Usage: org.apache.spark.deploy.yarn.Client [options]
|Options:
- | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster
+ | --jar JAR_PATH Path to your application's JAR file (required in YARN cluster
| mode)
| --class CLASS_NAME Name of your application's main class (required)
| --primary-py-file A main Python file
diff --git a/sql/README.md b/sql/README.md
index f0ea848a41d09..67e3225e2c275 100644
--- a/sql/README.md
+++ b/sql/README.md
@@ -6,7 +6,7 @@ This module provides support for executing relational queries expressed in eithe
Spark SQL is broken up into four subprojects:
- Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions.
- Execution (sql/core) - A query planner / execution engine for translating Catalyst's logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files.
- - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs.
+ - Hive Support (sql/hive) - Includes extensions that allow users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs.
- HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server.
Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index 45d0bd4122535..2054874e5e07b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -479,97 +479,6 @@ class SQLContext private[sql](val sparkSession: SparkSession)
def readStream: DataStreamReader = sparkSession.readStream
- /**
- * Creates an external table from the given path and returns the corresponding DataFrame.
- * It will use the default data source configured by spark.sql.sources.default.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(tableName: String, path: String): DataFrame = {
- sparkSession.catalog.createTable(tableName, path)
- }
-
- /**
- * Creates an external table from the given path based on a data source
- * and returns the corresponding DataFrame.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- path: String,
- source: String): DataFrame = {
- sparkSession.catalog.createTable(tableName, path, source)
- }
-
- /**
- * Creates an external table from the given path based on a data source and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- options: java.util.Map[String, String]): DataFrame = {
- sparkSession.catalog.createTable(tableName, source, options)
- }
-
- /**
- * (Scala-specific)
- * Creates an external table from the given path based on a data source and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- options: Map[String, String]): DataFrame = {
- sparkSession.catalog.createTable(tableName, source, options)
- }
-
- /**
- * Create an external table from the given path based on a data source, a schema and
- * a set of options. Then, returns the corresponding DataFrame.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- schema: StructType,
- options: java.util.Map[String, String]): DataFrame = {
- sparkSession.catalog.createTable(tableName, source, schema, options)
- }
-
- /**
- * (Scala-specific)
- * Create an external table from the given path based on a data source, a schema and
- * a set of options. Then, returns the corresponding DataFrame.
- *
- * @group ddl_ops
- * @since 1.3.0
- */
- @deprecated("use sparkSession.catalog.createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- schema: StructType,
- options: Map[String, String]): DataFrame = {
- sparkSession.catalog.createTable(tableName, source, schema, options)
- }
-
/**
* Registers the given `DataFrame` as a temporary table in the catalog. Temporary tables exist
* only during the lifetime of this instance of SQLContext.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
index 60738e6d4ef9e..318cc629e7a34 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog
import scala.collection.JavaConverters._
-import org.apache.spark.annotation.{Evolving, Experimental, Stable}
+import org.apache.spark.annotation.Stable
import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset}
import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
@@ -208,20 +208,6 @@ abstract class Catalog {
*/
def functionExists(dbName: String, functionName: String): Boolean
- /**
- * Creates a table from the given path and returns the corresponding DataFrame.
- * It will use the default data source configured by spark.sql.sources.default.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(tableName: String, path: String): DataFrame = {
- createTable(tableName, path)
- }
-
/**
* Creates a table from the given path and returns the corresponding DataFrame.
* It will use the default data source configured by spark.sql.sources.default.
@@ -233,20 +219,6 @@ abstract class Catalog {
*/
def createTable(tableName: String, path: String): DataFrame
- /**
- * Creates a table from the given path based on a data source and returns the corresponding
- * DataFrame.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(tableName: String, path: String, source: String): DataFrame = {
- createTable(tableName, path, source)
- }
-
/**
* Creates a table from the given path based on a data source and returns the corresponding
* DataFrame.
@@ -258,23 +230,6 @@ abstract class Catalog {
*/
def createTable(tableName: String, path: String, source: String): DataFrame
- /**
- * Creates a table from the given path based on a data source and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- options: java.util.Map[String, String]): DataFrame = {
- createTable(tableName, source, options)
- }
-
/**
* Creates a table based on the dataset in a data source and a set of options.
* Then, returns the corresponding DataFrame.
@@ -291,24 +246,6 @@ abstract class Catalog {
createTable(tableName, source, options.asScala.toMap)
}
- /**
- * (Scala-specific)
- * Creates a table from the given path based on a data source and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- options: Map[String, String]): DataFrame = {
- createTable(tableName, source, options)
- }
-
/**
* (Scala-specific)
* Creates a table based on the dataset in a data source and a set of options.
@@ -324,24 +261,6 @@ abstract class Catalog {
source: String,
options: Map[String, String]): DataFrame
- /**
- * Create a table from the given path based on a data source, a schema and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- schema: StructType,
- options: java.util.Map[String, String]): DataFrame = {
- createTable(tableName, source, schema, options)
- }
-
/**
* Create a table based on the dataset in a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
@@ -359,25 +278,6 @@ abstract class Catalog {
createTable(tableName, source, schema, options.asScala.toMap)
}
- /**
- * (Scala-specific)
- * Create a table from the given path based on a data source, a schema and a set of options.
- * Then, returns the corresponding DataFrame.
- *
- * @param tableName is either a qualified or unqualified name that designates a table.
- * If no database identifier is provided, it refers to a table in
- * the current database.
- * @since 2.0.0
- */
- @deprecated("use createTable instead.", "2.2.0")
- def createExternalTable(
- tableName: String,
- source: String,
- schema: StructType,
- options: Map[String, String]): DataFrame = {
- createTable(tableName, source, schema, options)
- }
-
/**
* (Scala-specific)
* Create a table based on the dataset in a data source, a schema and a set of options.
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
deleted file mode 100644
index 02a5117f005e8..0000000000000
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import org.apache.spark.SparkContext
-import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.internal.Logging
-import org.apache.spark.sql.{SparkSession, SQLContext}
-
-
-/**
- * An instance of the Spark SQL execution engine that integrates with data stored in Hive.
- * Configuration for Hive is read from hive-site.xml on the classpath.
- */
-@deprecated("Use SparkSession.builder.enableHiveSupport instead", "2.0.0")
-class HiveContext private[hive](_sparkSession: SparkSession)
- extends SQLContext(_sparkSession) with Logging {
-
- self =>
-
- def this(sc: SparkContext) = {
- this(SparkSession.builder().sparkContext(HiveUtils.withHiveExternalCatalog(sc)).getOrCreate())
- }
-
- def this(sc: JavaSparkContext) = this(sc.sc)
-
- /**
- * Returns a new HiveContext as new session, which will have separated SQLConf, UDF/UDAF,
- * temporary tables and SessionState, but sharing the same CacheManager, IsolatedClientLoader
- * and Hive client (both of execution and metadata) with existing HiveContext.
- */
- override def newSession(): HiveContext = {
- new HiveContext(sparkSession.newSession())
- }
-
- /**
- * Invalidate and refresh all the cached the metadata of the given table. For performance reasons,
- * Spark SQL or the external data source library it uses might cache certain metadata about a
- * table, such as the location of blocks. When those change outside of Spark SQL, users should
- * call this function to invalidate the cache.
- *
- * @since 1.3.0
- */
- def refreshTable(tableName: String): Unit = {
- sparkSession.catalog.refreshTable(tableName)
- }
-
-}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala
index db074361ef03c..14276c9b583f2 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/package.scala
@@ -23,8 +23,5 @@ package org.apache.spark.sql
* - Using HiveQL to express queries.
* - Reading metadata from the Hive Metastore using HiveSerDes.
* - Hive UDFs, UDAs, UDTs
- *
- * Users that would like access to this functionality should create a
- * [[hive.HiveContext HiveContext]] instead of a [[SQLContext]].
*/
package object hive
diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
index 25bd4d0017bd8..5304052b45a41 100644
--- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
+++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java
@@ -38,9 +38,6 @@
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.hive.test.TestHive$;
-import org.apache.spark.sql.types.DataTypes;
-import org.apache.spark.sql.types.StructField;
-import org.apache.spark.sql.types.StructType;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.catalyst.TableIdentifier;
import org.apache.spark.util.Utils;
@@ -94,57 +91,6 @@ public void tearDown() throws IOException {
}
}
- @Test
- public void saveExternalTableAndQueryIt() {
- Map options = new HashMap<>();
- options.put("path", path.toString());
- df.write()
- .format("org.apache.spark.sql.json")
- .mode(SaveMode.Append)
- .options(options)
- .saveAsTable("javaSavedTable");
-
- checkAnswer(
- sqlContext.sql("SELECT * FROM javaSavedTable"),
- df.collectAsList());
-
- Dataset loadedDF =
- sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", options);
-
- checkAnswer(loadedDF, df.collectAsList());
- checkAnswer(
- sqlContext.sql("SELECT * FROM externalTable"),
- df.collectAsList());
- }
-
- @Test
- public void saveExternalTableWithSchemaAndQueryIt() {
- Map options = new HashMap<>();
- options.put("path", path.toString());
- df.write()
- .format("org.apache.spark.sql.json")
- .mode(SaveMode.Append)
- .options(options)
- .saveAsTable("javaSavedTable");
-
- checkAnswer(
- sqlContext.sql("SELECT * FROM javaSavedTable"),
- df.collectAsList());
-
- List fields = new ArrayList<>();
- fields.add(DataTypes.createStructField("b", DataTypes.StringType, true));
- StructType schema = DataTypes.createStructType(fields);
- Dataset loadedDF =
- sqlContext.createExternalTable("externalTable", "org.apache.spark.sql.json", schema, options);
-
- checkAnswer(
- loadedDF,
- sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList());
- checkAnswer(
- sqlContext.sql("SELECT * FROM externalTable"),
- sqlContext.sql("SELECT b FROM javaSavedTable").collectAsList());
- }
-
@Test
public void saveTableAndQueryIt() {
Map options = new HashMap<>();
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
index fcf0b4591ff84..4cbc03d05c9e3 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -216,7 +216,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
// Drop the table and create it again.
sql("DROP TABLE refreshTable")
- sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet")
+ sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet")
// It is not cached.
assert(!isCached("refreshTable"), "refreshTable should not be cached.")
// Refresh the table. REFRESH TABLE command should not make a uncached
@@ -237,7 +237,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
tempPath.delete()
table("src").write.mode(SaveMode.Overwrite).parquet(tempPath.toString)
sql("DROP TABLE IF EXISTS refreshTable")
- sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet")
+ sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet")
checkAnswer(
table("refreshTable"),
table("src").collect())
@@ -256,7 +256,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto
// Drop the table and create it again.
sql("DROP TABLE refreshTable")
- sparkSession.catalog.createExternalTable("refreshTable", tempPath.toString, "parquet")
+ sparkSession.catalog.createTable("refreshTable", tempPath.toString, "parquet")
// It is not cached.
assert(!isCached("refreshTable"), "refreshTable should not be cached.")
// Refresh the table. REFRESH command should not make a uncached
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
deleted file mode 100644
index a80db765846e9..0000000000000
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import org.scalatest.BeforeAndAfterEach
-
-import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
-
-
-class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach {
-
- override protected val enableAutoThreadAudit = false
- private var sc: SparkContext = null
- private var hc: HiveContext = null
-
- override def beforeAll(): Unit = {
- super.beforeAll()
- sc = SparkContext.getOrCreate(new SparkConf().setMaster("local").setAppName("test"))
- HiveUtils.newTemporaryConfiguration(useInMemoryDerby = true).foreach { case (k, v) =>
- sc.hadoopConfiguration.set(k, v)
- }
- hc = new HiveContext(sc)
- }
-
- override def afterEach(): Unit = {
- try {
- hc.sharedState.cacheManager.clearCache()
- hc.sessionState.catalog.reset()
- } finally {
- super.afterEach()
- }
- }
-
- override def afterAll(): Unit = {
- try {
- sc = null
- hc = null
- } finally {
- super.afterAll()
- }
- }
-
- test("basic operations") {
- val _hc = hc
- import _hc.implicits._
- val df1 = (1 to 20).map { i => (i, i) }.toDF("a", "x")
- val df2 = (1 to 100).map { i => (i, i % 10, i % 2 == 0) }.toDF("a", "b", "c")
- .select($"a", $"b")
- .filter($"a" > 10 && $"b" > 6 && $"c")
- val df3 = df1.join(df2, "a")
- val res = df3.collect()
- val expected = Seq((18, 18, 8)).toDF("a", "x", "b").collect()
- assert(res.toSeq == expected.toSeq)
- df3.createOrReplaceTempView("mai_table")
- val df4 = hc.table("mai_table")
- val res2 = df4.collect()
- assert(res2.toSeq == expected.toSeq)
- }
-
- test("basic DDLs") {
- val _hc = hc
- import _hc.implicits._
- val databases = hc.sql("SHOW DATABASES").collect().map(_.getString(0))
- assert(databases.toSeq == Seq("default"))
- hc.sql("CREATE DATABASE mee_db")
- hc.sql("USE mee_db")
- val databases2 = hc.sql("SHOW DATABASES").collect().map(_.getString(0))
- assert(databases2.toSet == Set("default", "mee_db"))
- val df = (1 to 10).map { i => ("bob" + i.toString, i) }.toDF("name", "age")
- df.createOrReplaceTempView("mee_table")
- hc.sql("CREATE TABLE moo_table (name string, age int)")
- hc.sql("INSERT INTO moo_table SELECT * FROM mee_table")
- assert(
- hc.sql("SELECT * FROM moo_table order by name").collect().toSeq ==
- df.collect().toSeq.sortBy(_.getString(0)))
- val tables = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0))
- assert(tables.toSet == Set("moo_table", "mee_table"))
- hc.sql("DROP TABLE moo_table")
- hc.sql("DROP TABLE mee_table")
- val tables2 = hc.sql("SHOW TABLES IN mee_db").select("tableName").collect().map(_.getString(0))
- assert(tables2.isEmpty)
- hc.sql("USE default")
- hc.sql("DROP DATABASE mee_db CASCADE")
- val databases3 = hc.sql("SHOW DATABASES").collect().map(_.getString(0))
- assert(databases3.toSeq == Seq("default"))
- }
-
-}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index ba807fb58fe40..41a26344f7c21 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -516,13 +516,13 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
}
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") {
- sparkSession.catalog.createExternalTable("createdJsonTable", tempPath.toString)
+ sparkSession.catalog.createTable("createdJsonTable", tempPath.toString)
assert(table("createdJsonTable").schema === df.schema)
checkAnswer(sql("SELECT * FROM createdJsonTable"), df)
assert(
intercept[AnalysisException] {
- sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString)
+ sparkSession.catalog.createTable("createdJsonTable", jsonFilePath.toString)
}.getMessage.contains("Table createdJsonTable already exists."),
"We should complain that createdJsonTable already exists")
}
@@ -534,7 +534,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
// Try to specify the schema.
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source name") {
val schema = StructType(StructField("b", StringType, true) :: Nil)
- sparkSession.catalog.createExternalTable(
+ sparkSession.catalog.createTable(
"createdJsonTable",
"org.apache.spark.sql.json",
schema,
@@ -553,7 +553,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
test("path required error") {
assert(
intercept[AnalysisException] {
- sparkSession.catalog.createExternalTable(
+ sparkSession.catalog.createTable(
"createdJsonTable",
"org.apache.spark.sql.json",
Map.empty[String, String])
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala
index 9060ce2e0eb4b..2d3e462531245 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala
@@ -66,14 +66,14 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle
}
}
- test(s"createExternalTable() to non-default database - with USE") {
+ test(s"createTable() to non-default database - with USE") {
withTempDatabase { db =>
activateDatabase(db) {
withTempPath { dir =>
val path = dir.getCanonicalPath
df.write.format("parquet").mode(SaveMode.Overwrite).save(path)
- spark.catalog.createExternalTable("t", path, "parquet")
+ spark.catalog.createTable("t", path, "parquet")
assert(getTableNames(Option(db)).contains("t"))
checkAnswer(spark.table("t"), df)
@@ -92,12 +92,12 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle
}
}
- test(s"createExternalTable() to non-default database - without USE") {
+ test(s"createTable() to non-default database - without USE") {
withTempDatabase { db =>
withTempPath { dir =>
val path = dir.getCanonicalPath
df.write.format("parquet").mode(SaveMode.Overwrite).save(path)
- spark.catalog.createExternalTable(s"$db.t", path, "parquet")
+ spark.catalog.createTable(s"$db.t", path, "parquet")
assert(getTableNames(Option(db)).contains("t"))
checkAnswer(spark.table(s"$db.t"), df)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index cd8e2eaa2b4dc..c6e4407c80e83 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -1749,7 +1749,7 @@ class HiveDDLSuite
test("create hive serde table with Catalog") {
withTable("t") {
withTempDir { dir =>
- val df = spark.catalog.createExternalTable(
+ val df = spark.catalog.createTable(
"t",
"hive",
new StructType().add("i", "int"),