From ba2a07e2b6c5a39597b64041cd5bf342ef9631f5 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Wed, 19 Aug 2015 09:54:03 +0100 Subject: [PATCH 001/232] [SPARK-9977] [DOCS] Update documentation for StringIndexer By using `StringIndexer`, we can obtain indexed label on new column. So a following estimator should use this new column through pipeline if it wants to use string indexed label. I think it is better to make it explicit on documentation. Author: lewuathe Closes #8205 from Lewuathe/SPARK-9977. --- docs/ml-features.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index d82c85ee75a0e..8d56dc32ca3c0 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -725,7 +725,11 @@ dctDf.select("featuresDCT").show(3); `StringIndexer` encodes a string column of labels to a column of label indices. The indices are in `[0, numLabels)`, ordered by label frequencies. So the most frequent label gets index `0`. -If the input column is numeric, we cast it to string and index the string values. +If the input column is numeric, we cast it to string and index the string +values. When downstream pipeline components such as `Estimator` or +`Transformer` make use of this string-indexed label, you must set the input +column of the component to this string-indexed column name. In many cases, +you can set the input column with `setInputCol`. **Examples** From 3d16a545007922ee6fa36e5f5c3959406cb46484 Mon Sep 17 00:00:00 2001 From: Han JU Date: Wed, 19 Aug 2015 13:04:16 +0100 Subject: [PATCH 002/232] [SPARK-8949] Print warnings when using preferred locations feature Add warnings according to SPARK-8949 in `SparkContext` - warnings in scaladoc - log warnings when preferred locations feature is used through `SparkContext`'s constructor However I didn't found any documentation reference of this feature. Please direct me if you know any reference to this feature. Author: Han JU Closes #7874 from darkjh/SPARK-8949. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 2e01a9a18c784..1ddaca8a5ba8c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -118,9 +118,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. */ + @deprecated("Passing in preferred locations has no effect at all, see SPARK-8949", "1.5.0") @DeveloperApi def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { this(config) + logWarning("Passing in preferred locations has no effect at all, see SPARK-8949") this.preferredNodeLocationData = preferredNodeLocationData } @@ -153,6 +155,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + if (preferredNodeLocationData.nonEmpty) { + logWarning("Passing in preferred locations has no effect at all, see SPARK-8949") + } this.preferredNodeLocationData = preferredNodeLocationData } From 39e4ebd521defdb68a0787bcd3bde6bc855f5198 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 19 Aug 2015 07:38:27 -0700 Subject: [PATCH 003/232] [SPARK-10060] [ML] [DOC] spark.ml DecisionTree user guide New user guide section ml-decision-tree.md, including code examples. I have run all examples, including the Java ones. CC: manishamde yanboliang mengxr Author: Joseph K. Bradley Closes #8244 from jkbradley/ml-dt-docs. --- docs/ml-decision-tree.md | 510 +++++++++++++++++++++++++++++++++++++++ docs/ml-features.md | 2 - docs/ml-guide.md | 9 +- docs/mllib-ensembles.md | 2 +- docs/mllib-guide.md | 9 +- 5 files changed, 519 insertions(+), 13 deletions(-) create mode 100644 docs/ml-decision-tree.md diff --git a/docs/ml-decision-tree.md b/docs/ml-decision-tree.md new file mode 100644 index 0000000000000..958c6f5e4716c --- /dev/null +++ b/docs/ml-decision-tree.md @@ -0,0 +1,510 @@ +--- +layout: global +title: Decision Trees - SparkML +displayTitle: ML - Decision Trees +--- + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + + +# Overview + +[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) +and their ensembles are popular methods for the machine learning tasks of +classification and regression. Decision trees are widely used since they are easy to interpret, +handle categorical features, extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. Tree ensemble +algorithms such as random forests and boosting are among the top performers for classification and +regression tasks. + +MLlib supports decision trees for binary and multiclass classification and for regression, +using both continuous and categorical features. The implementation partitions data by rows, +allowing distributed training with millions or even billions of instances. + +Users can find more information about the decision tree algorithm in the [MLlib Decision Tree guide](mllib-decision-tree.html). In this section, we demonstrate the Pipelines API for Decision Trees. + +The Pipelines API for Decision Trees offers a bit more functionality than the original API. In particular, for classification, users can get the predicted probability of each class (a.k.a. class conditional probabilities). + +Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described in the [Ensembles guide](ml-ensembles.html). + +# Inputs and Outputs (Predictions) + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +## Input Columns + + + + + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescription
labelColDouble"label"Label to predict
featuresColVector"features"Feature vector
+ +## Output Columns + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescriptionNotes
predictionColDouble"prediction"Predicted label
rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
+ +# Examples + +The below examples demonstrate the Pipelines API for Decision Trees. The main differences between this API and the [original MLlib Decision Tree API](mllib-decision-tree.html) are: + +* support for ML Pipelines +* separation of Decision Trees for classification vs. regression +* use of DataFrame metadata to distinguish continuous and categorical features + + +## Classification + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the Decision Tree algorithm can recognize. + +
+
+ +More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.classification.DecisionTreeClassifier). + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.DecisionTreeClassifier +import org.apache.spark.ml.classification.DecisionTreeClassificationModel +import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) +// Automatically identify categorical features, and index them. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a DecisionTree model. +val dt = new DecisionTreeClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + +// Convert indexed labels back to original labels. +val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + +// Chain indexers and tree in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, dt, labelConverter)) + +// Train model. This also runs the indexers. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision") +val accuracy = evaluator.evaluate(predictions) +println("Test Error = " + (1.0 - accuracy)) + +val treeModel = model.stages(2).asInstanceOf[DecisionTreeClassificationModel] +println("Learned classification tree model:\n" + treeModel.toDebugString) +{% endhighlight %} +
+ +
+ +More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/classification/DecisionTreeClassifier.html). + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.DecisionTreeClassifier; +import org.apache.spark.ml.classification.DecisionTreeClassificationModel; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); +// Automatically identify categorical features, and index them. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a DecisionTree model. +DecisionTreeClassifier dt = new DecisionTreeClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures"); + +// Convert indexed labels back to original labels. +IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + +// Chain indexers and tree in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); + +// Train model. This also runs the indexers. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision"); +double accuracy = evaluator.evaluate(predictions); +System.out.println("Test Error = " + (1.0 - accuracy)); + +DecisionTreeClassificationModel treeModel = + (DecisionTreeClassificationModel)(model.stages()[2]); +System.out.println("Learned classification tree model:\n" + treeModel.toDebugString()); +{% endhighlight %} +
+ +
+ +More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.classification.DecisionTreeClassifier). + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.classification import DecisionTreeClassifier +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Index labels, adding metadata to the label column. +# Fit on whole dataset to include all labels in index. +labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) +# Automatically identify categorical features, and index them. +# We specify maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a DecisionTree model. +dt = DecisionTreeClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") + +# Chain indexers and tree in a Pipeline +pipeline = Pipeline(stages=[labelIndexer, featureIndexer, dt]) + +# Train model. This also runs the indexers. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "indexedLabel", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="precision") +accuracy = evaluator.evaluate(predictions) +print "Test Error = %g" % (1.0 - accuracy) + +treeModel = model.stages[2] +print treeModel # summary only +{% endhighlight %} +
+ +
+ + +## Regression + +
+
+ +More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.classification.DecisionTreeClassifier). + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.regression.DecisionTreeRegressor +import org.apache.spark.ml.regression.DecisionTreeRegressionModel +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Automatically identify categorical features, and index them. +// Here, we treat features with > 4 distinct values as continuous. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a DecisionTree model. +val dt = new DecisionTreeRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + +// Chain indexers and tree in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(featureIndexer, dt)) + +// Train model. This also runs the indexer. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("prediction", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") +// We negate the RMSE value since RegressionEvalutor returns negated RMSE +// (since evaluation metrics are meant to be maximized by CrossValidator). +val rmse = - evaluator.evaluate(predictions) +println("Root Mean Squared Error (RMSE) on test data = " + rmse) + +val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] +println("Learned regression tree model:\n" + treeModel.toDebugString) +{% endhighlight %} +
+ +
+ +More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/classification/DecisionTreeClassifier.html). + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.ml.regression.DecisionTreeRegressionModel; +import org.apache.spark.ml.regression.DecisionTreeRegressor; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); +// Automatically identify categorical features, and index them. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a DecisionTree model. +DecisionTreeRegressor dt = new DecisionTreeRegressor() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures"); + +// Convert indexed labels back to original labels. +IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + +// Chain indexers and tree in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); + +// Train model. This also runs the indexers. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("rmse"); +// We negate the RMSE value since RegressionEvalutor returns negated RMSE +// (since evaluation metrics are meant to be maximized by CrossValidator). +double rmse = - evaluator.evaluate(predictions); +System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + +DecisionTreeRegressionModel treeModel = + (DecisionTreeRegressionModel)(model.stages()[2]); +System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); +{% endhighlight %} +
+ +
+ +More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.classification.DecisionTreeClassifier). + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.regression import DecisionTreeRegressor +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Index labels, adding metadata to the label column. +# Fit on whole dataset to include all labels in index. +labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) +# Automatically identify categorical features, and index them. +# We specify maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a DecisionTree model. +dt = DecisionTreeRegressor(labelCol="indexedLabel", featuresCol="indexedFeatures") + +# Chain indexers and tree in a Pipeline +pipeline = Pipeline(stages=[labelIndexer, featureIndexer, dt]) + +# Train model. This also runs the indexers. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "indexedLabel", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = RegressionEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="rmse") +# We negate the RMSE value since RegressionEvalutor returns negated RMSE +# (since evaluation metrics are meant to be maximized by CrossValidator). +rmse = -evaluator.evaluate(predictions) +print "Root Mean Squared Error (RMSE) on test data = %g" % rmse + +treeModel = model.stages[1] +print treeModel # summary only +{% endhighlight %} +
+ +
diff --git a/docs/ml-features.md b/docs/ml-features.md index 8d56dc32ca3c0..d0e8eeb7a757e 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1477,5 +1477,3 @@ print(output.select("features", "clicked").first()) -# Feature Selectors - diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 4fe0ea78bbe13..c64fff7c0315a 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -32,10 +32,7 @@ Users should be comfortable using `spark.mllib` features and expect more feature Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. -Guides for sub-packages of `spark.ml` include: - -* [Feature Extraction, Transformation, and Selection](ml-features.html): Details on transformers supported in the Pipelines API, including a few not in the lower-level `spark.mllib` API -* [Ensembles](ml-ensembles.html): Details on ensemble learning methods in the Pipelines API +See the [Algorithm Guides section](#algorithm-guides) below for guides on sub-packages of `spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. **Table of Contents** @@ -179,10 +176,8 @@ There are now several algorithms in the Pipelines API which are not in the lower **Pipelines API Algorithm Guides** * [Feature Extraction, Transformation, and Selection](ml-features.html) +* [Decision Trees for Classification and Regression](ml-decision-tree.html) * [Ensembles](ml-ensembles.html) - -**Algorithms in `spark.ml`** - * [Linear methods with elastic net regularization](ml-linear-methods.html) # Code Examples diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index 7521fb14a7bd6..1e00b2083ed73 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -9,7 +9,7 @@ displayTitle: MLlib - Ensembles An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) is a learning algorithm which creates a model composed of a set of other base models. -MLlib supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBosotedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). +MLlib supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). Both use [decision trees](mllib-decision-tree.html) as their base models. ## Gradient-Boosted Trees vs. Random Forests diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 0573e5d4bc5ff..6330c977552d1 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -73,11 +73,14 @@ Users should be comfortable using `spark.mllib` features and expect more feature Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. -More detailed guides for `spark.ml` include: +Guides for `spark.ml` include: * **[spark.ml programming guide](ml-guide.html)**: overview of the Pipelines API and major concepts -* [Feature transformers](ml-features.html): Details on transformers supported in the Pipelines API, including a few not in the lower-level `spark.mllib` API -* [Ensembles](ml-ensembles.html): Details on ensemble learning methods in the Pipelines API +* Guides on using algorithms within the Pipelines API: + * [Feature transformers](ml-features.html), including a few not in the lower-level `spark.mllib` API + * [Decision trees](ml-decision-tree.html) + * [Ensembles](ml-ensembles.html) + * [Linear methods](ml-linear-methods.html) # Dependencies From 802b5b8791fc2c892810981b2479a04175aa3dcd Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 19 Aug 2015 08:53:34 -0700 Subject: [PATCH 004/232] [SPARK-10084] [MLLIB] [DOC] Add Python example for mllib FP-growth user guide 1, Add Python example for mllib FP-growth user guide. 2, Correct mistakes of Scala and Java examples. Author: Yanbo Liang Closes #8279 from yanboliang/spark-10084. --- docs/mllib-frequent-pattern-mining.md | 73 ++++++++++++++++++--------- 1 file changed, 50 insertions(+), 23 deletions(-) diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md index 6c06550703d46..4d4f5cfdc564e 100644 --- a/docs/mllib-frequent-pattern-mining.md +++ b/docs/mllib-frequent-pattern-mining.md @@ -39,9 +39,9 @@ MLlib's FP-growth implementation takes the following (hyper-)parameters:
-[`FPGrowth`](api/scala/index.html#org.apache.spark.mllib.fpm.FPGrowth) -implements the FP-growth algorithm. It take an `RDD` of transactions, -where each transaction is an `Iterable` of items of a generic type. +[`FPGrowth`](api/scala/index.html#org.apache.spark.mllib.fpm.FPGrowth) implements the +FP-growth algorithm. +It take a `RDD` of transactions, where each transaction is an `Array` of items of a generic type. Calling `FPGrowth.run` with transactions returns an [`FPGrowthModel`](api/scala/index.html#org.apache.spark.mllib.fpm.FPGrowthModel) that stores the frequent itemsets with their frequencies. The following @@ -53,16 +53,11 @@ details) from `transactions`. {% highlight scala %} import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.mllib.fpm.FPGrowth -val transactions: RDD[Array[String]] = sc.parallelize(Seq( - "r z h k p", - "z y x w v u t s", - "s x o n r", - "x z y m t s q e", - "z", - "x z y r q t p") - .map(_.split(" "))) +val data = sc.textFile("data/mllib/sample_fpgrowth.txt") + +val transactions: RDD[Array[String]] = data.map(s => s.trim.split(' ')) val fpg = new FPGrowth() .setMinSupport(0.2) @@ -86,10 +81,10 @@ model.generateAssociationRules(minConfidence).collect().foreach { rule =>
-[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) -implements the FP-growth algorithm. It take a `JavaRDD` of -transactions, where each transaction is an `Array` of items of a generic -type. Calling `FPGrowth.run` with transactions returns an +[`FPGrowth`](api/java/org/apache/spark/mllib/fpm/FPGrowth.html) implements the +FP-growth algorithm. +It take an `JavaRDD` of transactions, where each transaction is an `Iterable` of items of a generic type. +Calling `FPGrowth.run` with transactions returns an [`FPGrowthModel`](api/java/org/apache/spark/mllib/fpm/FPGrowthModel.html) that stores the frequent itemsets with their frequencies. The following example illustrates how to mine frequent itemsets and association rules @@ -107,13 +102,19 @@ import org.apache.spark.mllib.fpm.AssociationRules; import org.apache.spark.mllib.fpm.FPGrowth; import org.apache.spark.mllib.fpm.FPGrowthModel; -JavaRDD> transactions = sc.parallelize(Arrays.asList( - Arrays.asList("r z h k p".split(" ")), - Arrays.asList("z y x w v u t s".split(" ")), - Arrays.asList("s x o n r".split(" ")), - Arrays.asList("x z y m t s q e".split(" ")), - Arrays.asList("z".split(" ")), - Arrays.asList("x z y r q t p".split(" "))), 2); +SparkConf conf = new SparkConf().setAppName("FP-growth Example"); +JavaSparkContext sc = new JavaSparkContext(conf); + +JavaRDD data = sc.textFile("data/mllib/sample_fpgrowth.txt"); + +JavaRDD> transactions = data.map( + new Function>() { + public List call(String line) { + String[] parts = line.split(" "); + return Arrays.asList(parts); + } + } +); FPGrowth fpg = new FPGrowth() .setMinSupport(0.2) @@ -133,6 +134,32 @@ for (AssociationRules.Rule rule {% endhighlight %}
+ +
+ +[`FPGrowth`](api/python/pyspark.mllib.html#pyspark.mllib.fpm.FPGrowth) implements the +FP-growth algorithm. +It take an `RDD` of transactions, where each transaction is an `List` of items of a generic type. +Calling `FPGrowth.train` with transactions returns an +[`FPGrowthModel`](api/python/pyspark.mllib.html#pyspark.mllib.fpm.FPGrowthModel) +that stores the frequent itemsets with their frequencies. + +{% highlight python %} +from pyspark.mllib.fpm import FPGrowth + +data = sc.textFile("data/mllib/sample_fpgrowth.txt") + +transactions = data.map(lambda line: line.strip().split(' ')) + +model = FPGrowth.train(transactions, minSupport=0.2, numPartitions=10) + +result = model.freqItemsets().collect() +for fi in result: + print(fi) +{% endhighlight %} + +
+
## Association Rules From f3e177917dd093873dca286d2419704a561ba44b Mon Sep 17 00:00:00 2001 From: Carsten Blank Date: Wed, 19 Aug 2015 10:19:24 -0700 Subject: [PATCH 005/232] [SPARK-5754] [YARN] Spark/Yarn/Windows driver/executor escaping Fix This is my retry to suggest a fix for using Spark on Yarn on Windows. The former request lacked coding style which I hope to have learned to do better, and wasn't a true solution as I didn't really understand where the problem came from. Albeit being still a bit obscure, I can name the "players" and have come up with a better explaination of why I am suggesting this fix. I also used vanzin and srowen input to *try* to give a more elegant solution. I am not so sure if that worked out though. I still hope that this PR is a lot more useful than the last. Also do I hope that this is a _solution_ to the problem that Spark doesn't work on Yarn on Windows. With these changes it works (and I can also explain why!). I still believe that a Unit Test should be included, kind of like the one I committed the last time. But that was premature, as I want to get the principal 'Go' from vanzin and srowen. Thanks for your time both of you. Author: Carsten Blank Author: cbvoxel Closes #8053 from cbvoxel/master. --- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 2 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 56 +++++++++++++++---- .../launcher/YarnCommandBuilderUtils.scala | 27 +++++++++ 4 files changed, 75 insertions(+), 12 deletions(-) create mode 100644 yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8672ef601784b..262c6a8d6397f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -431,7 +431,7 @@ private[spark] class Client( } // Distribute an archive with Hadoop and Spark configuration for the AM. - val (_, confLocalizedPath) = distribute(createConfArchive().getAbsolutePath(), + val (_, confLocalizedPath) = distribute(createConfArchive().toURI().getPath(), resType = LocalResourceType.ARCHIVE, destName = Some(LOCALIZED_CONF_DIR), appMasterOnly = true) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 52580deb372c2..4cc50483a17ff 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -217,7 +217,7 @@ class ExecutorRunnable( // an inconsistent state. // TODO: If the OOM is not recoverable by rescheduling it on different node, then do // 'something' to fail job ... akin to blacklisting trackers in mapred ? - "-XX:OnOutOfMemoryError='kill %p'") ++ + YarnSparkHadoopUtil.getOutOfMemoryErrorArgument) ++ javaOpts ++ Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", "--driver-url", masterAddress.toString, diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 68d01c17ef720..445d3dcd266db 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, P import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.util.Utils @@ -219,26 +220,61 @@ object YarnSparkHadoopUtil { } } + /** + * The handler if an OOM Exception is thrown by the JVM must be configured on Windows + * differently: the 'taskkill' command should be used, whereas Unix-based systems use 'kill'. + * + * As the JVM interprets both %p and %%p as the same, we can use either of them. However, + * some tests on Windows computers suggest, that the JVM only accepts '%%p'. + * + * Furthermore, the behavior of the character '%' on the Windows command line differs from + * the behavior of '%' in a .cmd file: it gets interpreted as an incomplete environment + * variable. Windows .cmd files escape a '%' by '%%'. Thus, the correct way of writing + * '%%p' in an escaped way is '%%%%p'. + * + * @return The correct OOM Error handler JVM option, platform dependent. + */ + def getOutOfMemoryErrorArgument : String = { + if (Utils.isWindows) { + escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") + } else { + "-XX:OnOutOfMemoryError='kill %p'" + } + } + /** * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands - * using `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. The - * argument is enclosed in single quotes and some key characters are escaped. + * using either + * + * (Unix-based) `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. + * The argument is enclosed in single quotes and some key characters are escaped. + * + * (Windows-based) part of a .cmd file in which case windows escaping for each argument must be + * applied. Windows is quite lenient, however it is usually Java that causes trouble, needing to + * distinguish between arguments starting with '-' and class names. If arguments are surrounded + * by ' java takes the following string as is, hence an argument is mistakenly taken as a class + * name which happens to start with a '-'. The way to avoid this, is to surround nothing with + * a ', but instead with a ". * * @param arg A single argument. * @return Argument quoted for execution via Yarn's generated shell script. */ def escapeForShell(arg: String): String = { if (arg != null) { - val escaped = new StringBuilder("'") - for (i <- 0 to arg.length() - 1) { - arg.charAt(i) match { - case '$' => escaped.append("\\$") - case '"' => escaped.append("\\\"") - case '\'' => escaped.append("'\\''") - case c => escaped.append(c) + if (Utils.isWindows) { + YarnCommandBuilderUtils.quoteForBatchScript(arg) + } else { + val escaped = new StringBuilder("'") + for (i <- 0 to arg.length() - 1) { + arg.charAt(i) match { + case '$' => escaped.append("\\$") + case '"' => escaped.append("\\\"") + case '\'' => escaped.append("'\\''") + case c => escaped.append(c) + } } + escaped.append("'").toString() } - escaped.append("'").toString() } else { arg } diff --git a/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala b/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala new file mode 100644 index 0000000000000..3ac36ef0a1c3f --- /dev/null +++ b/yarn/src/main/scala/org/apache/spark/launcher/YarnCommandBuilderUtils.scala @@ -0,0 +1,27 @@ +/* + * 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.launcher + +/** + * Exposes needed methods + */ +private[spark] object YarnCommandBuilderUtils { + def quoteForBatchScript(arg: String) : String = { + CommandBuilderUtils.quoteForBatchScript(arg) + } +} From 2fcb9cb9552dac1d78dcca5d4d5032b4fa6c985c Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Wed, 19 Aug 2015 10:41:14 -0700 Subject: [PATCH 006/232] [SPARK-9856] [SPARKR] Add expression functions into SparkR whose params are complicated I added lots of Column functinos into SparkR. And I also added `rand(seed: Int)` and `randn(seed: Int)` in Scala. Since we need such APIs for R integer type. ### JIRA [[SPARK-9856] Add expression functions into SparkR whose params are complicated - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-9856) Author: Yu ISHIKAWA Closes #8264 from yu-iskw/SPARK-9856-3. --- R/pkg/NAMESPACE | 28 ++ R/pkg/R/functions.R | 415 ++++++++++++++++++ R/pkg/R/generics.R | 113 +++++ R/pkg/inst/tests/test_sparkSQL.R | 98 ++++- .../apache/spark/api/r/RBackendHandler.scala | 1 + 5 files changed, 649 insertions(+), 6 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 8fa12d5adef8a..111a2dc30db73 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -84,6 +84,7 @@ exportClasses("Column") exportMethods("abs", "acos", + "add_months", "alias", "approxCountDistinct", "asc", @@ -101,12 +102,17 @@ exportMethods("abs", "ceil", "ceiling", "concat", + "concat_ws", "contains", + "conv", "cos", "cosh", "count", "countDistinct", "crc32", + "date_add", + "date_format", + "date_sub", "datediff", "dayofmonth", "dayofyear", @@ -115,9 +121,14 @@ exportMethods("abs", "exp", "explode", "expm1", + "expr", "factorial", "first", "floor", + "format_number", + "format_string", + "from_unixtime", + "from_utc_timestamp", "getField", "getItem", "greatest", @@ -125,6 +136,7 @@ exportMethods("abs", "hour", "hypot", "initcap", + "instr", "isNaN", "isNotNull", "isNull", @@ -135,11 +147,13 @@ exportMethods("abs", "levenshtein", "like", "lit", + "locate", "log", "log10", "log1p", "log2", "lower", + "lpad", "ltrim", "max", "md5", @@ -152,16 +166,26 @@ exportMethods("abs", "n_distinct", "nanvl", "negate", + "next_day", "otherwise", "pmod", "quarter", + "rand", + "randn", + "regexp_extract", + "regexp_replace", "reverse", "rint", "rlike", "round", + "rpad", "rtrim", "second", "sha1", + "sha2", + "shiftLeft", + "shiftRight", + "shiftRightUnsigned", "sign", "signum", "sin", @@ -171,6 +195,7 @@ exportMethods("abs", "sqrt", "startsWith", "substr", + "substring_index", "sum", "sumDistinct", "tan", @@ -178,9 +203,12 @@ exportMethods("abs", "toDegrees", "toRadians", "to_date", + "to_utc_timestamp", + "translate", "trim", "unbase64", "unhex", + "unix_timestamp", "upper", "weekofyear", "when", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 366c230e1e85b..5dba0887d150d 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -166,6 +166,421 @@ setMethod("n", signature(x = "Column"), count(x) }) +#' date_format +#' +#' Converts a date/timestamp/string to a value of string in the format specified by the date +#' format given by the second argument. +#' +#' A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All +#' pattern letters of `java.text.SimpleDateFormat` can be used. +#' +#' NOTE: Use when ever possible specialized functions like `year`. These benefit from a +#' specialized implementation. +#' +#' @rdname functions +setMethod("date_format", signature(y = "Column", x = "character"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "date_format", y@jc, x) + column(jc) + }) + +#' from_utc_timestamp +#' +#' Assumes given timestamp is UTC and converts to given timezone. +#' +#' @rdname functions +setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) + column(jc) + }) + +#' instr +#' +#' Locate the position of the first occurrence of substr column in the given string. +#' Returns null if either of the arguments are null. +#' +#' NOTE: The position is not zero based, but 1 based index, returns 0 if substr +#' could not be found in str. +#' +#' @rdname functions +setMethod("instr", signature(y = "Column", x = "character"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "instr", y@jc, x) + column(jc) + }) + +#' next_day +#' +#' Given a date column, returns the first date which is later than the value of the date column +#' that is on the specified day of the week. +#' +#' For example, `next <- day('2015-07-27', "Sunday")` returns 2015-08-02 because that is the first +#' Sunday after 2015-07-27. +#' +#' Day of the week parameter is case insensitive, and accepts: +#' "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". +#' +#' @rdname functions +setMethod("next_day", signature(y = "Column", x = "character"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "next_day", y@jc, x) + column(jc) + }) + +#' to_utc_timestamp +#' +#' Assumes given timestamp is in given timezone and converts to UTC. +#' +#' @rdname functions +setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) + column(jc) + }) + +#' add_months +#' +#' Returns the date that is numMonths after startDate. +#' +#' @rdname functions +setMethod("add_months", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "add_months", y@jc, as.integer(x)) + column(jc) + }) + +#' date_add +#' +#' Returns the date that is `days` days after `start` +#' +#' @rdname functions +setMethod("date_add", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "date_add", y@jc, as.integer(x)) + column(jc) + }) + +#' date_sub +#' +#' Returns the date that is `days` days before `start` +#' +#' @rdname functions +setMethod("date_sub", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "date_sub", y@jc, as.integer(x)) + column(jc) + }) + +#' format_number +#' +#' Formats numeric column x to a format like '#,###,###.##', rounded to d decimal places, +#' and returns the result as a string column. +#' +#' If d is 0, the result has no decimal point or fractional part. +#' If d < 0, the result will be null.' +#' +#' @rdname functions +setMethod("format_number", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", + "format_number", + y@jc, as.integer(x)) + column(jc) + }) + +#' sha2 +#' +#' Calculates the SHA-2 family of hash functions of a binary column and +#' returns the value as a hex string. +#' +#' @rdname functions +#' @param y column to compute SHA-2 on. +#' @param x one of 224, 256, 384, or 512. +setMethod("sha2", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sha2", y@jc, as.integer(x)) + column(jc) + }) + +#' shiftLeft +#' +#' Shift the the given value numBits left. If the given value is a long value, this function +#' will return a long value else it will return an integer value. +#' +#' @rdname functions +setMethod("shiftLeft", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", + "shiftLeft", + y@jc, as.integer(x)) + column(jc) + }) + +#' shiftRight +#' +#' Shift the the given value numBits right. If the given value is a long value, it will return +#' a long value else it will return an integer value. +#' +#' @rdname functions +setMethod("shiftRight", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", + "shiftRight", + y@jc, as.integer(x)) + column(jc) + }) + +#' shiftRightUnsigned +#' +#' Unsigned shift the the given value numBits right. If the given value is a long value, +#' it will return a long value else it will return an integer value. +#' +#' @rdname functions +setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), + function(y, x) { + jc <- callJStatic("org.apache.spark.sql.functions", + "shiftRightUnsigned", + y@jc, as.integer(x)) + column(jc) + }) + +#' concat_ws +#' +#' Concatenates multiple input string columns together into a single string column, +#' using the given separator. +#' +#' @rdname functions +setMethod("concat_ws", signature(sep = "character", x = "Column"), + function(sep, x, ...) { + jcols <- listToSeq(lapply(list(x, ...), function(x) { x@jc })) + jc <- callJStatic("org.apache.spark.sql.functions", "concat_ws", sep, jcols) + column(jc) + }) + +#' conv +#' +#' Convert a number in a string column from one base to another. +#' +#' @rdname functions +setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeric"), + function(x, fromBase, toBase) { + fromBase <- as.integer(fromBase) + toBase <- as.integer(toBase) + jc <- callJStatic("org.apache.spark.sql.functions", + "conv", + x@jc, fromBase, toBase) + column(jc) + }) + +#' expr +#' +#' Parses the expression string into the column that it represents, similar to +#' DataFrame.selectExpr +#' +#' @rdname functions +setMethod("expr", signature(x = "character"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "expr", x) + column(jc) + }) + +#' format_string +#' +#' Formats the arguments in printf-style and returns the result as a string column. +#' +#' @rdname functions +setMethod("format_string", signature(format = "character", x = "Column"), + function(format, x, ...) { + jcols <- listToSeq(lapply(list(x, ...), function(arg) { arg@jc })) + jc <- callJStatic("org.apache.spark.sql.functions", + "format_string", + format, jcols) + column(jc) + }) + +#' from_unixtime +#' +#' Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string +#' representing the timestamp of that moment in the current system time zone in the given +#' format. +#' +#' @rdname functions +setMethod("from_unixtime", signature(x = "Column"), + function(x, format = "yyyy-MM-dd HH:mm:ss") { + jc <- callJStatic("org.apache.spark.sql.functions", + "from_unixtime", + x@jc, format) + column(jc) + }) + +#' locate +#' +#' Locate the position of the first occurrence of substr. +#' NOTE: The position is not zero based, but 1 based index, returns 0 if substr +#' could not be found in str. +#' +#' @rdname functions +setMethod("locate", signature(substr = "character", str = "Column"), + function(substr, str, pos = 0) { + jc <- callJStatic("org.apache.spark.sql.functions", + "locate", + substr, str@jc, as.integer(pos)) + column(jc) + }) + +#' lpad +#' +#' Left-pad the string column with +#' +#' @rdname functions +setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), + function(x, len, pad) { + jc <- callJStatic("org.apache.spark.sql.functions", + "lpad", + x@jc, as.integer(len), pad) + column(jc) + }) + +#' rand +#' +#' Generate a random column with i.i.d. samples from U[0.0, 1.0]. +#' +#' @rdname functions +setMethod("rand", signature(seed = "missing"), + function(seed) { + jc <- callJStatic("org.apache.spark.sql.functions", "rand") + column(jc) + }) +setMethod("rand", signature(seed = "numeric"), + function(seed) { + jc <- callJStatic("org.apache.spark.sql.functions", "rand", as.integer(seed)) + column(jc) + }) + +#' randn +#' +#' Generate a column with i.i.d. samples from the standard normal distribution. +#' +#' @rdname functions +setMethod("randn", signature(seed = "missing"), + function(seed) { + jc <- callJStatic("org.apache.spark.sql.functions", "randn") + column(jc) + }) +setMethod("randn", signature(seed = "numeric"), + function(seed) { + jc <- callJStatic("org.apache.spark.sql.functions", "randn", as.integer(seed)) + column(jc) + }) + +#' regexp_extract +#' +#' Extract a specific(idx) group identified by a java regex, from the specified string column. +#' +#' @rdname functions +setMethod("regexp_extract", + signature(x = "Column", pattern = "character", idx = "numeric"), + function(x, pattern, idx) { + jc <- callJStatic("org.apache.spark.sql.functions", + "regexp_extract", + x@jc, pattern, as.integer(idx)) + column(jc) + }) + +#' regexp_replace +#' +#' Replace all substrings of the specified string value that match regexp with rep. +#' +#' @rdname functions +setMethod("regexp_replace", + signature(x = "Column", pattern = "character", replacement = "character"), + function(x, pattern, replacement) { + jc <- callJStatic("org.apache.spark.sql.functions", + "regexp_replace", + x@jc, pattern, replacement) + column(jc) + }) + +#' rpad +#' +#' Right-padded with pad to a length of len. +#' +#' @rdname functions +setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), + function(x, len, pad) { + jc <- callJStatic("org.apache.spark.sql.functions", + "rpad", + x@jc, as.integer(len), pad) + column(jc) + }) + +#' substring_index +#' +#' Returns the substring from string str before count occurrences of the delimiter delim. +#' If count is positive, everything the left of the final delimiter (counting from left) is +#' returned. If count is negative, every to the right of the final delimiter (counting from the +#' right) is returned. substring <- index performs a case-sensitive match when searching for delim. +#' +#' @rdname functions +setMethod("substring_index", + signature(x = "Column", delim = "character", count = "numeric"), + function(x, delim, count) { + jc <- callJStatic("org.apache.spark.sql.functions", + "substring_index", + x@jc, delim, as.integer(count)) + column(jc) + }) + +#' translate +#' +#' Translate any character in the src by a character in replaceString. +#' The characters in replaceString is corresponding to the characters in matchingString. +#' The translate will happen when any character in the string matching with the character +#' in the matchingString. +#' +#' @rdname functions +setMethod("translate", + signature(x = "Column", matchingString = "character", replaceString = "character"), + function(x, matchingString, replaceString) { + jc <- callJStatic("org.apache.spark.sql.functions", + "translate", x@jc, matchingString, replaceString) + column(jc) + }) + +#' unix_timestamp +#' +#' Gets current Unix timestamp in seconds. +#' +#' @rdname functions +setMethod("unix_timestamp", signature(x = "missing", format = "missing"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp") + column(jc) + }) +#' unix_timestamp +#' +#' Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), +#' using the default timezone and the default locale, return null if fail. +#' +#' @rdname functions +setMethod("unix_timestamp", signature(x = "Column", format = "missing"), + function(x, format) { + jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc) + column(jc) + }) +#' unix_timestamp +#' +#' Convert time string with given pattern +#' (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) +#' to Unix time stamp (in seconds), return null if fail. +#' +#' @rdname functions +setMethod("unix_timestamp", signature(x = "Column", format = "character"), + function(x, format = "yyyy-MM-dd HH:mm:ss") { + jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc, format) + column(jc) + }) #' when #' #' Evaluates a list of conditions and returns one of multiple possible result expressions. diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 338b32e648078..84cb8dfdaa2dd 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -662,6 +662,10 @@ setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) ###################### Expression Function Methods ########################## +#' @rdname functions +#' @export +setGeneric("add_months", function(y, x) { standardGeneric("add_months") }) + #' @rdname functions #' @export setGeneric("ascii", function(x) { standardGeneric("ascii") }) @@ -694,6 +698,14 @@ setGeneric("ceil", function(x) { standardGeneric("ceil") }) #' @export setGeneric("concat", function(x, ...) { standardGeneric("concat") }) +#' @rdname functions +#' @export +setGeneric("concat_ws", function(sep, x, ...) { standardGeneric("concat_ws") }) + +#' @rdname functions +#' @export +setGeneric("conv", function(x, fromBase, toBase) { standardGeneric("conv") }) + #' @rdname functions #' @export setGeneric("crc32", function(x) { standardGeneric("crc32") }) @@ -702,6 +714,18 @@ setGeneric("crc32", function(x) { standardGeneric("crc32") }) #' @export setGeneric("datediff", function(y, x) { standardGeneric("datediff") }) +#' @rdname functions +#' @export +setGeneric("date_add", function(y, x) { standardGeneric("date_add") }) + +#' @rdname functions +#' @export +setGeneric("date_format", function(y, x) { standardGeneric("date_format") }) + +#' @rdname functions +#' @export +setGeneric("date_sub", function(y, x) { standardGeneric("date_sub") }) + #' @rdname functions #' @export setGeneric("dayofmonth", function(x) { standardGeneric("dayofmonth") }) @@ -714,6 +738,26 @@ setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) #' @export setGeneric("explode", function(x) { standardGeneric("explode") }) +#' @rdname functions +#' @export +setGeneric("expr", function(x) { standardGeneric("expr") }) + +#' @rdname functions +#' @export +setGeneric("from_utc_timestamp", function(y, x) { standardGeneric("from_utc_timestamp") }) + +#' @rdname functions +#' @export +setGeneric("format_number", function(y, x) { standardGeneric("format_number") }) + +#' @rdname functions +#' @export +setGeneric("format_string", function(format, x, ...) { standardGeneric("format_string") }) + +#' @rdname functions +#' @export +setGeneric("from_unixtime", function(x, ...) { standardGeneric("from_unixtime") }) + #' @rdname functions #' @export setGeneric("greatest", function(x, ...) { standardGeneric("greatest") }) @@ -730,6 +774,10 @@ setGeneric("hour", function(x) { standardGeneric("hour") }) #' @export setGeneric("initcap", function(x) { standardGeneric("initcap") }) +#' @rdname functions +#' @export +setGeneric("instr", function(y, x) { standardGeneric("instr") }) + #' @rdname functions #' @export setGeneric("isNaN", function(x) { standardGeneric("isNaN") }) @@ -750,10 +798,18 @@ setGeneric("levenshtein", function(y, x) { standardGeneric("levenshtein") }) #' @export setGeneric("lit", function(x) { standardGeneric("lit") }) +#' @rdname functions +#' @export +setGeneric("locate", function(substr, str, ...) { standardGeneric("locate") }) + #' @rdname functions #' @export setGeneric("lower", function(x) { standardGeneric("lower") }) +#' @rdname functions +#' @export +setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) + #' @rdname functions #' @export setGeneric("ltrim", function(x) { standardGeneric("ltrim") }) @@ -782,6 +838,10 @@ setGeneric("nanvl", function(y, x) { standardGeneric("nanvl") }) #' @export setGeneric("negate", function(x) { standardGeneric("negate") }) +#' @rdname functions +#' @export +setGeneric("next_day", function(y, x) { standardGeneric("next_day") }) + #' @rdname functions #' @export setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) @@ -790,10 +850,31 @@ setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) #' @export setGeneric("quarter", function(x) { standardGeneric("quarter") }) +#' @rdname functions +#' @export +setGeneric("rand", function(seed) { standardGeneric("rand") }) + +#' @rdname functions +#' @export +setGeneric("randn", function(seed) { standardGeneric("randn") }) + +#' @rdname functions +#' @export +setGeneric("regexp_extract", function(x, pattern, idx) { standardGeneric("regexp_extract") }) + +#' @rdname functions +#' @export +setGeneric("regexp_replace", + function(x, pattern, replacement) { standardGeneric("regexp_replace") }) + #' @rdname functions #' @export setGeneric("reverse", function(x) { standardGeneric("reverse") }) +#' @rdname functions +#' @export +setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) + #' @rdname functions #' @export setGeneric("rtrim", function(x) { standardGeneric("rtrim") }) @@ -806,6 +887,22 @@ setGeneric("second", function(x) { standardGeneric("second") }) #' @export setGeneric("sha1", function(x) { standardGeneric("sha1") }) +#' @rdname functions +#' @export +setGeneric("sha2", function(y, x) { standardGeneric("sha2") }) + +#' @rdname functions +#' @export +setGeneric("shiftLeft", function(y, x) { standardGeneric("shiftLeft") }) + +#' @rdname functions +#' @export +setGeneric("shiftRight", function(y, x) { standardGeneric("shiftRight") }) + +#' @rdname functions +#' @export +setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUnsigned") }) + #' @rdname functions #' @export setGeneric("signum", function(x) { standardGeneric("signum") }) @@ -818,6 +915,10 @@ setGeneric("size", function(x) { standardGeneric("size") }) #' @export setGeneric("soundex", function(x) { standardGeneric("soundex") }) +#' @rdname functions +#' @export +setGeneric("substring_index", function(x, delim, count) { standardGeneric("substring_index") }) + #' @rdname functions #' @export setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) @@ -834,6 +935,14 @@ setGeneric("toRadians", function(x) { standardGeneric("toRadians") }) #' @export setGeneric("to_date", function(x) { standardGeneric("to_date") }) +#' @rdname functions +#' @export +setGeneric("to_utc_timestamp", function(y, x) { standardGeneric("to_utc_timestamp") }) + +#' @rdname functions +#' @export +setGeneric("translate", function(x, matchingString, replaceString) { standardGeneric("translate") }) + #' @rdname functions #' @export setGeneric("trim", function(x) { standardGeneric("trim") }) @@ -846,6 +955,10 @@ setGeneric("unbase64", function(x) { standardGeneric("unbase64") }) #' @export setGeneric("unhex", function(x) { standardGeneric("unhex") }) +#' @rdname functions +#' @export +setGeneric("unix_timestamp", function(x, format) { standardGeneric("unix_timestamp") }) + #' @rdname functions #' @export setGeneric("upper", function(x) { standardGeneric("upper") }) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 841de657df37e..670017ed34765 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -598,6 +598,11 @@ test_that("selectExpr() on a DataFrame", { expect_equal(count(selected2), 3) }) +test_that("expr() on a DataFrame", { + df <- jsonFile(sqlContext, jsonPath) + expect_equal(collect(select(df, expr("abs(-123)")))[1, 1], 123) +}) + test_that("column calculation", { df <- jsonFile(sqlContext, jsonPath) d <- collect(select(df, alias(df$age + 1, "age2"))) @@ -667,16 +672,15 @@ test_that("column functions", { c <- SparkR:::col("a") c1 <- abs(c) + acos(c) + approxCountDistinct(c) + ascii(c) + asin(c) + atan(c) c2 <- avg(c) + base64(c) + bin(c) + bitwiseNOT(c) + cbrt(c) + ceil(c) + cos(c) - c3 <- cosh(c) + count(c) + crc32(c) + dayofmonth(c) + dayofyear(c) + exp(c) + c3 <- cosh(c) + count(c) + crc32(c) + exp(c) c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) c5 <- hour(c) + initcap(c) + isNaN(c) + last(c) + last_day(c) + length(c) c6 <- log(c) + (c) + log1p(c) + log2(c) + lower(c) + ltrim(c) + max(c) + md5(c) - c7 <- mean(c) + min(c) + minute(c) + month(c) + negate(c) + quarter(c) - c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + second(c) + sha1(c) + c7 <- mean(c) + min(c) + month(c) + negate(c) + quarter(c) + c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) c9 <- signum(c) + sin(c) + sinh(c) + size(c) + soundex(c) + sqrt(c) + sum(c) c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) - c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) + weekofyear(c) - c12 <- year(c) + c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) df <- jsonFile(sqlContext, jsonPath) df2 <- select(df, between(df$age, c(20, 30)), between(df$age, c(10, 20))) @@ -689,8 +693,11 @@ test_that("column functions", { expect_equal(collect(df3)[[1, 1]], TRUE) expect_equal(collect(df3)[[2, 1]], FALSE) expect_equal(collect(df3)[[3, 1]], TRUE) -}) + df4 <- createDataFrame(sqlContext, list(list(a = "010101"))) + expect_equal(collect(select(df4, conv(df4$a, 2, 16)))[1, 1], "15") +}) +# test_that("column binary mathfunctions", { lines <- c("{\"a\":1, \"b\":5}", "{\"a\":2, \"b\":6}", @@ -709,6 +716,13 @@ test_that("column binary mathfunctions", { expect_equal(collect(select(df, hypot(df$a, df$b)))[3, "HYPOT(a, b)"], sqrt(3^2 + 7^2)) expect_equal(collect(select(df, hypot(df$a, df$b)))[4, "HYPOT(a, b)"], sqrt(4^2 + 8^2)) ## nolint end + expect_equal(collect(select(df, shiftLeft(df$b, 1)))[4, 1], 16) + expect_equal(collect(select(df, shiftRight(df$b, 1)))[4, 1], 4) + expect_equal(collect(select(df, shiftRightUnsigned(df$b, 1)))[4, 1], 4) + expect_equal(class(collect(select(df, rand()))[2, 1]), "numeric") + expect_equal(collect(select(df, rand(1)))[1, 1], 0.45, tolerance = 0.01) + expect_equal(class(collect(select(df, randn()))[2, 1]), "numeric") + expect_equal(collect(select(df, randn(1)))[1, 1], -0.0111, tolerance = 0.01) }) test_that("string operators", { @@ -718,6 +732,78 @@ test_that("string operators", { expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") expect_equal(collect(select(df, concat(df$name, lit(":"), df$age)))[[2, 1]], "Andy:30") + expect_equal(collect(select(df, concat_ws(":", df$name)))[[2, 1]], "Andy") + expect_equal(collect(select(df, concat_ws(":", df$name, df$age)))[[2, 1]], "Andy:30") + expect_equal(collect(select(df, instr(df$name, "i")))[, 1], c(2, 0, 5)) + expect_equal(collect(select(df, format_number(df$age, 2)))[2, 1], "30.00") + expect_equal(collect(select(df, sha1(df$name)))[2, 1], + "ab5a000e88b5d9d0fa2575f5c6263eb93452405d") + expect_equal(collect(select(df, sha2(df$name, 256)))[2, 1], + "80f2aed3c618c423ddf05a2891229fba44942d907173152442cf6591441ed6dc") + expect_equal(collect(select(df, format_string("Name:%s", df$name)))[2, 1], "Name:Andy") + expect_equal(collect(select(df, format_string("%s, %d", df$name, df$age)))[2, 1], "Andy, 30") + expect_equal(collect(select(df, regexp_extract(df$name, "(n.y)", 1)))[2, 1], "ndy") + expect_equal(collect(select(df, regexp_replace(df$name, "(n.y)", "ydn")))[2, 1], "Aydn") + + l2 <- list(list(a = "aaads")) + df2 <- createDataFrame(sqlContext, l2) + expect_equal(collect(select(df2, locate("aa", df2$a)))[1, 1], 1) + expect_equal(collect(select(df2, locate("aa", df2$a, 1)))[1, 1], 2) + expect_equal(collect(select(df2, lpad(df2$a, 8, "#")))[1, 1], "###aaads") + expect_equal(collect(select(df2, rpad(df2$a, 8, "#")))[1, 1], "aaads###") + + l3 <- list(list(a = "a.b.c.d")) + df3 <- createDataFrame(sqlContext, l3) + expect_equal(collect(select(df3, substring_index(df3$a, ".", 2)))[1, 1], "a.b") + expect_equal(collect(select(df3, substring_index(df3$a, ".", -3)))[1, 1], "b.c.d") + expect_equal(collect(select(df3, translate(df3$a, "bc", "12")))[1, 1], "a.1.2.d") +}) + +test_that("date functions on a DataFrame", { + .originalTimeZone <- Sys.getenv("TZ") + Sys.setenv(TZ = "UTC") + l <- list(list(a = 1L, b = as.Date("2012-12-13")), + list(a = 2L, b = as.Date("2013-12-14")), + list(a = 3L, b = as.Date("2014-12-15"))) + df <- createDataFrame(sqlContext, l) + expect_equal(collect(select(df, dayofmonth(df$b)))[, 1], c(13, 14, 15)) + expect_equal(collect(select(df, dayofyear(df$b)))[, 1], c(348, 348, 349)) + expect_equal(collect(select(df, weekofyear(df$b)))[, 1], c(50, 50, 51)) + expect_equal(collect(select(df, year(df$b)))[, 1], c(2012, 2013, 2014)) + expect_equal(collect(select(df, month(df$b)))[, 1], c(12, 12, 12)) + expect_equal(collect(select(df, last_day(df$b)))[, 1], + c(as.Date("2012-12-31"), as.Date("2013-12-31"), as.Date("2014-12-31"))) + expect_equal(collect(select(df, next_day(df$b, "MONDAY")))[, 1], + c(as.Date("2012-12-17"), as.Date("2013-12-16"), as.Date("2014-12-22"))) + expect_equal(collect(select(df, date_format(df$b, "y")))[, 1], c("2012", "2013", "2014")) + expect_equal(collect(select(df, add_months(df$b, 3)))[, 1], + c(as.Date("2013-03-13"), as.Date("2014-03-14"), as.Date("2015-03-15"))) + expect_equal(collect(select(df, date_add(df$b, 1)))[, 1], + c(as.Date("2012-12-14"), as.Date("2013-12-15"), as.Date("2014-12-16"))) + expect_equal(collect(select(df, date_sub(df$b, 1)))[, 1], + c(as.Date("2012-12-12"), as.Date("2013-12-13"), as.Date("2014-12-14"))) + + l2 <- list(list(a = 1L, b = as.POSIXlt("2012-12-13 12:34:00", tz = "UTC")), + list(a = 2L, b = as.POSIXlt("2014-12-15 01:24:34", tz = "UTC"))) + df2 <- createDataFrame(sqlContext, l2) + expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) + expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) + expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXlt("2012-12-13 21:34:00 UTC"), as.POSIXlt("2014-12-15 10:24:34 UTC"))) + expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], + c(as.POSIXlt("2012-12-13 03:34:00 UTC"), as.POSIXlt("2014-12-14 16:24:34 UTC"))) + expect_more_than(collect(select(df2, unix_timestamp()))[1, 1], 0) + expect_more_than(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) + expect_more_than(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) + + l3 <- list(list(a = 1000), list(a = -1000)) + df3 <- createDataFrame(sqlContext, l3) + result31 <- collect(select(df3, from_unixtime(df3$a))) + expect_equal(grep("\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}", result31[, 1], perl = TRUE), + c(1, 2)) + result32 <- collect(select(df3, from_unixtime(df3$a, "yyyy"))) + expect_equal(grep("\\d{4}", result32[, 1]), c(1, 2)) + Sys.setenv(TZ = .originalTimeZone) }) test_that("greatest() and least() on a DataFrame", { diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 14dac4ed28ce3..6ce02e2ea336a 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -182,6 +182,7 @@ private[r] class RBackendHandler(server: RBackend) if (parameterType.isPrimitive) { parameterWrapperType = parameterType match { case java.lang.Integer.TYPE => classOf[java.lang.Integer] + case java.lang.Long.TYPE => classOf[java.lang.Integer] case java.lang.Double.TYPE => classOf[java.lang.Double] case java.lang.Boolean.TYPE => classOf[java.lang.Boolean] case _ => parameterType From 5fd53c64bb01de74ae57a7068de85b34adc856cf Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 19 Aug 2015 10:51:59 -0700 Subject: [PATCH 007/232] [SPARK-9833] [YARN] Add options to disable delegation token retrieval. This allows skipping the code that tries to talk to Hive and HBase to fetch delegation tokens, in case that somehow conflicts with the application being run. Author: Marcelo Vanzin Closes #8134 from vanzin/SPARK-9833. --- docs/running-on-yarn.md | 12 +++++++++ .../org/apache/spark/deploy/yarn/Client.scala | 27 ++++++++++++++----- 2 files changed, 33 insertions(+), 6 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 8ac26e98da635..5159ef9e3394e 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -369,6 +369,18 @@ If you need a reference to the proper location to put log files in the YARN so t See spark.yarn.config.gatewayPath. + + spark.yarn.security.tokens.${service}.enabled + true + + Controls whether to retrieve delegation tokens for non-HDFS services when security is enabled. + By default, delegation tokens for all supported services are retrieved when those services are + configured, but it's possible to disable that behavior if it somehow conflicts with the + application being run. +

+ Currently supported services are: hive, hbase + + # Important notes diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 262c6a8d6397f..bff585b46cbbe 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -285,8 +285,8 @@ private[spark] class Client( // multiple times, YARN will fail to launch containers for the app with an internal // error. val distributedUris = new HashSet[String] - obtainTokenForHiveMetastore(hadoopConf, credentials) - obtainTokenForHBase(hadoopConf, credentials) + obtainTokenForHiveMetastore(sparkConf, hadoopConf, credentials) + obtainTokenForHBase(sparkConf, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -1239,8 +1239,11 @@ object Client extends Logging { /** * Obtains token for the Hive metastore and adds them to the credentials. */ - private def obtainTokenForHiveMetastore(conf: Configuration, credentials: Credentials) { - if (UserGroupInformation.isSecurityEnabled) { + private def obtainTokenForHiveMetastore( + sparkConf: SparkConf, + conf: Configuration, + credentials: Credentials) { + if (shouldGetTokens(sparkConf, "hive") && UserGroupInformation.isSecurityEnabled) { val mirror = universe.runtimeMirror(getClass.getClassLoader) try { @@ -1297,8 +1300,11 @@ object Client extends Logging { /** * Obtain security token for HBase. */ - def obtainTokenForHBase(conf: Configuration, credentials: Credentials): Unit = { - if (UserGroupInformation.isSecurityEnabled) { + def obtainTokenForHBase( + sparkConf: SparkConf, + conf: Configuration, + credentials: Credentials): Unit = { + if (shouldGetTokens(sparkConf, "hbase") && UserGroupInformation.isSecurityEnabled) { val mirror = universe.runtimeMirror(getClass.getClassLoader) try { @@ -1394,4 +1400,13 @@ object Client extends Logging { components.mkString(Path.SEPARATOR) } + /** + * Return whether delegation tokens should be retrieved for the given service when security is + * enabled. By default, tokens are retrieved, but that behavior can be changed by setting + * a service-specific configuration. + */ + def shouldGetTokens(conf: SparkConf, service: String): Boolean = { + conf.getBoolean(s"spark.yarn.security.tokens.${service}.enabled", true) + } + } From 28a98464ea65aa7b35e24fca5ddaa60c2e5d53ee Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Wed, 19 Aug 2015 11:35:05 -0700 Subject: [PATCH 008/232] [SPARK-10097] Adds `shouldMaximize` flag to `ml.evaluation.Evaluator` Previously, users of evaluator (`CrossValidator` and `TrainValidationSplit`) would only maximize the metric in evaluator, leading to a hacky solution which negated metrics to be minimized and caused erroneous negative values to be reported to the user. This PR adds a `isLargerBetter` attribute to the `Evaluator` base class, instructing users of `Evaluator` on whether the chosen metric should be maximized or minimized. CC jkbradley Author: Feynman Liang Author: Joseph K. Bradley Closes #8290 from feynmanliang/SPARK-10097. --- .../BinaryClassificationEvaluator.scala | 20 +++++++++++-------- .../spark/ml/evaluation/Evaluator.scala | 7 +++++++ .../MulticlassClassificationEvaluator.scala | 8 ++++++++ .../ml/evaluation/RegressionEvaluator.scala | 19 ++++++++++-------- .../spark/ml/tuning/CrossValidator.scala | 4 +++- .../ml/tuning/TrainValidationSplit.scala | 4 +++- .../evaluation/RegressionEvaluatorSuite.scala | 4 ++-- .../spark/ml/tuning/CrossValidatorSuite.scala | 2 ++ .../ml/tuning/TrainValidationSplitSuite.scala | 2 ++ python/pyspark/ml/evaluation.py | 4 ++-- 10 files changed, 52 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 5d5cb7e94f45b..56419a0a15952 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -40,8 +40,11 @@ class BinaryClassificationEvaluator(override val uid: String) * param for metric name in evaluation * @group param */ - val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (areaUnderROC|areaUnderPR)") + val metricName: Param[String] = { + val allowedParams = ParamValidators.inArray(Array("areaUnderROC", "areaUnderPR")) + new Param( + this, "metricName", "metric name in evaluation (areaUnderROC|areaUnderPR)", allowedParams) + } /** @group getParam */ def getMetricName: String = $(metricName) @@ -76,16 +79,17 @@ class BinaryClassificationEvaluator(override val uid: String) } val metrics = new BinaryClassificationMetrics(scoreAndLabels) val metric = $(metricName) match { - case "areaUnderROC" => - metrics.areaUnderROC() - case "areaUnderPR" => - metrics.areaUnderPR() - case other => - throw new IllegalArgumentException(s"Does not support metric $other.") + case "areaUnderROC" => metrics.areaUnderROC() + case "areaUnderPR" => metrics.areaUnderPR() } metrics.unpersist() metric } + override def isLargerBetter: Boolean = $(metricName) match { + case "areaUnderROC" => true + case "areaUnderPR" => true + } + override def copy(extra: ParamMap): BinaryClassificationEvaluator = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala index e56c946a063e8..13bd3307f8a2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/Evaluator.scala @@ -46,5 +46,12 @@ abstract class Evaluator extends Params { */ def evaluate(dataset: DataFrame): Double + /** + * Indicates whether the metric returned by [[evaluate()]] should be maximized (true, default) + * or minimized (false). + * A given evaluator may support multiple metrics which may be maximized or minimized. + */ + def isLargerBetter: Boolean = true + override def copy(extra: ParamMap): Evaluator } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 44f779c1908d7..f73d2345078e6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -81,5 +81,13 @@ class MulticlassClassificationEvaluator (override val uid: String) metric } + override def isLargerBetter: Boolean = $(metricName) match { + case "f1" => true + case "precision" => true + case "recall" => true + case "weightedPrecision" => true + case "weightedRecall" => true + } + override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index 01c000b47514c..d21c88ab9b109 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -73,17 +73,20 @@ final class RegressionEvaluator(override val uid: String) } val metrics = new RegressionMetrics(predictionAndLabels) val metric = $(metricName) match { - case "rmse" => - -metrics.rootMeanSquaredError - case "mse" => - -metrics.meanSquaredError - case "r2" => - metrics.r2 - case "mae" => - -metrics.meanAbsoluteError + case "rmse" => metrics.rootMeanSquaredError + case "mse" => metrics.meanSquaredError + case "r2" => metrics.r2 + case "mae" => metrics.meanAbsoluteError } metric } + override def isLargerBetter: Boolean = $(metricName) match { + case "rmse" => false + case "mse" => false + case "r2" => true + case "mae" => false + } + override def copy(extra: ParamMap): RegressionEvaluator = defaultCopy(extra) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 4792eb0f0a288..0679bfd0f3ffe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -100,7 +100,9 @@ class CrossValidator(override val uid: String) extends Estimator[CrossValidatorM } f2jBLAS.dscal(numModels, 1.0 / $(numFolds), metrics, 1) logInfo(s"Average cross-validation metrics: ${metrics.toSeq}") - val (bestMetric, bestIndex) = metrics.zipWithIndex.maxBy(_._1) + val (bestMetric, bestIndex) = + if (eval.isLargerBetter) metrics.zipWithIndex.maxBy(_._1) + else metrics.zipWithIndex.minBy(_._1) logInfo(s"Best set of parameters:\n${epm(bestIndex)}") logInfo(s"Best cross-validation metric: $bestMetric.") val bestModel = est.fit(dataset, epm(bestIndex)).asInstanceOf[Model[_]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index c0edc730b6fd6..73a14b8310157 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -99,7 +99,9 @@ class TrainValidationSplit(override val uid: String) extends Estimator[TrainVali validationDataset.unpersist() logInfo(s"Train validation split metrics: ${metrics.toSeq}") - val (bestMetric, bestIndex) = metrics.zipWithIndex.maxBy(_._1) + val (bestMetric, bestIndex) = + if (eval.isLargerBetter) metrics.zipWithIndex.maxBy(_._1) + else metrics.zipWithIndex.minBy(_._1) logInfo(s"Best set of parameters:\n${epm(bestIndex)}") logInfo(s"Best train validation split metric: $bestMetric.") val bestModel = est.fit(dataset, epm(bestIndex)).asInstanceOf[Model[_]] diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala index 5b203784559e2..aa722da323935 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RegressionEvaluatorSuite.scala @@ -63,7 +63,7 @@ class RegressionEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext // default = rmse val evaluator = new RegressionEvaluator() - assert(evaluator.evaluate(predictions) ~== -0.1019382 absTol 0.001) + assert(evaluator.evaluate(predictions) ~== 0.1019382 absTol 0.001) // r2 score evaluator.setMetricName("r2") @@ -71,6 +71,6 @@ class RegressionEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext // mae evaluator.setMetricName("mae") - assert(evaluator.evaluate(predictions) ~== -0.08036075 absTol 0.001) + assert(evaluator.evaluate(predictions) ~== 0.08036075 absTol 0.001) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index aaca08bb61a45..fde02e0c84bc0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -143,6 +143,8 @@ object CrossValidatorSuite { throw new UnsupportedOperationException } + override def isLargerBetter: Boolean = true + override val uid: String = "eval" override def copy(extra: ParamMap): MyEvaluator = defaultCopy(extra) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index c8e58f216cceb..ef24e6fb6b80f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -132,6 +132,8 @@ object TrainValidationSplitSuite { throw new UnsupportedOperationException } + override def isLargerBetter: Boolean = true + override val uid: String = "eval" override def copy(extra: ParamMap): MyEvaluator = defaultCopy(extra) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index e23ce053baeb7..6b0a9ffde9f42 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -163,11 +163,11 @@ class RegressionEvaluator(JavaEvaluator, HasLabelCol, HasPredictionCol): ... >>> evaluator = RegressionEvaluator(predictionCol="raw") >>> evaluator.evaluate(dataset) - -2.842... + 2.842... >>> evaluator.evaluate(dataset, {evaluator.metricName: "r2"}) 0.993... >>> evaluator.evaluate(dataset, {evaluator.metricName: "mae"}) - -2.649... + 2.649... """ # Because we will maximize evaluation value (ref: `CrossValidator`), # when we evaluate a metric that is needed to minimize (e.g., `"rmse"`, `"mse"`, `"mae"`), From d898c33f774b9a3db2fb6aa8f0cb2c2ac6004b58 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Wed, 19 Aug 2015 12:39:37 -0700 Subject: [PATCH 009/232] [SPARK-10106] [SPARKR] Add `ifelse` Column function to SparkR ### JIRA [[SPARK-10106] Add `ifelse` Column function to SparkR - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10106) Author: Yu ISHIKAWA Closes #8303 from yu-iskw/SPARK-10106. --- R/pkg/NAMESPACE | 1 + R/pkg/R/functions.R | 19 +++++++++++++++++++ R/pkg/inst/tests/test_sparkSQL.R | 3 ++- 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 111a2dc30db73..3e5c89d779b7b 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -135,6 +135,7 @@ exportMethods("abs", "hex", "hour", "hypot", + "ifelse", "initcap", "instr", "isNaN", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 5dba0887d150d..b5879bd9ad553 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -594,3 +594,22 @@ setMethod("when", signature(condition = "Column", value = "ANY"), jc <- callJStatic("org.apache.spark.sql.functions", "when", condition, value) column(jc) }) + +#' ifelse +#' +#' Evaluates a list of conditions and returns `yes` if the conditions are satisfied. +#' Otherwise `no` is returned for unmatched conditions. +#' +#' @rdname column +setMethod("ifelse", + signature(test = "Column", yes = "ANY", no = "ANY"), + function(test, yes, no) { + test <- test@jc + yes <- ifelse(class(yes) == "Column", yes@jc, yes) + no <- ifelse(class(no) == "Column", no@jc, no) + jc <- callJMethod(callJStatic("org.apache.spark.sql.functions", + "when", + test, yes), + "otherwise", no) + column(jc) + }) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 670017ed34765..556b8c5447054 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -813,11 +813,12 @@ test_that("greatest() and least() on a DataFrame", { expect_equal(collect(select(df, least(df$a, df$b)))[, 1], c(1, 3)) }) -test_that("when() and otherwise() on a DataFrame", { +test_that("when(), otherwise() and ifelse() on a DataFrame", { l <- list(list(a = 1, b = 2), list(a = 3, b = 4)) df <- createDataFrame(sqlContext, l) expect_equal(collect(select(df, when(df$a > 1 & df$b > 2, 1)))[, 1], c(NA, 1)) expect_equal(collect(select(df, otherwise(when(df$a > 1, 1), 0)))[, 1], c(0, 1)) + expect_equal(collect(select(df, ifelse(df$a > 1 & df$b > 2, 0, 1)))[, 1], c(1, 0)) }) test_that("group by", { From 5b62bef8cbf73f910513ef3b1f557aa94b384854 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 19 Aug 2015 13:17:26 -0700 Subject: [PATCH 010/232] [SPARK-8918] [MLLIB] [DOC] Add @since tags to mllib.clustering This continues the work from #8256. I removed `since` tags from private/protected/local methods/variables (see https://github.com/apache/spark/commit/72fdeb64630470f6f46cf3eed8ffbfe83a7c4659). MechCoder Closes #8256 Author: Xiangrui Meng Author: Xiaoqing Wang Author: MechCoder Closes #8288 from mengxr/SPARK-8918. --- .../mllib/clustering/GaussianMixture.scala | 56 +++++++++++---- .../clustering/GaussianMixtureModel.scala | 32 +++++++-- .../spark/mllib/clustering/KMeans.scala | 36 +++++++++- .../spark/mllib/clustering/KMeansModel.scala | 37 ++++++++-- .../apache/spark/mllib/clustering/LDA.scala | 71 ++++++++++++++++--- .../spark/mllib/clustering/LDAModel.scala | 64 +++++++++++++++-- .../spark/mllib/clustering/LDAOptimizer.scala | 12 +++- .../clustering/PowerIterationClustering.scala | 29 +++++++- .../mllib/clustering/StreamingKMeans.scala | 53 +++++++++++--- 9 files changed, 338 insertions(+), 52 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index e459367333d26..bc27b1fe7390b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -62,6 +62,7 @@ class GaussianMixture private ( /** * Constructs a default instance. The default parameters are {k: 2, convergenceTol: 0.01, * maxIterations: 100, seed: random}. + * @since 1.3.0 */ def this() = this(2, 0.01, 100, Utils.random.nextLong()) @@ -72,9 +73,11 @@ class GaussianMixture private ( // default random starting point private var initialModel: Option[GaussianMixtureModel] = None - /** Set the initial GMM starting point, bypassing the random initialization. - * You must call setK() prior to calling this method, and the condition - * (model.k == this.k) must be met; failure will result in an IllegalArgumentException + /** + * Set the initial GMM starting point, bypassing the random initialization. + * You must call setK() prior to calling this method, and the condition + * (model.k == this.k) must be met; failure will result in an IllegalArgumentException + * @since 1.3.0 */ def setInitialModel(model: GaussianMixtureModel): this.type = { if (model.k == k) { @@ -85,30 +88,46 @@ class GaussianMixture private ( this } - /** Return the user supplied initial GMM, if supplied */ + /** + * Return the user supplied initial GMM, if supplied + * @since 1.3.0 + */ def getInitialModel: Option[GaussianMixtureModel] = initialModel - /** Set the number of Gaussians in the mixture model. Default: 2 */ + /** + * Set the number of Gaussians in the mixture model. Default: 2 + * @since 1.3.0 + */ def setK(k: Int): this.type = { this.k = k this } - /** Return the number of Gaussians in the mixture model */ + /** + * Return the number of Gaussians in the mixture model + * @since 1.3.0 + */ def getK: Int = k - /** Set the maximum number of iterations to run. Default: 100 */ + /** + * Set the maximum number of iterations to run. Default: 100 + * @since 1.3.0 + */ def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this } - /** Return the maximum number of iterations to run */ + /** + * Return the maximum number of iterations to run + * @since 1.3.0 + */ def getMaxIterations: Int = maxIterations /** * Set the largest change in log-likelihood at which convergence is * considered to have occurred. + * @since 1.3.0 */ def setConvergenceTol(convergenceTol: Double): this.type = { this.convergenceTol = convergenceTol @@ -118,19 +137,29 @@ class GaussianMixture private ( /** * Return the largest change in log-likelihood at which convergence is * considered to have occurred. + * @since 1.3.0 */ def getConvergenceTol: Double = convergenceTol - /** Set the random seed */ + /** + * Set the random seed + * @since 1.3.0 + */ def setSeed(seed: Long): this.type = { this.seed = seed this } - /** Return the random seed */ + /** + * Return the random seed + * @since 1.3.0 + */ def getSeed: Long = seed - /** Perform expectation maximization */ + /** + * Perform expectation maximization + * @since 1.3.0 + */ def run(data: RDD[Vector]): GaussianMixtureModel = { val sc = data.sparkContext @@ -204,7 +233,10 @@ class GaussianMixture private ( new GaussianMixtureModel(weights, gaussians) } - /** Java-friendly version of [[run()]] */ + /** + * Java-friendly version of [[run()]] + * @since 1.3.0 + */ def run(data: JavaRDD[Vector]): GaussianMixtureModel = run(data.rdd) private def updateWeightsAndGaussians( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 76aeebd703d4e..2fa0473737aae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.{SQLContext, Row} * the weight for Gaussian i, and weights.sum == 1 * @param gaussians Array of MultivariateGaussian where gaussians(i) represents * the Multivariate Gaussian (Normal) Distribution for Gaussian i + * @since 1.3.0 */ @Experimental class GaussianMixtureModel( @@ -53,32 +54,48 @@ class GaussianMixtureModel( override protected def formatVersion = "1.0" + /** + * @since 1.4.0 + */ override def save(sc: SparkContext, path: String): Unit = { GaussianMixtureModel.SaveLoadV1_0.save(sc, path, weights, gaussians) } - /** Number of gaussians in mixture */ + /** + * Number of gaussians in mixture + * @since 1.3.0 + */ def k: Int = weights.length - /** Maps given points to their cluster indices. */ + /** + * Maps given points to their cluster indices. + * @since 1.3.0 + */ def predict(points: RDD[Vector]): RDD[Int] = { val responsibilityMatrix = predictSoft(points) responsibilityMatrix.map(r => r.indexOf(r.max)) } - /** Maps given point to its cluster index. */ + /** + * Maps given point to its cluster index. + * @since 1.5.0 + */ def predict(point: Vector): Int = { val r = computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) r.indexOf(r.max) } - /** Java-friendly version of [[predict()]] */ + /** + * Java-friendly version of [[predict()]] + * @since 1.4.0 + */ def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] /** * Given the input vectors, return the membership value of each vector * to all mixture components. + * @since 1.3.0 */ def predictSoft(points: RDD[Vector]): RDD[Array[Double]] = { val sc = points.sparkContext @@ -91,6 +108,7 @@ class GaussianMixtureModel( /** * Given the input vector, return the membership values to all mixture components. + * @since 1.4.0 */ def predictSoft(point: Vector): Array[Double] = { computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) @@ -115,6 +133,9 @@ class GaussianMixtureModel( } } +/** + * @since 1.4.0 + */ @Experimental object GaussianMixtureModel extends Loader[GaussianMixtureModel] { @@ -165,6 +186,9 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { } } + /** + * @since 1.4.0 + */ override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats 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 0a65403f4ec95..9ef6834e5ea8d 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 @@ -49,15 +49,20 @@ class KMeans private ( /** * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4, seed: random}. + * @since 0.8.0 */ def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong()) /** * Number of clusters to create (k). + * @since 1.4.0 */ def getK: Int = k - /** Set the number of clusters to create (k). Default: 2. */ + /** + * Set the number of clusters to create (k). Default: 2. + * @since 0.8.0 + */ def setK(k: Int): this.type = { this.k = k this @@ -65,10 +70,14 @@ class KMeans private ( /** * Maximum number of iterations to run. + * @since 1.4.0 */ def getMaxIterations: Int = maxIterations - /** Set maximum number of iterations to run. Default: 20. */ + /** + * Set maximum number of iterations to run. Default: 20. + * @since 0.8.0 + */ def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this @@ -76,6 +85,7 @@ class KMeans private ( /** * The initialization algorithm. This can be either "random" or "k-means||". + * @since 1.4.0 */ def getInitializationMode: String = initializationMode @@ -83,6 +93,7 @@ class KMeans private ( * Set the initialization algorithm. This can be either "random" to choose random points as * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. + * @since 0.8.0 */ def setInitializationMode(initializationMode: String): this.type = { KMeans.validateInitMode(initializationMode) @@ -93,6 +104,7 @@ class KMeans private ( /** * :: Experimental :: * Number of runs of the algorithm to execute in parallel. + * @since 1.4.0 */ @Experimental def getRuns: Int = runs @@ -102,6 +114,7 @@ class KMeans private ( * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. + * @since 0.8.0 */ @Experimental def setRuns(runs: Int): this.type = { @@ -114,12 +127,14 @@ class KMeans private ( /** * Number of steps for the k-means|| initialization mode + * @since 1.4.0 */ def getInitializationSteps: Int = initializationSteps /** * Set the number of steps for the k-means|| initialization mode. This is an advanced * setting -- the default of 5 is almost always enough. Default: 5. + * @since 0.8.0 */ def setInitializationSteps(initializationSteps: Int): this.type = { if (initializationSteps <= 0) { @@ -131,12 +146,14 @@ class KMeans private ( /** * The distance threshold within which we've consider centers to have converged. + * @since 1.4.0 */ def getEpsilon: Double = epsilon /** * Set the distance threshold within which we've consider centers to have converged. * If all centers move less than this Euclidean distance, we stop iterating one run. + * @since 0.8.0 */ def setEpsilon(epsilon: Double): this.type = { this.epsilon = epsilon @@ -145,10 +162,14 @@ class KMeans private ( /** * The random seed for cluster initialization. + * @since 1.4.0 */ def getSeed: Long = seed - /** Set the random seed for cluster initialization. */ + /** + * Set the random seed for cluster initialization. + * @since 1.4.0 + */ def setSeed(seed: Long): this.type = { this.seed = seed this @@ -162,6 +183,7 @@ class KMeans private ( * Set the initial starting point, bypassing the random initialization or k-means|| * The condition model.k == this.k must be met, failure results * in an IllegalArgumentException. + * @since 1.4.0 */ def setInitialModel(model: KMeansModel): this.type = { require(model.k == k, "mismatched cluster count") @@ -172,6 +194,7 @@ class KMeans private ( /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. + * @since 0.8.0 */ def run(data: RDD[Vector]): KMeansModel = { @@ -430,11 +453,14 @@ class KMeans private ( /** * Top-level methods for calling K-means clustering. + * @since 0.8.0 */ object KMeans { // Initialization mode names + /** @since 0.8.0 */ val RANDOM = "random" + /** @since 0.8.0 */ val K_MEANS_PARALLEL = "k-means||" /** @@ -446,6 +472,7 @@ object KMeans { * @param runs number of parallel runs, defaults to 1. The best model is returned. * @param initializationMode initialization model, either "random" or "k-means||" (default). * @param seed random seed value for cluster initialization + * @since 1.3.0 */ def train( data: RDD[Vector], @@ -470,6 +497,7 @@ object KMeans { * @param maxIterations max number of iterations * @param runs number of parallel runs, defaults to 1. The best model is returned. * @param initializationMode initialization model, either "random" or "k-means||" (default). + * @since 0.8.0 */ def train( data: RDD[Vector], @@ -486,6 +514,7 @@ object KMeans { /** * Trains a k-means model using specified parameters and the default values for unspecified. + * @since 0.8.0 */ def train( data: RDD[Vector], @@ -496,6 +525,7 @@ object KMeans { /** * Trains a k-means model using specified parameters and the default values for unspecified. + * @since 0.8.0 */ def train( data: RDD[Vector], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 96359024fa228..8de2087ceb4df 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -34,35 +34,52 @@ import org.apache.spark.sql.Row /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. + * @since 0.8.0 */ class KMeansModel ( val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { - /** A Java-friendly constructor that takes an Iterable of Vectors. */ + /** + * A Java-friendly constructor that takes an Iterable of Vectors. + * @since 1.4.0 + */ def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) - /** Total number of clusters. */ + /** + * Total number of clusters. + * @since 0.8.0 + */ def k: Int = clusterCenters.length - /** Returns the cluster index that a given point belongs to. */ + /** + * Returns the cluster index that a given point belongs to. + * @since 0.8.0 + */ def predict(point: Vector): Int = { KMeans.findClosest(clusterCentersWithNorm, new VectorWithNorm(point))._1 } - /** Maps given points to their cluster indices. */ + /** + * Maps given points to their cluster indices. + * @since 1.0.0 + */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm val bcCentersWithNorm = points.context.broadcast(centersWithNorm) points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new VectorWithNorm(p))._1) } - /** Maps given points to their cluster indices. */ + /** + * Maps given points to their cluster indices. + * @since 1.0.0 + */ def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. + * @since 0.8.0 */ def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm @@ -73,6 +90,9 @@ class KMeansModel ( private def clusterCentersWithNorm: Iterable[VectorWithNorm] = clusterCenters.map(new VectorWithNorm(_)) + /** + * @since 1.4.0 + */ override def save(sc: SparkContext, path: String): Unit = { KMeansModel.SaveLoadV1_0.save(sc, this, path) } @@ -80,7 +100,14 @@ class KMeansModel ( override protected def formatVersion: String = "1.0" } +/** + * @since 1.4.0 + */ object KMeansModel extends Loader[KMeansModel] { + + /** + * @since 1.4.0 + */ override def load(sc: SparkContext, path: String): KMeansModel = { KMeansModel.SaveLoadV1_0.load(sc, path) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 0fc9b1ac4d716..2a8c6acbaec61 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -43,6 +43,7 @@ import org.apache.spark.util.Utils * * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation * (Wikipedia)]] + * @since 1.3.0 */ @Experimental class LDA private ( @@ -54,18 +55,25 @@ class LDA private ( private var checkpointInterval: Int, private var ldaOptimizer: LDAOptimizer) extends Logging { + /** + * Constructs a LDA instance with default parameters. + * @since 1.3.0 + */ def this() = this(k = 10, maxIterations = 20, docConcentration = Vectors.dense(-1), topicConcentration = -1, seed = Utils.random.nextLong(), checkpointInterval = 10, ldaOptimizer = new EMLDAOptimizer) /** * Number of topics to infer. I.e., the number of soft cluster centers. + * + * @since 1.3.0 */ def getK: Int = k /** * Number of topics to infer. I.e., the number of soft cluster centers. * (default = 10) + * @since 1.3.0 */ def setK(k: Int): this.type = { require(k > 0, s"LDA k (number of clusters) must be > 0, but was set to $k") @@ -78,6 +86,7 @@ class LDA private ( * distributions over topics ("theta"). * * This is the parameter to a Dirichlet distribution. + * @since 1.5.0 */ def getAsymmetricDocConcentration: Vector = this.docConcentration @@ -87,6 +96,7 @@ class LDA private ( * * This method assumes the Dirichlet distribution is symmetric and can be described by a single * [[Double]] parameter. It should fail if docConcentration is asymmetric. + * @since 1.3.0 */ def getDocConcentration: Double = { val parameter = docConcentration(0) @@ -121,6 +131,7 @@ class LDA private ( * - Values should be >= 0 * - default = uniformly (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. + * @since 1.5.0 */ def setDocConcentration(docConcentration: Vector): this.type = { require(docConcentration.size > 0, "docConcentration must have > 0 elements") @@ -128,22 +139,37 @@ class LDA private ( this } - /** Replicates a [[Double]] docConcentration to create a symmetric prior. */ + /** + * Replicates a [[Double]] docConcentration to create a symmetric prior. + * @since 1.3.0 + */ def setDocConcentration(docConcentration: Double): this.type = { this.docConcentration = Vectors.dense(docConcentration) this } - /** Alias for [[getAsymmetricDocConcentration]] */ + /** + * Alias for [[getAsymmetricDocConcentration]] + * @since 1.5.0 + */ def getAsymmetricAlpha: Vector = getAsymmetricDocConcentration - /** Alias for [[getDocConcentration]] */ + /** + * Alias for [[getDocConcentration]] + * @since 1.3.0 + */ def getAlpha: Double = getDocConcentration - /** Alias for [[setDocConcentration()]] */ + /** + * Alias for [[setDocConcentration()]] + * @since 1.5.0 + */ def setAlpha(alpha: Vector): this.type = setDocConcentration(alpha) - /** Alias for [[setDocConcentration()]] */ + /** + * Alias for [[setDocConcentration()]] + * @since 1.3.0 + */ def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) /** @@ -154,6 +180,7 @@ class LDA private ( * * Note: The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. + * @since 1.3.0 */ def getTopicConcentration: Double = this.topicConcentration @@ -178,36 +205,51 @@ class LDA private ( * - Value should be >= 0 * - default = (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. + * @since 1.3.0 */ def setTopicConcentration(topicConcentration: Double): this.type = { this.topicConcentration = topicConcentration this } - /** Alias for [[getTopicConcentration]] */ + /** + * Alias for [[getTopicConcentration]] + * @since 1.3.0 + */ def getBeta: Double = getTopicConcentration - /** Alias for [[setTopicConcentration()]] */ + /** + * Alias for [[setTopicConcentration()]] + * @since 1.3.0 + */ def setBeta(beta: Double): this.type = setTopicConcentration(beta) /** * Maximum number of iterations for learning. + * @since 1.3.0 */ def getMaxIterations: Int = maxIterations /** * Maximum number of iterations for learning. * (default = 20) + * @since 1.3.0 */ def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this } - /** Random seed */ + /** + * Random seed + * @since 1.3.0 + */ def getSeed: Long = seed - /** Random seed */ + /** + * Random seed + * @since 1.3.0 + */ def setSeed(seed: Long): this.type = { this.seed = seed this @@ -215,6 +257,7 @@ class LDA private ( /** * Period (in iterations) between checkpoints. + * @since 1.3.0 */ def getCheckpointInterval: Int = checkpointInterval @@ -225,6 +268,7 @@ class LDA private ( * [[org.apache.spark.SparkContext]], this setting is ignored. * * @see [[org.apache.spark.SparkContext#setCheckpointDir]] + * @since 1.3.0 */ def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval @@ -236,6 +280,7 @@ class LDA private ( * :: DeveloperApi :: * * LDAOptimizer used to perform the actual calculation + * @since 1.4.0 */ @DeveloperApi def getOptimizer: LDAOptimizer = ldaOptimizer @@ -244,6 +289,7 @@ class LDA private ( * :: DeveloperApi :: * * LDAOptimizer used to perform the actual calculation (default = EMLDAOptimizer) + * @since 1.4.0 */ @DeveloperApi def setOptimizer(optimizer: LDAOptimizer): this.type = { @@ -254,6 +300,7 @@ class LDA private ( /** * Set the LDAOptimizer used to perform the actual calculation by algorithm name. * Currently "em", "online" are supported. + * @since 1.4.0 */ def setOptimizer(optimizerName: String): this.type = { this.ldaOptimizer = @@ -274,6 +321,7 @@ class LDA private ( * (where the vocabulary size is the length of the vector). * Document IDs must be unique and >= 0. * @return Inferred LDA model + * @since 1.3.0 */ def run(documents: RDD[(Long, Vector)]): LDAModel = { val state = ldaOptimizer.initialize(documents, this) @@ -289,7 +337,10 @@ class LDA private ( state.getLDAModel(iterationTimes) } - /** Java-friendly version of [[run()]] */ + /** + * Java-friendly version of [[run()]] + * @since 1.3.0 + */ def run(documents: JavaPairRDD[java.lang.Long, Vector]): LDAModel = { run(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 82f05e4a18cee..b70e380c0393e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -192,12 +192,24 @@ class LocalLDAModel private[clustering] ( override protected[clustering] val gammaShape: Double = 100) extends LDAModel with Serializable { + /** + * @since 1.3.0 + */ override def k: Int = topics.numCols + /** + * @since 1.3.0 + */ override def vocabSize: Int = topics.numRows + /** + * @since 1.3.0 + */ override def topicsMatrix: Matrix = topics + /** + * @since 1.3.0 + */ override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val brzTopics = topics.toBreeze.toDenseMatrix Range(0, k).map { topicIndex => @@ -210,6 +222,9 @@ class LocalLDAModel private[clustering] ( override protected def formatVersion = "1.0" + /** + * @since 1.5.0 + */ override def save(sc: SparkContext, path: String): Unit = { LocalLDAModel.SaveLoadV1_0.save(sc, path, topicsMatrix, docConcentration, topicConcentration, gammaShape) @@ -223,12 +238,16 @@ class LocalLDAModel private[clustering] ( * * @param documents test corpus to use for calculating log likelihood * @return variational lower bound on the log likelihood of the entire corpus + * @since 1.5.0 */ def logLikelihood(documents: RDD[(Long, Vector)]): Double = logLikelihoodBound(documents, docConcentration, topicConcentration, topicsMatrix.toBreeze.toDenseMatrix, gammaShape, k, vocabSize) - /** Java-friendly version of [[logLikelihood]] */ + /** + * Java-friendly version of [[logLikelihood]] + * @since 1.5.0 + */ def logLikelihood(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logLikelihood(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } @@ -239,6 +258,7 @@ class LocalLDAModel private[clustering] ( * * @param documents test corpus to use for calculating perplexity * @return Variational upper bound on log perplexity per token. + * @since 1.5.0 */ def logPerplexity(documents: RDD[(Long, Vector)]): Double = { val corpusTokenCount = documents @@ -247,7 +267,9 @@ class LocalLDAModel private[clustering] ( -logLikelihood(documents) / corpusTokenCount } - /** Java-friendly version of [[logPerplexity]] */ + /** Java-friendly version of [[logPerplexity]] + * @since 1.5.0 + */ def logPerplexity(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logPerplexity(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } @@ -325,6 +347,7 @@ class LocalLDAModel private[clustering] ( * for each document. * @param documents documents to predict topic mixture distributions for * @return An RDD of (document ID, topic mixture distribution for document) + * @since 1.3.0 */ // TODO: declare in LDAModel and override once implemented in DistributedLDAModel def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = { @@ -351,7 +374,10 @@ class LocalLDAModel private[clustering] ( } } - /** Java-friendly version of [[topicDistributions]] */ + /** + * Java-friendly version of [[topicDistributions]] + * @since 1.4.1 + */ def topicDistributions( documents: JavaPairRDD[java.lang.Long, Vector]): JavaPairRDD[java.lang.Long, Vector] = { val distributions = topicDistributions(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) @@ -425,6 +451,9 @@ object LocalLDAModel extends Loader[LocalLDAModel] { } } + /** + * @since 1.5.0 + */ override def load(sc: SparkContext, path: String): LocalLDAModel = { val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats @@ -481,6 +510,7 @@ class DistributedLDAModel private[clustering] ( * Convert model to a local model. * The local model stores the inferred topics but not the topic distributions for training * documents. + * @since 1.3.0 */ def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix, docConcentration, topicConcentration, gammaShape) @@ -491,6 +521,7 @@ class DistributedLDAModel private[clustering] ( * No guarantees are given about the ordering of the topics. * * WARNING: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. + * @since 1.3.0 */ override lazy val topicsMatrix: Matrix = { // Collect row-major topics @@ -510,6 +541,9 @@ class DistributedLDAModel private[clustering] ( Matrices.fromBreeze(brzTopics) } + /** + * @since 1.3.0 + */ override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val numTopics = k // Note: N_k is not needed to find the top terms, but it is needed to normalize weights @@ -548,6 +582,7 @@ class DistributedLDAModel private[clustering] ( * @return Array over topics. Each element represent as a pair of matching arrays: * (IDs for the documents, weights of the topic in these documents). * For each topic, documents are sorted in order of decreasing topic weights. + * @since 1.5.0 */ def topDocumentsPerTopic(maxDocumentsPerTopic: Int): Array[(Array[Long], Array[Double])] = { val numTopics = k @@ -587,6 +622,7 @@ class DistributedLDAModel private[clustering] ( * - This excludes the prior; for that, use [[logPrior]]. * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the * hyperparameters. + * @since 1.3.0 */ lazy val logLikelihood: Double = { // TODO: generalize this for asymmetric (non-scalar) alpha @@ -612,7 +648,8 @@ class DistributedLDAModel private[clustering] ( /** * Log probability of the current parameter estimate: - * log P(topics, topic distributions for docs | alpha, eta) + * log P(topics, topic distributions for docs | alpha, eta) + * @since 1.3.0 */ lazy val logPrior: Double = { // TODO: generalize this for asymmetric (non-scalar) alpha @@ -644,6 +681,7 @@ class DistributedLDAModel private[clustering] ( * ("theta_doc"). * * @return RDD of (document ID, topic distribution) pairs + * @since 1.3.0 */ def topicDistributions: RDD[(Long, Vector)] = { graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => @@ -651,7 +689,10 @@ class DistributedLDAModel private[clustering] ( } } - /** Java-friendly version of [[topicDistributions]] */ + /** + * Java-friendly version of [[topicDistributions]] + * @since 1.4.1 + */ def javaTopicDistributions: JavaPairRDD[java.lang.Long, Vector] = { JavaPairRDD.fromRDD(topicDistributions.asInstanceOf[RDD[(java.lang.Long, Vector)]]) } @@ -659,6 +700,7 @@ class DistributedLDAModel private[clustering] ( /** * For each document, return the top k weighted topics for that document and their weights. * @return RDD of (doc ID, topic indices, topic weights) + * @since 1.5.0 */ def topTopicsPerDocument(k: Int): RDD[(Long, Array[Int], Array[Double])] = { graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => @@ -673,7 +715,10 @@ class DistributedLDAModel private[clustering] ( } } - /** Java-friendly version of [[topTopicsPerDocument]] */ + /** + * Java-friendly version of [[topTopicsPerDocument]] + * @since 1.5.0 + */ def javaTopTopicsPerDocument(k: Int): JavaRDD[(java.lang.Long, Array[Int], Array[Double])] = { val topics = topTopicsPerDocument(k) topics.asInstanceOf[RDD[(java.lang.Long, Array[Int], Array[Double])]].toJavaRDD() @@ -684,6 +729,10 @@ class DistributedLDAModel private[clustering] ( override protected def formatVersion = "1.0" + /** + * Java-friendly version of [[topicDistributions]] + * @since 1.5.0 + */ override def save(sc: SparkContext, path: String): Unit = { DistributedLDAModel.SaveLoadV1_0.save( sc, path, graph, globalTopicTotals, k, vocabSize, docConcentration, topicConcentration, @@ -784,6 +833,9 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { } + /** + * @since 1.5.0 + */ override def load(sc: SparkContext, path: String): DistributedLDAModel = { val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index a0008f9c99ad7..360241c8081ac 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -35,6 +35,7 @@ import org.apache.spark.rdd.RDD * * An LDAOptimizer specifies which optimization/learning/inference algorithm to use, and it can * hold optimizer-specific parameters for users to set. + * @since 1.4.0 */ @DeveloperApi sealed trait LDAOptimizer { @@ -73,7 +74,7 @@ sealed trait LDAOptimizer { * - Paper which clearly explains several algorithms, including EM: * Asuncion, Welling, Smyth, and Teh. * "On Smoothing and Inference for Topic Models." UAI, 2009. - * + * @since 1.4.0 */ @DeveloperApi final class EMLDAOptimizer extends LDAOptimizer { @@ -225,6 +226,7 @@ final class EMLDAOptimizer extends LDAOptimizer { * * Original Online LDA paper: * Hoffman, Blei and Bach, "Online Learning for Latent Dirichlet Allocation." NIPS, 2010. + * @since 1.4.0 */ @DeveloperApi final class OnlineLDAOptimizer extends LDAOptimizer { @@ -274,6 +276,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. + * @since 1.4.0 */ def getTau0: Double = this.tau0 @@ -281,6 +284,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. * Default: 1024, following the original Online LDA paper. + * @since 1.4.0 */ def setTau0(tau0: Double): this.type = { require(tau0 > 0, s"LDA tau0 must be positive, but was set to $tau0") @@ -290,6 +294,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Learning rate: exponential decay rate + * @since 1.4.0 */ def getKappa: Double = this.kappa @@ -297,6 +302,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * Learning rate: exponential decay rate---should be between * (0.5, 1.0] to guarantee asymptotic convergence. * Default: 0.51, based on the original Online LDA paper. + * @since 1.4.0 */ def setKappa(kappa: Double): this.type = { require(kappa >= 0, s"Online LDA kappa must be nonnegative, but was set to $kappa") @@ -306,6 +312,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Mini-batch fraction, which sets the fraction of document sampled and used in each iteration + * @since 1.4.0 */ def getMiniBatchFraction: Double = this.miniBatchFraction @@ -318,6 +325,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * maxIterations * miniBatchFraction >= 1. * * Default: 0.05, i.e., 5% of total documents. + * @since 1.4.0 */ def setMiniBatchFraction(miniBatchFraction: Double): this.type = { require(miniBatchFraction > 0.0 && miniBatchFraction <= 1.0, @@ -329,6 +337,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Optimize alpha, indicates whether alpha (Dirichlet parameter for document-topic distribution) * will be optimized during training. + * @since 1.5.0 */ def getOptimzeAlpha: Boolean = this.optimizeAlpha @@ -336,6 +345,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * Sets whether to optimize alpha parameter during training. * * Default: false + * @since 1.5.0 */ def setOptimzeAlpha(optimizeAlpha: Boolean): this.type = { this.optimizeAlpha = optimizeAlpha diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 407e43a024a2e..b4733ca975152 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -39,12 +39,16 @@ import org.apache.spark.{Logging, SparkContext, SparkException} * * @param k number of clusters * @param assignments an RDD of clustering [[PowerIterationClustering#Assignment]]s + * @since 1.3.0 */ @Experimental class PowerIterationClusteringModel( val k: Int, val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { + /** + * @since 1.4.0 + */ override def save(sc: SparkContext, path: String): Unit = { PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) } @@ -52,6 +56,9 @@ class PowerIterationClusteringModel( override protected def formatVersion: String = "1.0" } +/** + * @since 1.4.0 + */ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) @@ -65,6 +72,9 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode private[clustering] val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" + /** + * @since 1.4.0 + */ def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { val sqlContext = new SQLContext(sc) import sqlContext.implicits._ @@ -77,6 +87,9 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode dataRDD.write.parquet(Loader.dataPath(path)) } + /** + * @since 1.4.0 + */ def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { implicit val formats = DefaultFormats val sqlContext = new SQLContext(sc) @@ -120,13 +133,16 @@ class PowerIterationClustering private[clustering] ( import org.apache.spark.mllib.clustering.PowerIterationClustering._ - /** Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100, - * initMode: "random"}. + /** + * Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100, + * initMode: "random"}. + * @since 1.3.0 */ def this() = this(k = 2, maxIterations = 100, initMode = "random") /** * Set the number of clusters. + * @since 1.3.0 */ def setK(k: Int): this.type = { this.k = k @@ -135,6 +151,7 @@ class PowerIterationClustering private[clustering] ( /** * Set maximum number of iterations of the power iteration loop + * @since 1.3.0 */ def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations @@ -144,6 +161,7 @@ class PowerIterationClustering private[clustering] ( /** * Set the initialization mode. This can be either "random" to use a random vector * as vertex properties, or "degree" to use normalized sum similarities. Default: random. + * @since 1.3.0 */ def setInitializationMode(mode: String): this.type = { this.initMode = mode match { @@ -164,6 +182,7 @@ class PowerIterationClustering private[clustering] ( * assume s,,ij,, = 0.0. * * @return a [[PowerIterationClusteringModel]] that contains the clustering result + * @since 1.5.0 */ def run(graph: Graph[Double, Double]): PowerIterationClusteringModel = { val w = normalize(graph) @@ -185,6 +204,7 @@ class PowerIterationClustering private[clustering] ( * assume s,,ij,, = 0.0. * * @return a [[PowerIterationClusteringModel]] that contains the clustering result + * @since 1.3.0 */ def run(similarities: RDD[(Long, Long, Double)]): PowerIterationClusteringModel = { val w = normalize(similarities) @@ -197,6 +217,7 @@ class PowerIterationClustering private[clustering] ( /** * A Java-friendly version of [[PowerIterationClustering.run]]. + * @since 1.3.0 */ def run(similarities: JavaRDD[(java.lang.Long, java.lang.Long, java.lang.Double)]) : PowerIterationClusteringModel = { @@ -221,6 +242,9 @@ class PowerIterationClustering private[clustering] ( } } +/** + * @since 1.3.0 + */ @Experimental object PowerIterationClustering extends Logging { @@ -229,6 +253,7 @@ object PowerIterationClustering extends Logging { * Cluster assignment. * @param id node id * @param cluster assigned cluster id + * @since 1.3.0 */ @Experimental case class Assignment(id: Long, cluster: Int) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index d9b34cec64894..a915804b02c90 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -63,6 +63,7 @@ import org.apache.spark.util.random.XORShiftRandom * such that at time t + h the discount applied to the data from t is 0.5. * The definition remains the same whether the time unit is given * as batches or points. + * @since 1.2.0 * */ @Experimental @@ -70,7 +71,10 @@ class StreamingKMeansModel( override val clusterCenters: Array[Vector], val clusterWeights: Array[Double]) extends KMeansModel(clusterCenters) with Logging { - /** Perform a k-means update on a batch of data. */ + /** + * Perform a k-means update on a batch of data. + * @since 1.2.0 + */ def update(data: RDD[Vector], decayFactor: Double, timeUnit: String): StreamingKMeansModel = { // find nearest cluster to each point @@ -82,6 +86,7 @@ class StreamingKMeansModel( (p1._1, p1._2 + p2._2) } val dim = clusterCenters(0).size + val pointStats: Array[(Int, (Vector, Long))] = closest .aggregateByKey((Vectors.zeros(dim), 0L))(mergeContribs, mergeContribs) .collect() @@ -161,6 +166,7 @@ class StreamingKMeansModel( * .setRandomCenters(5, 100.0) * .trainOn(DStream) * }}} + * @since 1.2.0 */ @Experimental class StreamingKMeans( @@ -168,23 +174,33 @@ class StreamingKMeans( var decayFactor: Double, var timeUnit: String) extends Logging with Serializable { + /** @since 1.2.0 */ def this() = this(2, 1.0, StreamingKMeans.BATCHES) protected var model: StreamingKMeansModel = new StreamingKMeansModel(null, null) - /** Set the number of clusters. */ + /** + * Set the number of clusters. + * @since 1.2.0 + */ def setK(k: Int): this.type = { this.k = k this } - /** Set the decay factor directly (for forgetful algorithms). */ + /** + * Set the decay factor directly (for forgetful algorithms). + * @since 1.2.0 + */ def setDecayFactor(a: Double): this.type = { this.decayFactor = a this } - /** Set the half life and time unit ("batches" or "points") for forgetful algorithms. */ + /** + * Set the half life and time unit ("batches" or "points") for forgetful algorithms. + * @since 1.2.0 + */ def setHalfLife(halfLife: Double, timeUnit: String): this.type = { if (timeUnit != StreamingKMeans.BATCHES && timeUnit != StreamingKMeans.POINTS) { throw new IllegalArgumentException("Invalid time unit for decay: " + timeUnit) @@ -195,7 +211,10 @@ class StreamingKMeans( this } - /** Specify initial centers directly. */ + /** + * Specify initial centers directly. + * @since 1.2.0 + */ def setInitialCenters(centers: Array[Vector], weights: Array[Double]): this.type = { model = new StreamingKMeansModel(centers, weights) this @@ -207,6 +226,7 @@ class StreamingKMeans( * @param dim Number of dimensions * @param weight Weight for each center * @param seed Random seed + * @since 1.2.0 */ def setRandomCenters(dim: Int, weight: Double, seed: Long = Utils.random.nextLong): this.type = { val random = new XORShiftRandom(seed) @@ -216,7 +236,10 @@ class StreamingKMeans( this } - /** Return the latest model. */ + /** + * Return the latest model. + * @since 1.2.0 + */ def latestModel(): StreamingKMeansModel = { model } @@ -228,6 +251,7 @@ class StreamingKMeans( * and updates the model using each batch of data from the stream. * * @param data DStream containing vector data + * @since 1.2.0 */ def trainOn(data: DStream[Vector]) { assertInitialized() @@ -236,7 +260,10 @@ class StreamingKMeans( } } - /** Java-friendly version of `trainOn`. */ + /** + * Java-friendly version of `trainOn`. + * @since 1.4.0 + */ def trainOn(data: JavaDStream[Vector]): Unit = trainOn(data.dstream) /** @@ -244,13 +271,17 @@ class StreamingKMeans( * * @param data DStream containing vector data * @return DStream containing predictions + * @since 1.2.0 */ def predictOn(data: DStream[Vector]): DStream[Int] = { assertInitialized() data.map(model.predict) } - /** Java-friendly version of `predictOn`. */ + /** + * Java-friendly version of `predictOn`. + * @since 1.4.0 + */ def predictOn(data: JavaDStream[Vector]): JavaDStream[java.lang.Integer] = { JavaDStream.fromDStream(predictOn(data.dstream).asInstanceOf[DStream[java.lang.Integer]]) } @@ -261,13 +292,17 @@ class StreamingKMeans( * @param data DStream containing (key, feature vector) pairs * @tparam K key type * @return DStream containing the input keys and the predictions as values + * @since 1.2.0 */ def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Int)] = { assertInitialized() data.mapValues(model.predict) } - /** Java-friendly version of `predictOnValues`. */ + /** + * Java-friendly version of `predictOnValues`. + * @since 1.4.0 + */ def predictOnValues[K]( data: JavaPairDStream[K, Vector]): JavaPairDStream[K, java.lang.Integer] = { implicit val tag = fakeClassTag[K] From f3391ff2b8b9c1f1308755dc223776692e3b7725 Mon Sep 17 00:00:00 2001 From: Joshi Date: Wed, 19 Aug 2015 21:23:02 +0100 Subject: [PATCH 011/232] [SPARK-8889] [CORE] Fix for OOM for graph creation Fix for OOM for graph creation Author: Joshi Author: Rekha Joshi Closes #7602 from rekhajoshm/SPARK-8889. --- .../spark/ui/scope/RDDOperationGraph.scala | 23 +++++------ .../org/apache/spark/ui/UISeleniumSuite.scala | 39 +++++++++++++++++++ 2 files changed, 51 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index ffea9817c0b08..81f168a447ead 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.scope import scala.collection.mutable -import scala.collection.mutable.ListBuffer +import scala.collection.mutable.{StringBuilder, ListBuffer} import org.apache.spark.Logging import org.apache.spark.scheduler.StageInfo @@ -167,7 +167,7 @@ private[ui] object RDDOperationGraph extends Logging { def makeDotFile(graph: RDDOperationGraph): String = { val dotFile = new StringBuilder dotFile.append("digraph G {\n") - dotFile.append(makeDotSubgraph(graph.rootCluster, indent = " ")) + makeDotSubgraph(dotFile, graph.rootCluster, indent = " ") graph.edges.foreach { edge => dotFile.append(s""" ${edge.fromId}->${edge.toId};\n""") } dotFile.append("}") val result = dotFile.toString() @@ -180,18 +180,19 @@ private[ui] object RDDOperationGraph extends Logging { s"""${node.id} [label="${node.name} [${node.id}]"]""" } - /** Return the dot representation of a subgraph in an RDDOperationGraph. */ - private def makeDotSubgraph(cluster: RDDOperationCluster, indent: String): String = { - val subgraph = new StringBuilder - subgraph.append(indent + s"subgraph cluster${cluster.id} {\n") - subgraph.append(indent + s""" label="${cluster.name}";\n""") + /** Update the dot representation of the RDDOperationGraph in cluster to subgraph. */ + private def makeDotSubgraph( + subgraph: StringBuilder, + cluster: RDDOperationCluster, + indent: String): Unit = { + subgraph.append(indent).append(s"subgraph cluster${cluster.id} {\n") + subgraph.append(indent).append(s""" label="${cluster.name}";\n""") cluster.childNodes.foreach { node => - subgraph.append(indent + s" ${makeDotNode(node)};\n") + subgraph.append(indent).append(s" ${makeDotNode(node)};\n") } cluster.childClusters.foreach { cscope => - subgraph.append(makeDotSubgraph(cscope, indent + " ")) + makeDotSubgraph(subgraph, cscope, indent + " ") } - subgraph.append(indent + "}\n") - subgraph.toString() + subgraph.append(indent).append("}\n") } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 3aa672f8b713c..69888b2694bae 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui import java.net.{HttpURLConnection, URL} import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import scala.io.Source import scala.collection.JavaConversions._ import scala.xml.Node @@ -603,6 +604,44 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } } + test("job stages should have expected dotfile under DAG visualization") { + withSpark(newSparkContext()) { sc => + // Create a multi-stage job + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + rdd.count() + + val stage0 = Source.fromURL(sc.ui.get.appUIAddress + + "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString + assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + + "label="Stage 0";\n subgraph ")) + assert(stage0.contains("{\n label="parallelize";\n " + + "0 [label="ParallelCollectionRDD [0]"];\n }")) + assert(stage0.contains("{\n label="map";\n " + + "1 [label="MapPartitionsRDD [1]"];\n }")) + assert(stage0.contains("{\n label="groupBy";\n " + + "2 [label="MapPartitionsRDD [2]"];\n }")) + + val stage1 = Source.fromURL(sc.ui.get.appUIAddress + + "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString + assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + + "label="Stage 1";\n subgraph ")) + assert(stage1.contains("{\n label="groupBy";\n " + + "3 [label="ShuffledRDD [3]"];\n }")) + assert(stage1.contains("{\n label="map";\n " + + "4 [label="MapPartitionsRDD [4]"];\n }")) + assert(stage1.contains("{\n label="groupBy";\n " + + "5 [label="MapPartitionsRDD [5]"];\n }")) + + val stage2 = Source.fromURL(sc.ui.get.appUIAddress + + "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString + assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + + "label="Stage 2";\n subgraph ")) + assert(stage2.contains("{\n label="groupBy";\n " + + "6 [label="ShuffledRDD [6]"];\n }")) + } + } + def goToUi(sc: SparkContext, path: String): Unit = { goToUi(sc.ui.get, path) } From e05da5cb5ea253e6372f648fc8203204f2a8df8d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 19 Aug 2015 13:43:04 -0700 Subject: [PATCH 012/232] [SPARK-10107] [SQL] fix NPE in format_number Author: Davies Liu Closes #8305 from davies/format_number. --- .../spark/sql/catalyst/expressions/stringOperations.scala | 2 +- .../scala/org/apache/spark/sql/StringFunctionsSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 134f1aa2af9a8..ca044d3e95e38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -1306,8 +1306,8 @@ case class FormatNumber(x: Expression, d: Expression) $df $dFormat = new $df($pattern.toString()); $lastDValue = $d; $numberFormat.applyPattern($dFormat.toPattern()); - ${ev.primitive} = UTF8String.fromString($numberFormat.format(${typeHelper(num)})); } + ${ev.primitive} = UTF8String.fromString($numberFormat.format(${typeHelper(num)})); } else { ${ev.primitive} = null; ${ev.isNull} = true; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index cc95eede005d7..b91438baea06f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -348,9 +348,9 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { // it will still use the interpretProjection if projection follows by a LocalRelation, // hence we add a filter operator. // See the optimizer rule `ConvertToLocalRelation` - val df2 = Seq((5L, 4), (4L, 3), (3L, 2)).toDF("a", "b") + val df2 = Seq((5L, 4), (4L, 3), (4L, 3), (4L, 3), (3L, 2)).toDF("a", "b") checkAnswer( df2.filter("b>0").selectExpr("format_number(a, b)"), - Row("5.0000") :: Row("4.000") :: Row("3.00") :: Nil) + Row("5.0000") :: Row("4.000") :: Row("4.000") :: Row("4.000") :: Row("3.00") :: Nil) } } From 08887369c890e0dd87eb8b34e8c32bb03307bf24 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 19 Aug 2015 13:56:40 -0700 Subject: [PATCH 013/232] [SPARK-10073] [SQL] Python withColumn should replace the old column DataFrame.withColumn in Python should be consistent with the Scala one (replacing the existing column that has the same name). cc marmbrus Author: Davies Liu Closes #8300 from davies/with_column. --- python/pyspark/sql/dataframe.py | 12 ++++++------ python/pyspark/sql/tests.py | 4 ++++ .../main/scala/org/apache/spark/sql/DataFrame.scala | 3 ++- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index da742d7ce7d13..025811f519293 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1202,7 +1202,9 @@ def freqItems(self, cols, support=None): @ignore_unicode_prefix @since(1.3) def withColumn(self, colName, col): - """Returns a new :class:`DataFrame` by adding a column. + """ + Returns a new :class:`DataFrame` by adding a column or replacing the + existing column that has the same name. :param colName: string, name of the new column. :param col: a :class:`Column` expression for the new column. @@ -1210,7 +1212,8 @@ def withColumn(self, colName, col): >>> df.withColumn('age2', df.age + 2).collect() [Row(age=2, name=u'Alice', age2=4), Row(age=5, name=u'Bob', age2=7)] """ - return self.select('*', col.alias(colName)) + assert isinstance(col, Column), "col should be Column" + return DataFrame(self._jdf.withColumn(colName, col._jc), self.sql_ctx) @ignore_unicode_prefix @since(1.3) @@ -1223,10 +1226,7 @@ def withColumnRenamed(self, existing, new): >>> df.withColumnRenamed('age', 'age2').collect() [Row(age2=2, name=u'Alice'), Row(age2=5, name=u'Bob')] """ - cols = [Column(_to_java_column(c)).alias(new) - if c == existing else c - for c in self.columns] - return self.select(*cols) + return DataFrame(self._jdf.withColumnRenamed(existing, new), self.sql_ctx) @since(1.4) @ignore_unicode_prefix diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 13cf647b66da8..aacfb34c77618 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1035,6 +1035,10 @@ def test_capture_illegalargument_exception(self): self.assertRaisesRegexp(IllegalArgumentException, "1024 is not in the permitted values", lambda: df.select(sha2(df.a, 1024)).collect()) + def test_with_column_with_existing_name(self): + keys = self.df.withColumn("key", self.df.key).select("key").collect() + self.assertEqual([r.key for r in keys], list(range(100))) + class HiveContextSQLTests(ReusedPySparkTestCase): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index fd0ead4401193..d6688b24ae7d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1133,7 +1133,8 @@ class DataFrame private[sql]( ///////////////////////////////////////////////////////////////////////////// /** - * Returns a new [[DataFrame]] by adding a column. + * Returns a new [[DataFrame]] by adding a column or replacing the existing column that has + * the same name. * @group dfops * @since 1.3.0 */ From 21bdbe9fe69be47be562de24216a469e5ee64c7b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 19 Aug 2015 13:57:52 -0700 Subject: [PATCH 014/232] [SPARK-9627] [SQL] Stops using Scala runtime reflection in DictionaryEncoding `DictionaryEncoding` uses Scala runtime reflection to avoid boxing costs while building the directory array. However, this code path may hit [SI-6240] [1] and throw exception. [1]: https://issues.scala-lang.org/browse/SI-6240 Author: Cheng Lian Closes #8306 from liancheng/spark-9627/in-memory-cache-scala-reflection. --- .../sql/columnar/InMemoryColumnarTableScan.scala | 1 - .../columnar/compression/compressionSchemes.scala | 15 ++++----------- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 45f15fd04d4e2..66d429bc06198 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -120,7 +120,6 @@ private[sql] case class InMemoryRelation( new Iterator[CachedBatch] { def next(): CachedBatch = { val columnBuilders = output.map { attribute => - val columnType = ColumnType(attribute.dataType) ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index c91d960a0932b..ca910a99db082 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -270,20 +270,13 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { class Decoder[T <: AtomicType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { - private val dictionary = { - // TODO Can we clean up this mess? Maybe move this to `DataType`? - implicit val classTag = { - val mirror = runtimeMirror(Utils.getSparkClassLoader) - ClassTag[T#InternalType](mirror.runtimeClass(columnType.scalaTag.tpe)) - } - - Array.fill(buffer.getInt()) { - columnType.extract(buffer) - } + private val dictionary: Array[Any] = { + val elementNum = buffer.getInt() + Array.fill[Any](elementNum)(columnType.extract(buffer).asInstanceOf[Any]) } override def next(row: MutableRow, ordinal: Int): Unit = { - columnType.setField(row, ordinal, dictionary(buffer.getShort())) + columnType.setField(row, ordinal, dictionary(buffer.getShort()).asInstanceOf[T#InternalType]) } override def hasNext: Boolean = buffer.hasRemaining From 1f4c4fe6dfd8cc52b5fddfd67a31a77edbb1a036 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 19 Aug 2015 14:03:47 -0700 Subject: [PATCH 015/232] [SPARK-10090] [SQL] fix decimal scale of division We should rounding the result of multiply/division of decimal to expected precision/scale, also check overflow. Author: Davies Liu Closes #8287 from davies/decimal_division. --- .../catalyst/analysis/HiveTypeCoercion.scala | 28 +++++---- .../spark/sql/catalyst/expressions/Cast.scala | 32 +++++----- .../expressions/decimalFunctions.scala | 38 ++++++++++- .../org/apache/spark/sql/types/Decimal.scala | 4 +- .../expressions/DecimalExpressionSuite.scala | 63 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 23 ++++++- 6 files changed, 157 insertions(+), 31 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 8581d6b496c15..62c27ee0b9ee0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -371,8 +371,8 @@ object HiveTypeCoercion { DecimalType.bounded(range + scale, scale) } - private def changePrecision(e: Expression, dataType: DataType): Expression = { - ChangeDecimalPrecision(Cast(e, dataType)) + private def promotePrecision(e: Expression, dataType: DataType): Expression = { + PromotePrecision(Cast(e, dataType)) } def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { @@ -383,36 +383,42 @@ object HiveTypeCoercion { case e if !e.childrenResolved => e // Skip nodes who is already promoted - case e: BinaryArithmetic if e.left.isInstanceOf[ChangeDecimalPrecision] => e + case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - Add(changePrecision(e1, dt), changePrecision(e2, dt)) + CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - Subtract(changePrecision(e1, dt), changePrecision(e2, dt)) + CheckOverflow(Subtract(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(p1 + p2 + 1, s1 + s2) - Multiply(changePrecision(e1, dt), changePrecision(e2, dt)) + val resultType = DecimalType.bounded(p1 + p2 + 1, s1 + s2) + val widerType = widerDecimalType(p1, s1, p2, s2) + CheckOverflow(Multiply(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), + resultType) case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) - Divide(changePrecision(e1, dt), changePrecision(e2, dt)) + val resultType = DecimalType.bounded(p1 - s1 + s2 + max(6, s1 + p2 + 1), + max(6, s1 + p2 + 1)) + val widerType = widerDecimalType(p1, s1, p2, s2) + CheckOverflow(Divide(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), + resultType) case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = DecimalType.bounded(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) // resultType may have lower precision, so we cast them into wider type first. val widerType = widerDecimalType(p1, s1, p2, s2) - Cast(Remainder(changePrecision(e1, widerType), changePrecision(e2, widerType)), + CheckOverflow(Remainder(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), resultType) case Pmod(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = DecimalType.bounded(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) // resultType may have lower precision, so we cast them into wider type first. val widerType = widerDecimalType(p1, s1, p2, s2) - Cast(Pmod(changePrecision(e1, widerType), changePrecision(e2, widerType)), resultType) + CheckOverflow(Pmod(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), + resultType) case b @ BinaryComparison(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 616b9e0e65b78..2db954257be35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -447,7 +447,7 @@ case class Cast(child: Expression, dataType: DataType) case StringType => castToStringCode(from, ctx) case BinaryType => castToBinaryCode(from) case DateType => castToDateCode(from, ctx) - case decimal: DecimalType => castToDecimalCode(from, decimal) + case decimal: DecimalType => castToDecimalCode(from, decimal, ctx) case TimestampType => castToTimestampCode(from, ctx) case CalendarIntervalType => castToIntervalCode(from) case BooleanType => castToBooleanCode(from) @@ -528,14 +528,18 @@ case class Cast(child: Expression, dataType: DataType) } """ - private[this] def castToDecimalCode(from: DataType, target: DecimalType): CastFunction = { + private[this] def castToDecimalCode( + from: DataType, + target: DecimalType, + ctx: CodeGenContext): CastFunction = { + val tmp = ctx.freshName("tmpDecimal") from match { case StringType => (c, evPrim, evNull) => s""" try { - Decimal tmpDecimal = Decimal.apply(new java.math.BigDecimal($c.toString())); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + Decimal $tmp = Decimal.apply(new java.math.BigDecimal($c.toString())); + ${changePrecision(tmp, target, evPrim, evNull)} } catch (java.lang.NumberFormatException e) { $evNull = true; } @@ -543,8 +547,8 @@ case class Cast(child: Expression, dataType: DataType) case BooleanType => (c, evPrim, evNull) => s""" - Decimal tmpDecimal = $c ? Decimal.apply(1) : Decimal.apply(0); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + Decimal $tmp = $c ? Decimal.apply(1) : Decimal.apply(0); + ${changePrecision(tmp, target, evPrim, evNull)} """ case DateType => // date can't cast to decimal in Hive @@ -553,29 +557,29 @@ case class Cast(child: Expression, dataType: DataType) // Note that we lose precision here. (c, evPrim, evNull) => s""" - Decimal tmpDecimal = Decimal.apply( + Decimal $tmp = Decimal.apply( scala.math.BigDecimal.valueOf(${timestampToDoubleCode(c)})); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull)} """ case DecimalType() => (c, evPrim, evNull) => s""" - Decimal tmpDecimal = $c.clone(); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + Decimal $tmp = $c.clone(); + ${changePrecision(tmp, target, evPrim, evNull)} """ case x: IntegralType => (c, evPrim, evNull) => s""" - Decimal tmpDecimal = Decimal.apply((long) $c); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + Decimal $tmp = Decimal.apply((long) $c); + ${changePrecision(tmp, target, evPrim, evNull)} """ case x: FractionalType => // All other numeric types can be represented precisely as Doubles (c, evPrim, evNull) => s""" try { - Decimal tmpDecimal = Decimal.apply(scala.math.BigDecimal.valueOf((double) $c)); - ${changePrecision("tmpDecimal", target, evPrim, evNull)} + Decimal $tmp = Decimal.apply(scala.math.BigDecimal.valueOf((double) $c)); + ${changePrecision(tmp, target, evPrim, evNull)} } catch (java.lang.NumberFormatException e) { $evNull = true; } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index adb33e4c8d4a1..b7be12f7aa741 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -66,10 +66,44 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un * An expression used to wrap the children when promote the precision of DecimalType to avoid * promote multiple times. */ -case class ChangeDecimalPrecision(child: Expression) extends UnaryExpression { +case class PromotePrecision(child: Expression) extends UnaryExpression { override def dataType: DataType = child.dataType override def eval(input: InternalRow): Any = child.eval(input) override def gen(ctx: CodeGenContext): GeneratedExpressionCode = child.gen(ctx) override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = "" - override def prettyName: String = "change_decimal_precision" + override def prettyName: String = "promote_precision" +} + +/** + * Rounds the decimal to given scale and check whether the decimal can fit in provided precision + * or not, returns null if not. + */ +case class CheckOverflow(child: Expression, dataType: DecimalType) extends UnaryExpression { + + override def nullable: Boolean = true + + override def nullSafeEval(input: Any): Any = { + val d = input.asInstanceOf[Decimal].clone() + if (d.changePrecision(dataType.precision, dataType.scale)) { + d + } else { + null + } + } + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + nullSafeCodeGen(ctx, ev, eval => { + val tmp = ctx.freshName("tmp") + s""" + | Decimal $tmp = $eval.clone(); + | if ($tmp.changePrecision(${dataType.precision}, ${dataType.scale})) { + | ${ev.primitive} = $tmp; + | } else { + | ${ev.isNull} = true; + | } + """.stripMargin + }) + } + + override def toString: String = s"CheckOverflow($child, $dataType)" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index d95805c24521c..c988f1d1b972e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -267,7 +267,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal + that.longVal, Math.max(precision, that.precision), scale) } else { - Decimal(toBigDecimal + that.toBigDecimal, precision, scale) + Decimal(toBigDecimal + that.toBigDecimal) } } @@ -275,7 +275,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { Decimal(longVal - that.longVal, Math.max(precision, that.precision), scale) } else { - Decimal(toBigDecimal - that.toBigDecimal, precision, scale) + Decimal(toBigDecimal - that.toBigDecimal) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala new file mode 100644 index 0000000000000..511f0307901df --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.{LongType, DecimalType, Decimal} + + +class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { + + test("UnscaledValue") { + val d1 = Decimal("10.1") + checkEvaluation(UnscaledValue(Literal(d1)), 101L) + val d2 = Decimal(101, 3, 1) + checkEvaluation(UnscaledValue(Literal(d2)), 101L) + checkEvaluation(UnscaledValue(Literal.create(null, DecimalType(2, 1))), null) + } + + test("MakeDecimal") { + checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) + checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) + } + + test("PromotePrecision") { + val d1 = Decimal("10.1") + checkEvaluation(PromotePrecision(Literal(d1)), d1) + val d2 = Decimal(101, 3, 1) + checkEvaluation(PromotePrecision(Literal(d2)), d2) + checkEvaluation(PromotePrecision(Literal.create(null, DecimalType(2, 1))), null) + } + + test("CheckOverflow") { + val d1 = Decimal("10.1") + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 0)), Decimal("10")) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 1)), d1) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 2)), d1) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 3)), null) + + val d2 = Decimal(101, 3, 1) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 0)), Decimal("10")) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 1)), d2) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 2)), d2) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 3)), null) + + checkEvaluation(CheckOverflow(Literal.create(null, DecimalType(2, 1)), DecimalType(3, 2)), null) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index c329fdb2a6bb1..141468ca00d67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,16 +17,17 @@ package org.apache.spark.sql +import java.math.MathContext import java.sql.Timestamp import org.apache.spark.AccumulatorSuite -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.DefaultParserDialect +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.errors.DialectException import org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ +import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ /** A SQL Dialect for testing purpose, and it can not be nested type */ @@ -1608,6 +1609,24 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("decimal precision with multiply/division") { + checkAnswer(sql("select 10.3 * 3.0"), Row(BigDecimal("30.90"))) + checkAnswer(sql("select 10.3000 * 3.0"), Row(BigDecimal("30.90000"))) + checkAnswer(sql("select 10.30000 * 30.0"), Row(BigDecimal("309.000000"))) + checkAnswer(sql("select 10.300000000000000000 * 3.000000000000000000"), + Row(BigDecimal("30.900000000000000000000000000000000000", new MathContext(38)))) + checkAnswer(sql("select 10.300000000000000000 * 3.0000000000000000000"), + Row(null)) + + checkAnswer(sql("select 10.3 / 3.0"), Row(BigDecimal("3.433333"))) + checkAnswer(sql("select 10.3000 / 3.0"), Row(BigDecimal("3.4333333"))) + checkAnswer(sql("select 10.30000 / 30.0"), Row(BigDecimal("0.343333333"))) + checkAnswer(sql("select 10.300000000000000000 / 3.00000000000000000"), + Row(BigDecimal("3.4333333333333333333333333333333333333", new MathContext(38)))) + checkAnswer(sql("select 10.3000000000000000000 / 3.00000000000000000"), + Row(null)) + } + test("external sorting updates peak execution memory") { withSQLConf((SQLConf.EXTERNAL_SORT.key, "true")) { val sc = sqlContext.sparkContext From f3ff4c41d2e32bd0f2419d1c9c68fcd0c2593e41 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 19 Aug 2015 14:15:28 -0700 Subject: [PATCH 016/232] [SPARK-9899] [SQL] Disables customized output committer when speculation is on Speculation hates direct output committer, as there are multiple corner cases that may cause data corruption and/or data loss. Please see this [PR comment] [1] for more details. [1]: https://github.com/apache/spark/pull/8191#issuecomment-131598385 Author: Cheng Lian Closes #8317 from liancheng/spark-9899/speculation-hates-direct-output-committer. --- .../datasources/WriterContainer.scala | 16 ++++++++- .../sql/sources/hadoopFsRelationSuites.scala | 34 +++++++++++++++++++ 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index e0147079e6997..78f48a5cd72c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -58,6 +58,9 @@ private[sql] abstract class BaseWriterContainer( // This is only used on driver side. @transient private val jobContext: JobContext = job + private val speculationEnabled: Boolean = + relation.sqlContext.sparkContext.conf.getBoolean("spark.speculation", defaultValue = false) + // The following fields are initialized and used on both driver and executor side. @transient protected var outputCommitter: OutputCommitter = _ @transient private var jobId: JobID = _ @@ -126,10 +129,21 @@ private[sql] abstract class BaseWriterContainer( // associated with the file output format since it is not safe to use a custom // committer for appending. For example, in S3, direct parquet output committer may // leave partial data in the destination dir when the the appending job fails. + // + // See SPARK-8578 for more details logInfo( - s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName} " + + s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + "for appending.") defaultOutputCommitter + } else if (speculationEnabled) { + // When speculation is enabled, it's not safe to use customized output committer classes, + // especially direct output committers (e.g. `DirectParquetOutputCommitter`). + // + // See SPARK-9899 for more details. + logInfo( + s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + + "because spark.speculation is configured to be true.") + defaultOutputCommitter } else { val committerClass = context.getConfiguration.getClass( SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 8d0d9218ddd6a..5bbca14bad320 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -570,6 +570,40 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils { df.write.format(dataSourceName).partitionBy("c", "d", "e").saveAsTable("t") } } + + test("SPARK-9899 Disable customized output committer when speculation is on") { + val clonedConf = new Configuration(configuration) + val speculationEnabled = + sqlContext.sparkContext.conf.getBoolean("spark.speculation", defaultValue = false) + + try { + withTempPath { dir => + // Enables task speculation + sqlContext.sparkContext.conf.set("spark.speculation", "true") + + // Uses a customized output committer which always fails + configuration.set( + SQLConf.OUTPUT_COMMITTER_CLASS.key, + classOf[AlwaysFailOutputCommitter].getName) + + // Code below shouldn't throw since customized output committer should be disabled. + val df = sqlContext.range(10).coalesce(1) + df.write.format(dataSourceName).save(dir.getCanonicalPath) + checkAnswer( + sqlContext + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .load(dir.getCanonicalPath), + df) + } + } finally { + // Hadoop 1 doesn't have `Configuration.unset` + configuration.clear() + clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + sqlContext.sparkContext.conf.set("spark.speculation", speculationEnabled.toString) + } + } } // This class is used to test SPARK-8578. We should not use any custom output committer when From 373a376c04320aab228b5c385e2b788809877d3e Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 19 Aug 2015 14:31:51 -0700 Subject: [PATCH 017/232] [SPARK-10083] [SQL] CaseWhen should support type coercion of DecimalType and FractionalType create t1 (a decimal(7, 2), b long); select case when 1=1 then a else 1.0 end from t1; select case when 1=1 then a else b end from t1; Author: Daoyuan Wang Closes #8270 from adrian-wang/casewhenfractional. --- .../sql/catalyst/analysis/HiveTypeCoercion.scala | 4 ++-- .../sql/catalyst/analysis/HiveTypeCoercionSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 62c27ee0b9ee0..f2f2ba2f96552 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -605,7 +605,7 @@ object HiveTypeCoercion { def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { case c: CaseWhenLike if c.childrenResolved && !c.valueTypesEqual => logDebug(s"Input values for null casting ${c.valueTypes.mkString(",")}") - val maybeCommonType = findTightestCommonTypeAndPromoteToString(c.valueTypes) + val maybeCommonType = findWiderCommonType(c.valueTypes) maybeCommonType.map { commonType => val castedBranches = c.branches.grouped(2).map { case Seq(when, value) if value.dataType != commonType => @@ -622,7 +622,7 @@ object HiveTypeCoercion { case c: CaseKeyWhen if c.childrenResolved && !c.resolved => val maybeCommonType = - findTightestCommonTypeAndPromoteToString((c.key +: c.whenList).map(_.dataType)) + findWiderCommonType((c.key +: c.whenList).map(_.dataType)) maybeCommonType.map { commonType => val castedBranches = c.branches.grouped(2).map { case Seq(whenExpr, thenExpr) if whenExpr.dataType != commonType => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index cbdf453f600ab..6f33ab733b615 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -285,6 +285,17 @@ class HiveTypeCoercionSuite extends PlanTest { CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) + ruleTest(HiveTypeCoercion.CaseWhenCoercion, + CaseWhen(Seq(Literal(true), Literal(1.2), Literal.create(1, DecimalType(7, 2)))), + CaseWhen(Seq( + Literal(true), Literal(1.2), Cast(Literal.create(1, DecimalType(7, 2)), DoubleType))) + ) + ruleTest(HiveTypeCoercion.CaseWhenCoercion, + CaseWhen(Seq(Literal(true), Literal(100L), Literal.create(1, DecimalType(7, 2)))), + CaseWhen(Seq( + Literal(true), Cast(Literal(100L), DecimalType(22, 2)), + Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2)))) + ) } test("type coercion simplification for equal to") { From e0dd1309ac248375f429639801923570f14de18d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 19 Aug 2015 14:33:32 -0700 Subject: [PATCH 018/232] [SPARK-10119] [CORE] Fix isDynamicAllocationEnabled when config is expliticly disabled. Author: Marcelo Vanzin Closes #8316 from vanzin/SPARK-10119. --- .../main/scala/org/apache/spark/util/Utils.scala | 2 +- .../scala/org/apache/spark/util/UtilsSuite.scala | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index fddc24dbfc237..8313312226713 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2141,7 +2141,7 @@ private[spark] object Utils extends Logging { * the latter should override the former (SPARK-9092). */ def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { - conf.contains("spark.dynamicAllocation.enabled") && + conf.getBoolean("spark.dynamicAllocation.enabled", false) && conf.getInt("spark.executor.instances", 0) == 0 } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 8f7e402d5f2a6..1fb81ad565b41 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -720,4 +720,18 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.nanSafeCompareFloats(Float.PositiveInfinity, Float.NaN) === -1) assert(Utils.nanSafeCompareFloats(Float.NegativeInfinity, Float.NaN) === -1) } + + test("isDynamicAllocationEnabled") { + val conf = new SparkConf() + assert(Utils.isDynamicAllocationEnabled(conf) === false) + assert(Utils.isDynamicAllocationEnabled( + conf.set("spark.dynamicAllocation.enabled", "false")) === false) + assert(Utils.isDynamicAllocationEnabled( + conf.set("spark.dynamicAllocation.enabled", "true")) === true) + assert(Utils.isDynamicAllocationEnabled( + conf.set("spark.executor.instances", "1")) === false) + assert(Utils.isDynamicAllocationEnabled( + conf.set("spark.executor.instances", "0")) === true) + } + } From b0dbaec4f942a47afde3490b9339ad3bd187024d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 19 Aug 2015 15:04:56 -0700 Subject: [PATCH 019/232] [SPARK-6489] [SQL] add column pruning for Generate This PR takes over https://github.com/apache/spark/pull/5358 Author: Wenchen Fan Closes #8268 from cloud-fan/6489. --- .../sql/catalyst/expressions/generators.scala | 2 - .../sql/catalyst/optimizer/Optimizer.scala | 16 ++++ .../optimizer/ColumnPruningSuite.scala | 84 +++++++++++++++++++ 3 files changed, 100 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index d474853355e5b..c0845e1a0102f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.Map - import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 47b06cae15436..42457d5318b48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -165,6 +165,7 @@ object SetOperationPushDown extends Rule[LogicalPlan] { * * - Inserting Projections beneath the following operators: * - Aggregate + * - Generate * - Project <- Join * - LeftSemiJoin */ @@ -178,6 +179,21 @@ object ColumnPruning extends Rule[LogicalPlan] { case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => a.copy(child = Project(a.references.toSeq, child)) + // Eliminate attributes that are not needed to calculate the Generate. + case g: Generate if !g.join && (g.child.outputSet -- g.references).nonEmpty => + g.copy(child = Project(g.references.toSeq, g.child)) + + case p @ Project(_, g: Generate) if g.join && p.references.subsetOf(g.generatedSet) => + p.copy(child = g.copy(join = false)) + + case p @ Project(projectList, g: Generate) if g.join => + val neededChildOutput = p.references -- g.generatorOutput ++ g.references + if (neededChildOutput == g.child.outputSet) { + p + } else { + Project(projectList, g.copy(child = Project(neededChildOutput.toSeq, g.child))) + } + case p @ Project(projectList, a @ Aggregate(groupingExpressions, aggregateExpressions, child)) if (a.outputSet -- p.references).nonEmpty => Project( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala new file mode 100644 index 0000000000000..dbebcb86809de --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -0,0 +1,84 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions.Explode +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation, Generate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.types.StringType + +class ColumnPruningSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("Column pruning", FixedPoint(100), + ColumnPruning) :: Nil + } + + test("Column pruning for Generate when Generate.join = false") { + val input = LocalRelation('a.int, 'b.array(StringType)) + + val query = Generate(Explode('b), false, false, None, 's.string :: Nil, input).analyze + val optimized = Optimize.execute(query) + + val correctAnswer = + Generate(Explode('b), false, false, None, 's.string :: Nil, + Project('b.attr :: Nil, input)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Column pruning for Generate when Generate.join = true") { + val input = LocalRelation('a.int, 'b.int, 'c.array(StringType)) + + val query = + Project(Seq('a, 's), + Generate(Explode('c), true, false, None, 's.string :: Nil, + input)).analyze + val optimized = Optimize.execute(query) + + val correctAnswer = + Project(Seq('a, 's), + Generate(Explode('c), true, false, None, 's.string :: Nil, + Project(Seq('a, 'c), + input))).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Turn Generate.join to false if possible") { + val input = LocalRelation('b.array(StringType)) + + val query = + Project(('s + 1).as("s+1") :: Nil, + Generate(Explode('b), true, false, None, 's.string :: Nil, + input)).analyze + val optimized = Optimize.execute(query) + + val correctAnswer = + Project(('s + 1).as("s+1") :: Nil, + Generate(Explode('b), false, false, None, 's.string :: Nil, + input)).analyze + + comparePlans(optimized, correctAnswer) + } + + // todo: add more tests for column pruning +} From 8e0a072f78b4902d5f7ccc6b15232ed202a117f9 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 19 Aug 2015 15:43:08 -0700 Subject: [PATCH 020/232] [SPARK-9895] User Guide for RFormula Feature Transformer mengxr Author: Eric Liang Closes #8293 from ericl/docs-2. --- docs/ml-features.md | 108 ++++++++++++++++++ .../apache/spark/ml/feature/RFormula.scala | 4 +- 2 files changed, 110 insertions(+), 2 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index d0e8eeb7a757e..6309db97be4d0 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1477,3 +1477,111 @@ print(output.select("features", "clicked").first())

+## RFormula + +`RFormula` selects columns specified by an [R model formula](https://stat.ethz.ch/R-manual/R-devel/library/stats/html/formula.html). It produces a vector column of features and a double column of labels. Like when formulas are used in R for linear regression, string input columns will be one-hot encoded, and numeric columns will be cast to doubles. If not already present in the DataFrame, the output label column will be created from the specified response variable in the formula. + +**Examples** + +Assume that we have a DataFrame with the columns `id`, `country`, `hour`, and `clicked`: + +~~~ +id | country | hour | clicked +---|---------|------|--------- + 7 | "US" | 18 | 1.0 + 8 | "CA" | 12 | 0.0 + 9 | "NZ" | 15 | 0.0 +~~~ + +If we use `RFormula` with a formula string of `clicked ~ country + hour`, which indicates that we want to +predict `clicked` based on `country` and `hour`, after transformation we should get the following DataFrame: + +~~~ +id | country | hour | clicked | features | label +---|---------|------|---------|------------------|------- + 7 | "US" | 18 | 1.0 | [0.0, 0.0, 18.0] | 1.0 + 8 | "CA" | 12 | 0.0 | [0.0, 1.0, 12.0] | 0.0 + 9 | "NZ" | 15 | 0.0 | [1.0, 0.0, 15.0] | 0.0 +~~~ + +
+
+ +[`RFormula`](api/scala/index.html#org.apache.spark.ml.feature.RFormula) takes an R formula string, and optional parameters for the names of its output columns. + +{% highlight scala %} +import org.apache.spark.ml.feature.RFormula + +val dataset = sqlContext.createDataFrame(Seq( + (7, "US", 18, 1.0), + (8, "CA", 12, 0.0), + (9, "NZ", 15, 0.0) +)).toDF("id", "country", "hour", "clicked") +val formula = new RFormula() + .setFormula("clicked ~ country + hour") + .setFeaturesCol("features") + .setLabelCol("label") +val output = formula.fit(dataset).transform(dataset) +output.select("features", "label").show() +{% endhighlight %} +
+ +
+ +[`RFormula`](api/java/org/apache/spark/ml/feature/RFormula.html) takes an R formula string, and optional parameters for the names of its output columns. + +{% highlight java %} +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.RFormula; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.types.DataTypes.*; + +StructType schema = createStructType(new StructField[] { + createStructField("id", IntegerType, false), + createStructField("country", StringType, false), + createStructField("hour", IntegerType, false), + createStructField("clicked", DoubleType, false) +}); +JavaRDD rdd = jsc.parallelize(Arrays.asList( + RowFactory.create(7, "US", 18, 1.0), + RowFactory.create(8, "CA", 12, 0.0), + RowFactory.create(9, "NZ", 15, 0.0) +)); +DataFrame dataset = sqlContext.createDataFrame(rdd, schema); + +RFormula formula = new RFormula() + .setFormula("clicked ~ country + hour") + .setFeaturesCol("features") + .setLabelCol("label"); + +DataFrame output = formula.fit(dataset).transform(dataset); +output.select("features", "label").show(); +{% endhighlight %} +
+ +
+ +[`RFormula`](api/python/pyspark.ml.html#pyspark.ml.feature.RFormula) takes an R formula string, and optional parameters for the names of its output columns. + +{% highlight python %} +from pyspark.ml.feature import RFormula + +dataset = sqlContext.createDataFrame( + [(7, "US", 18, 1.0), + (8, "CA", 12, 0.0), + (9, "NZ", 15, 0.0)], + ["id", "country", "hour", "clicked"]) +formula = RFormula( + formula="clicked ~ country + hour", + featuresCol="features", + labelCol="label") +output = formula.fit(dataset).transform(dataset) +output.select("features", "label").show() +{% endhighlight %} +
+
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index a752dacd72d95..a7fa50444209b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -42,8 +42,8 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol { /** * :: Experimental :: * Implements the transforms required for fitting a dataset against an R model formula. Currently - * we support a limited subset of the R operators, including '~' and '+'. Also see the R formula - * docs here: http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html + * we support a limited subset of the R operators, including '.', '~', '+', and '-'. Also see the + * R formula docs here: http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html */ @Experimental class RFormula(override val uid: String) extends Estimator[RFormulaModel] with RFormulaBase { From ba5f7e1842f2c5852b5309910c0d39926643da69 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 20 Aug 2015 08:13:25 +0800 Subject: [PATCH 021/232] [SPARK-10035] [SQL] Parquet filters does not process EqualNullSafe filter. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit As I talked with Lian, 1. I added EquelNullSafe to ParquetFilters - It uses the same equality comparison filter with EqualTo since the Parquet filter performs actually null-safe equality comparison. 2. Updated the test code (ParquetFilterSuite) - Convert catalyst.Expression to sources.Filter - Removed Cast since only Literal is picked up as a proper Filter in DataSourceStrategy - Added EquelNullSafe comparison 3. Removed deprecated createFilter for catalyst.Expression Author: hyukjinkwon Author: 권혁진 Closes #8275 from HyukjinKwon/master. --- .../datasources/parquet/ParquetFilters.scala | 113 +++--------------- .../parquet/ParquetFilterSuite.scala | 63 ++++------ 2 files changed, 37 insertions(+), 139 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 63915e0a28655..c74c8388632f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -22,8 +22,6 @@ import java.nio.ByteBuffer import com.google.common.io.BaseEncoding import org.apache.hadoop.conf.Configuration -import org.apache.parquet.filter2.compat.FilterCompat -import org.apache.parquet.filter2.compat.FilterCompat._ import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.filter2.predicate._ import org.apache.parquet.io.api.Binary @@ -39,12 +37,6 @@ import org.apache.spark.unsafe.types.UTF8String private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" - def createRecordFilter(filterExpressions: Seq[Expression]): Option[Filter] = { - filterExpressions.flatMap { filter => - createFilter(filter) - }.reduceOption(FilterApi.and).map(FilterCompat.get) - } - case class SetInFilter[T <: Comparable[T]]( valueSet: Set[T]) extends UserDefinedPredicate[T] with Serializable { @@ -205,6 +197,16 @@ private[sql] object ParquetFilters { // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, // which can be casted to `false` implicitly. Please refer to the `eval` method of these // operators and the `SimplifyFilters` rule for details. + + // Hyukjin: + // I added [[EqualNullSafe]] with [[org.apache.parquet.filter2.predicate.Operators.Eq]]. + // So, it performs equality comparison identically when given [[sources.Filter]] is [[EqualTo]]. + // The reason why I did this is, that the actual Parquet filter checks null-safe equality + // comparison. + // So I added this and maybe [[EqualTo]] should be changed. It still seems fine though, because + // physical planning does not set `NULL` to [[EqualTo]] but changes it to [[IsNull]] and etc. + // Probably I missed something and obviously this should be changed. + predicate match { case sources.IsNull(name) => makeEq.lift(dataTypeOf(name)).map(_(name, null)) @@ -216,6 +218,11 @@ private[sql] object ParquetFilters { case sources.Not(sources.EqualTo(name, value)) => makeNotEq.lift(dataTypeOf(name)).map(_(name, value)) + case sources.EqualNullSafe(name, value) => + makeEq.lift(dataTypeOf(name)).map(_(name, value)) + case sources.Not(sources.EqualNullSafe(name, value)) => + makeNotEq.lift(dataTypeOf(name)).map(_(name, value)) + case sources.LessThan(name, value) => makeLt.lift(dataTypeOf(name)).map(_(name, value)) case sources.LessThanOrEqual(name, value) => @@ -273,96 +280,6 @@ private[sql] object ParquetFilters { addMethod.invoke(null, classOf[Binary], enumTypeDescriptor) } - /** - * Converts Catalyst predicate expressions to Parquet filter predicates. - * - * @todo This can be removed once we get rid of the old Parquet support. - */ - def createFilter(predicate: Expression): Option[FilterPredicate] = { - // NOTE: - // - // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, - // which can be casted to `false` implicitly. Please refer to the `eval` method of these - // operators and the `SimplifyFilters` rule for details. - predicate match { - case IsNull(NamedExpression(name, dataType)) => - makeEq.lift(dataType).map(_(name, null)) - case IsNotNull(NamedExpression(name, dataType)) => - makeNotEq.lift(dataType).map(_(name, null)) - - case EqualTo(NamedExpression(name, _), NonNullLiteral(value, dataType)) => - makeEq.lift(dataType).map(_(name, value)) - case EqualTo(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _)) => - makeEq.lift(dataType).map(_(name, value)) - case EqualTo(NonNullLiteral(value, dataType), NamedExpression(name, _)) => - makeEq.lift(dataType).map(_(name, value)) - case EqualTo(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType)) => - makeEq.lift(dataType).map(_(name, value)) - - case Not(EqualTo(NamedExpression(name, _), NonNullLiteral(value, dataType))) => - makeNotEq.lift(dataType).map(_(name, value)) - case Not(EqualTo(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _))) => - makeNotEq.lift(dataType).map(_(name, value)) - case Not(EqualTo(NonNullLiteral(value, dataType), NamedExpression(name, _))) => - makeNotEq.lift(dataType).map(_(name, value)) - case Not(EqualTo(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType))) => - makeNotEq.lift(dataType).map(_(name, value)) - - case LessThan(NamedExpression(name, _), NonNullLiteral(value, dataType)) => - makeLt.lift(dataType).map(_(name, value)) - case LessThan(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _)) => - makeLt.lift(dataType).map(_(name, value)) - case LessThan(NonNullLiteral(value, dataType), NamedExpression(name, _)) => - makeGt.lift(dataType).map(_(name, value)) - case LessThan(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType)) => - makeGt.lift(dataType).map(_(name, value)) - - case LessThanOrEqual(NamedExpression(name, _), NonNullLiteral(value, dataType)) => - makeLtEq.lift(dataType).map(_(name, value)) - case LessThanOrEqual(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _)) => - makeLtEq.lift(dataType).map(_(name, value)) - case LessThanOrEqual(NonNullLiteral(value, dataType), NamedExpression(name, _)) => - makeGtEq.lift(dataType).map(_(name, value)) - case LessThanOrEqual(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType)) => - makeGtEq.lift(dataType).map(_(name, value)) - - case GreaterThan(NamedExpression(name, _), NonNullLiteral(value, dataType)) => - makeGt.lift(dataType).map(_(name, value)) - case GreaterThan(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _)) => - makeGt.lift(dataType).map(_(name, value)) - case GreaterThan(NonNullLiteral(value, dataType), NamedExpression(name, _)) => - makeLt.lift(dataType).map(_(name, value)) - case GreaterThan(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType)) => - makeLt.lift(dataType).map(_(name, value)) - - case GreaterThanOrEqual(NamedExpression(name, _), NonNullLiteral(value, dataType)) => - makeGtEq.lift(dataType).map(_(name, value)) - case GreaterThanOrEqual(Cast(NamedExpression(name, _), dataType), NonNullLiteral(value, _)) => - makeGtEq.lift(dataType).map(_(name, value)) - case GreaterThanOrEqual(NonNullLiteral(value, dataType), NamedExpression(name, _)) => - makeLtEq.lift(dataType).map(_(name, value)) - case GreaterThanOrEqual(NonNullLiteral(value, _), Cast(NamedExpression(name, _), dataType)) => - makeLtEq.lift(dataType).map(_(name, value)) - - case And(lhs, rhs) => - (createFilter(lhs) ++ createFilter(rhs)).reduceOption(FilterApi.and) - - case Or(lhs, rhs) => - for { - lhsFilter <- createFilter(lhs) - rhsFilter <- createFilter(rhs) - } yield FilterApi.or(lhsFilter, rhsFilter) - - case Not(pred) => - createFilter(pred).map(FilterApi.not) - - case InSet(NamedExpression(name, dataType), valueSet) => - makeInSet.lift(dataType).map(_(name, valueSet)) - - case _ => None - } - } - /** * Note: Inside the Hadoop API we only have access to `Configuration`, not to * [[org.apache.spark.SparkContext]], so we cannot use broadcasts to convey diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 5b4e568bb9838..f067112cfca95 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -24,9 +24,8 @@ import org.apache.spark.sql.{Column, DataFrame, QueryTest, Row, SQLConf} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, LogicalRelation} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types._ /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. @@ -55,20 +54,22 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + val analyzedPredicate = query.queryExecution.optimizedPlan.collect { case PhysicalOperation(_, filters, LogicalRelation(_: ParquetRelation)) => filters - }.flatten.reduceOption(_ && _) + }.flatten + assert(analyzedPredicate.nonEmpty) - assert(maybeAnalyzedPredicate.isDefined) - maybeAnalyzedPredicate.foreach { pred => - val maybeFilter = ParquetFilters.createFilter(pred) + val selectedFilters = DataSourceStrategy.selectFilters(analyzedPredicate) + assert(selectedFilters.nonEmpty) + + selectedFilters.foreach { pred => + val maybeFilter = ParquetFilters.createFilter(df.schema, pred) assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") maybeFilter.foreach { f => // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) assert(f.getClass === filterClass) } } - checker(query, expected) } } @@ -109,43 +110,18 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false))) checkFilterPredicate('_1 === true, classOf[Eq[_]], true) + checkFilterPredicate('_1 <=> true, classOf[Eq[_]], true) checkFilterPredicate('_1 !== true, classOf[NotEq[_]], false) } } - test("filter pushdown - short") { - withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit df => - checkFilterPredicate(Cast('_1, IntegerType) === 1, classOf[Eq[_]], 1) - checkFilterPredicate( - Cast('_1, IntegerType) !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) - - checkFilterPredicate(Cast('_1, IntegerType) < 2, classOf[Lt[_]], 1) - checkFilterPredicate(Cast('_1, IntegerType) > 3, classOf[Gt[_]], 4) - checkFilterPredicate(Cast('_1, IntegerType) <= 1, classOf[LtEq[_]], 1) - checkFilterPredicate(Cast('_1, IntegerType) >= 4, classOf[GtEq[_]], 4) - - checkFilterPredicate(Literal(1) === Cast('_1, IntegerType), classOf[Eq[_]], 1) - checkFilterPredicate(Literal(2) > Cast('_1, IntegerType), classOf[Lt[_]], 1) - checkFilterPredicate(Literal(3) < Cast('_1, IntegerType), classOf[Gt[_]], 4) - checkFilterPredicate(Literal(1) >= Cast('_1, IntegerType), classOf[LtEq[_]], 1) - checkFilterPredicate(Literal(4) <= Cast('_1, IntegerType), classOf[GtEq[_]], 4) - - checkFilterPredicate(!(Cast('_1, IntegerType) < 4), classOf[GtEq[_]], 4) - checkFilterPredicate( - Cast('_1, IntegerType) > 2 && Cast('_1, IntegerType) < 4, classOf[Operators.And], 3) - checkFilterPredicate( - Cast('_1, IntegerType) < 2 || Cast('_1, IntegerType) > 3, - classOf[Operators.Or], - Seq(Row(1), Row(4))) - } - } - test("filter pushdown - integer") { withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) @@ -154,13 +130,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } } @@ -171,6 +147,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) @@ -179,13 +156,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } } @@ -196,6 +173,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) @@ -204,13 +182,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } } @@ -221,6 +199,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_))) checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1) + checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1) checkFilterPredicate('_1 !== 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_))) checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1) @@ -229,13 +208,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4) checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1) + checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1) checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1) checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4) checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1) checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4) checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4) - checkFilterPredicate('_1 > 2 && '_1 < 4, classOf[Operators.And], 3) checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4))) } } @@ -247,6 +226,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.toString))) checkFilterPredicate('_1 === "1", classOf[Eq[_]], "1") + checkFilterPredicate('_1 <=> "1", classOf[Eq[_]], "1") checkFilterPredicate( '_1 !== "1", classOf[NotEq[_]], (2 to 4).map(i => Row.apply(i.toString))) @@ -256,13 +236,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate('_1 >= "4", classOf[GtEq[_]], "4") checkFilterPredicate(Literal("1") === '_1, classOf[Eq[_]], "1") + checkFilterPredicate(Literal("1") <=> '_1, classOf[Eq[_]], "1") checkFilterPredicate(Literal("2") > '_1, classOf[Lt[_]], "1") checkFilterPredicate(Literal("3") < '_1, classOf[Gt[_]], "4") checkFilterPredicate(Literal("1") >= '_1, classOf[LtEq[_]], "1") checkFilterPredicate(Literal("4") <= '_1, classOf[GtEq[_]], "4") checkFilterPredicate(!('_1 < "4"), classOf[GtEq[_]], "4") - checkFilterPredicate('_1 > "2" && '_1 < "4", classOf[Operators.And], "3") checkFilterPredicate('_1 < "2" || '_1 > "3", classOf[Operators.Or], Seq(Row("1"), Row("4"))) } } @@ -274,6 +254,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex withParquetDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) + checkBinaryFilterPredicate('_1 <=> 1.b, classOf[Eq[_]], 1.b) checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) checkBinaryFilterPredicate( @@ -288,13 +269,13 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkBinaryFilterPredicate('_1 >= 4.b, classOf[GtEq[_]], 4.b) checkBinaryFilterPredicate(Literal(1.b) === '_1, classOf[Eq[_]], 1.b) + checkBinaryFilterPredicate(Literal(1.b) <=> '_1, classOf[Eq[_]], 1.b) checkBinaryFilterPredicate(Literal(2.b) > '_1, classOf[Lt[_]], 1.b) checkBinaryFilterPredicate(Literal(3.b) < '_1, classOf[Gt[_]], 4.b) checkBinaryFilterPredicate(Literal(1.b) >= '_1, classOf[LtEq[_]], 1.b) checkBinaryFilterPredicate(Literal(4.b) <= '_1, classOf[GtEq[_]], 4.b) checkBinaryFilterPredicate(!('_1 < 4.b), classOf[GtEq[_]], 4.b) - checkBinaryFilterPredicate('_1 > 2.b && '_1 < 4.b, classOf[Operators.And], 3.b) checkBinaryFilterPredicate( '_1 < 2.b || '_1 > 3.b, classOf[Operators.Or], Seq(Row(1.b), Row(4.b))) } From 2f2686a73f5a2a53ca5b1023e0d7e0e6c9be5896 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 19 Aug 2015 17:35:41 -0700 Subject: [PATCH 022/232] [SPARK-9242] [SQL] Audit UDAF interface. A few minor changes: 1. Improved documentation 2. Rename apply(distinct....) to distinct. 3. Changed MutableAggregationBuffer from a trait to an abstract class. 4. Renamed returnDataType to dataType to be more consistent with other expressions. And unrelated to UDAFs: 1. Renamed file names in expressions to use suffix "Expressions" to be more consistent. 2. Moved regexp related expressions out to its own file. 3. Renamed StringComparison => StringPredicate. Author: Reynold Xin Closes #8321 from rxin/SPARK-9242. --- ...bitwise.scala => bitwiseExpressions.scala} | 0 ...als.scala => conditionalExpressions.scala} | 0 ...ctions.scala => datetimeExpressions.scala} | 0 ...nctions.scala => decimalExpressions.scala} | 0 ...nFunctions.scala => jsonExpressions.scala} | 0 .../{math.scala => mathExpressions.scala} | 0 ...lFunctions.scala => nullExpressions.scala} | 0 .../{random.scala => randomExpressions.scala} | 0 .../expressions/regexpExpressions.scala | 346 ++++++++++++++++++ ...erations.scala => stringExpressions.scala} | 332 +---------------- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../expressions/StringExpressionsSuite.scala | 2 +- .../apache/spark/sql/UDFRegistration.scala | 1 + .../spark/sql/execution/aggregate/udaf.scala | 2 +- .../apache/spark/sql/expressions/udaf.scala | 44 ++- .../spark/sql/hive/JavaDataFrameSuite.java | 2 +- .../spark/sql/hive/aggregate/MyDoubleAvg.java | 2 +- .../spark/sql/hive/aggregate/MyDoubleSum.java | 2 +- 18 files changed, 386 insertions(+), 349 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{bitwise.scala => bitwiseExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{conditionals.scala => conditionalExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{datetimeFunctions.scala => datetimeExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{decimalFunctions.scala => decimalExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{jsonFunctions.scala => jsonExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{math.scala => mathExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{nullFunctions.scala => nullExpressions.scala} (100%) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{random.scala => randomExpressions.scala} (100%) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{stringOperations.scala => stringExpressions.scala} (74%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala new file mode 100644 index 0000000000000..6dff28a7cde46 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -0,0 +1,346 @@ +/* + * 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.catalyst.expressions + +import java.util.regex.{MatchResult, Pattern} + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + + +trait StringRegexExpression extends ImplicitCastInputTypes { + self: BinaryExpression => + + def escape(v: String): String + def matches(regex: Pattern, str: String): Boolean + + override def dataType: DataType = BooleanType + override def inputTypes: Seq[DataType] = Seq(StringType, StringType) + + // try cache the pattern for Literal + private lazy val cache: Pattern = right match { + case x @ Literal(value: String, StringType) => compile(value) + case _ => null + } + + protected def compile(str: String): Pattern = if (str == null) { + null + } else { + // Let it raise exception if couldn't compile the regex string + Pattern.compile(escape(str)) + } + + protected def pattern(str: String) = if (cache == null) compile(str) else cache + + protected override def nullSafeEval(input1: Any, input2: Any): Any = { + val regex = pattern(input2.asInstanceOf[UTF8String].toString) + if(regex == null) { + null + } else { + matches(regex, input1.asInstanceOf[UTF8String].toString) + } + } +} + + +/** + * Simple RegEx pattern matching function + */ +case class Like(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression with CodegenFallback { + + override def escape(v: String): String = StringUtils.escapeLikeRegex(v) + + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() + + override def toString: String = s"$left LIKE $right" + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val patternClass = classOf[Pattern].getName + val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" + val pattern = ctx.freshName("pattern") + + if (right.foldable) { + val rVal = right.eval() + if (rVal != null) { + val regexStr = + StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString())) + ctx.addMutableState(patternClass, pattern, + s"""$pattern = ${patternClass}.compile("$regexStr");""") + + // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. + val eval = left.gen(ctx) + s""" + ${eval.code} + boolean ${ev.isNull} = ${eval.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${ev.primitive} = $pattern.matcher(${eval.primitive}.toString()).matches(); + } + """ + } else { + s""" + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + """ + } + } else { + nullSafeCodeGen(ctx, ev, (eval1, eval2) => { + s""" + String rightStr = ${eval2}.toString(); + ${patternClass} $pattern = ${patternClass}.compile($escapeFunc(rightStr)); + ${ev.primitive} = $pattern.matcher(${eval1}.toString()).matches(); + """ + }) + } + } +} + + +case class RLike(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression with CodegenFallback { + + override def escape(v: String): String = v + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) + override def toString: String = s"$left RLIKE $right" + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val patternClass = classOf[Pattern].getName + val pattern = ctx.freshName("pattern") + + if (right.foldable) { + val rVal = right.eval() + if (rVal != null) { + val regexStr = + StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString()) + ctx.addMutableState(patternClass, pattern, + s"""$pattern = ${patternClass}.compile("$regexStr");""") + + // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. + val eval = left.gen(ctx) + s""" + ${eval.code} + boolean ${ev.isNull} = ${eval.isNull}; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + if (!${ev.isNull}) { + ${ev.primitive} = $pattern.matcher(${eval.primitive}.toString()).find(0); + } + """ + } else { + s""" + boolean ${ev.isNull} = true; + ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; + """ + } + } else { + nullSafeCodeGen(ctx, ev, (eval1, eval2) => { + s""" + String rightStr = ${eval2}.toString(); + ${patternClass} $pattern = ${patternClass}.compile(rightStr); + ${ev.primitive} = $pattern.matcher(${eval1}.toString()).find(0); + """ + }) + } + } +} + + +/** + * Splits str around pat (pattern is a regular expression). + */ +case class StringSplit(str: Expression, pattern: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + + override def left: Expression = str + override def right: Expression = pattern + override def dataType: DataType = ArrayType(StringType) + override def inputTypes: Seq[DataType] = Seq(StringType, StringType) + + override def nullSafeEval(string: Any, regex: Any): Any = { + val strings = string.asInstanceOf[UTF8String].split(regex.asInstanceOf[UTF8String], -1) + new GenericArrayData(strings.asInstanceOf[Array[Any]]) + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val arrayClass = classOf[GenericArrayData].getName + nullSafeCodeGen(ctx, ev, (str, pattern) => + // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. + s"""${ev.primitive} = new $arrayClass($str.split($pattern, -1));""") + } + + override def prettyName: String = "split" +} + + +/** + * Replace all substrings of str that match regexp with rep. + * + * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. + */ +case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expression) + extends TernaryExpression with ImplicitCastInputTypes { + + // last regex in string, we will update the pattern iff regexp value changed. + @transient private var lastRegex: UTF8String = _ + // last regex pattern, we cache it for performance concern + @transient private var pattern: Pattern = _ + // last replacement string, we don't want to convert a UTF8String => java.langString every time. + @transient private var lastReplacement: String = _ + @transient private var lastReplacementInUTF8: UTF8String = _ + // result buffer write by Matcher + @transient private val result: StringBuffer = new StringBuffer + + override def nullSafeEval(s: Any, p: Any, r: Any): Any = { + if (!p.equals(lastRegex)) { + // regex value changed + lastRegex = p.asInstanceOf[UTF8String].clone() + pattern = Pattern.compile(lastRegex.toString) + } + if (!r.equals(lastReplacementInUTF8)) { + // replacement string changed + lastReplacementInUTF8 = r.asInstanceOf[UTF8String].clone() + lastReplacement = lastReplacementInUTF8.toString + } + val m = pattern.matcher(s.toString()) + result.delete(0, result.length()) + + while (m.find) { + m.appendReplacement(result, lastReplacement) + } + m.appendTail(result) + + UTF8String.fromString(result.toString) + } + + override def dataType: DataType = StringType + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, StringType) + override def children: Seq[Expression] = subject :: regexp :: rep :: Nil + override def prettyName: String = "regexp_replace" + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val termLastRegex = ctx.freshName("lastRegex") + val termPattern = ctx.freshName("pattern") + + val termLastReplacement = ctx.freshName("lastReplacement") + val termLastReplacementInUTF8 = ctx.freshName("lastReplacementInUTF8") + + val termResult = ctx.freshName("result") + + val classNamePattern = classOf[Pattern].getCanonicalName + val classNameStringBuffer = classOf[java.lang.StringBuffer].getCanonicalName + + ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;") + ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;") + ctx.addMutableState("String", termLastReplacement, s"${termLastReplacement} = null;") + ctx.addMutableState("UTF8String", + termLastReplacementInUTF8, s"${termLastReplacementInUTF8} = null;") + ctx.addMutableState(classNameStringBuffer, + termResult, s"${termResult} = new $classNameStringBuffer();") + + nullSafeCodeGen(ctx, ev, (subject, regexp, rep) => { + s""" + if (!$regexp.equals(${termLastRegex})) { + // regex value changed + ${termLastRegex} = $regexp.clone(); + ${termPattern} = ${classNamePattern}.compile(${termLastRegex}.toString()); + } + if (!$rep.equals(${termLastReplacementInUTF8})) { + // replacement string changed + ${termLastReplacementInUTF8} = $rep.clone(); + ${termLastReplacement} = ${termLastReplacementInUTF8}.toString(); + } + ${termResult}.delete(0, ${termResult}.length()); + java.util.regex.Matcher m = ${termPattern}.matcher($subject.toString()); + + while (m.find()) { + m.appendReplacement(${termResult}, ${termLastReplacement}); + } + m.appendTail(${termResult}); + ${ev.primitive} = UTF8String.fromString(${termResult}.toString()); + ${ev.isNull} = false; + """ + }) + } +} + +/** + * Extract a specific(idx) group identified by a Java regex. + * + * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. + */ +case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) + extends TernaryExpression with ImplicitCastInputTypes { + def this(s: Expression, r: Expression) = this(s, r, Literal(1)) + + // last regex in string, we will update the pattern iff regexp value changed. + @transient private var lastRegex: UTF8String = _ + // last regex pattern, we cache it for performance concern + @transient private var pattern: Pattern = _ + + override def nullSafeEval(s: Any, p: Any, r: Any): Any = { + if (!p.equals(lastRegex)) { + // regex value changed + lastRegex = p.asInstanceOf[UTF8String].clone() + pattern = Pattern.compile(lastRegex.toString) + } + val m = pattern.matcher(s.toString) + if (m.find) { + val mr: MatchResult = m.toMatchResult + UTF8String.fromString(mr.group(r.asInstanceOf[Int])) + } else { + UTF8String.EMPTY_UTF8 + } + } + + override def dataType: DataType = StringType + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) + override def children: Seq[Expression] = subject :: regexp :: idx :: Nil + override def prettyName: String = "regexp_extract" + + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val termLastRegex = ctx.freshName("lastRegex") + val termPattern = ctx.freshName("pattern") + val classNamePattern = classOf[Pattern].getCanonicalName + + ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;") + ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;") + + nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { + s""" + if (!$regexp.equals(${termLastRegex})) { + // regex value changed + ${termLastRegex} = $regexp.clone(); + ${termPattern} = ${classNamePattern}.compile(${termLastRegex}.toString()); + } + java.util.regex.Matcher m = + ${termPattern}.matcher($subject.toString()); + if (m.find()) { + java.util.regex.MatchResult mr = m.toMatchResult(); + ${ev.primitive} = UTF8String.fromString(mr.group($idx)); + ${ev.isNull} = false; + } else { + ${ev.primitive} = UTF8String.EMPTY_UTF8; + ${ev.isNull} = false; + }""" + }) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala similarity index 74% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index ca044d3e95e38..3c23f2ecfb57c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -21,13 +21,9 @@ import java.text.DecimalFormat import java.util.Arrays import java.util.{Map => JMap, HashMap} import java.util.Locale -import java.util.regex.{MatchResult, Pattern} - -import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -124,143 +120,6 @@ case class ConcatWs(children: Seq[Expression]) } } - -trait StringRegexExpression extends ImplicitCastInputTypes { - self: BinaryExpression => - - def escape(v: String): String - def matches(regex: Pattern, str: String): Boolean - - override def dataType: DataType = BooleanType - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) - - // try cache the pattern for Literal - private lazy val cache: Pattern = right match { - case x @ Literal(value: String, StringType) => compile(value) - case _ => null - } - - protected def compile(str: String): Pattern = if (str == null) { - null - } else { - // Let it raise exception if couldn't compile the regex string - Pattern.compile(escape(str)) - } - - protected def pattern(str: String) = if (cache == null) compile(str) else cache - - protected override def nullSafeEval(input1: Any, input2: Any): Any = { - val regex = pattern(input2.asInstanceOf[UTF8String].toString()) - if(regex == null) { - null - } else { - matches(regex, input1.asInstanceOf[UTF8String].toString()) - } - } -} - -/** - * Simple RegEx pattern matching function - */ -case class Like(left: Expression, right: Expression) - extends BinaryExpression with StringRegexExpression with CodegenFallback { - - override def escape(v: String): String = StringUtils.escapeLikeRegex(v) - - override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() - - override def toString: String = s"$left LIKE $right" - - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val patternClass = classOf[Pattern].getName - val escapeFunc = StringUtils.getClass.getName.stripSuffix("$") + ".escapeLikeRegex" - val pattern = ctx.freshName("pattern") - - if (right.foldable) { - val rVal = right.eval() - if (rVal != null) { - val regexStr = - StringEscapeUtils.escapeJava(escape(rVal.asInstanceOf[UTF8String].toString())) - ctx.addMutableState(patternClass, pattern, - s"""$pattern = ${patternClass}.compile("$regexStr");""") - - // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) - s""" - ${eval.code} - boolean ${ev.isNull} = ${eval.isNull}; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.primitive} = $pattern.matcher(${eval.primitive}.toString()).matches(); - } - """ - } else { - s""" - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - """ - } - } else { - nullSafeCodeGen(ctx, ev, (eval1, eval2) => { - s""" - String rightStr = ${eval2}.toString(); - ${patternClass} $pattern = ${patternClass}.compile($escapeFunc(rightStr)); - ${ev.primitive} = $pattern.matcher(${eval1}.toString()).matches(); - """ - }) - } - } -} - - -case class RLike(left: Expression, right: Expression) - extends BinaryExpression with StringRegexExpression with CodegenFallback { - - override def escape(v: String): String = v - override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) - override def toString: String = s"$left RLIKE $right" - - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val patternClass = classOf[Pattern].getName - val pattern = ctx.freshName("pattern") - - if (right.foldable) { - val rVal = right.eval() - if (rVal != null) { - val regexStr = - StringEscapeUtils.escapeJava(rVal.asInstanceOf[UTF8String].toString()) - ctx.addMutableState(patternClass, pattern, - s"""$pattern = ${patternClass}.compile("$regexStr");""") - - // We don't use nullSafeCodeGen here because we don't want to re-evaluate right again. - val eval = left.gen(ctx) - s""" - ${eval.code} - boolean ${ev.isNull} = ${eval.isNull}; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - if (!${ev.isNull}) { - ${ev.primitive} = $pattern.matcher(${eval.primitive}.toString()).find(0); - } - """ - } else { - s""" - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)}; - """ - } - } else { - nullSafeCodeGen(ctx, ev, (eval1, eval2) => { - s""" - String rightStr = ${eval2}.toString(); - ${patternClass} $pattern = ${patternClass}.compile(rightStr); - ${ev.primitive} = $pattern.matcher(${eval1}.toString()).find(0); - """ - }) - } - } -} - - trait String2StringExpression extends ImplicitCastInputTypes { self: UnaryExpression => @@ -305,7 +164,7 @@ case class Lower(child: Expression) extends UnaryExpression with String2StringEx } /** A base trait for functions that compare two strings, returning a boolean. */ -trait StringComparison extends ImplicitCastInputTypes { +trait StringPredicate extends Predicate with ImplicitCastInputTypes { self: BinaryExpression => def compare(l: UTF8String, r: UTF8String): Boolean @@ -322,7 +181,7 @@ trait StringComparison extends ImplicitCastInputTypes { * A function that returns true if the string `left` contains the string `right`. */ case class Contains(left: Expression, right: Expression) - extends BinaryExpression with Predicate with StringComparison { + extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).contains($c2)") @@ -333,7 +192,7 @@ case class Contains(left: Expression, right: Expression) * A function that returns true if the string `left` starts with the string `right`. */ case class StartsWith(left: Expression, right: Expression) - extends BinaryExpression with Predicate with StringComparison { + extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).startsWith($c2)") @@ -344,7 +203,7 @@ case class StartsWith(left: Expression, right: Expression) * A function that returns true if the string `left` ends with the string `right`. */ case class EndsWith(left: Expression, right: Expression) - extends BinaryExpression with Predicate with StringComparison { + extends BinaryExpression with StringPredicate { override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { defineCodeGen(ctx, ev, (c1, c2) => s"($c1).endsWith($c2)") @@ -769,32 +628,6 @@ case class StringSpace(child: Expression) override def prettyName: String = "space" } -/** - * Splits str around pat (pattern is a regular expression). - */ -case class StringSplit(str: Expression, pattern: Expression) - extends BinaryExpression with ImplicitCastInputTypes { - - override def left: Expression = str - override def right: Expression = pattern - override def dataType: DataType = ArrayType(StringType) - override def inputTypes: Seq[DataType] = Seq(StringType, StringType) - - override def nullSafeEval(string: Any, regex: Any): Any = { - val strings = string.asInstanceOf[UTF8String].split(regex.asInstanceOf[UTF8String], -1) - new GenericArrayData(strings.asInstanceOf[Array[Any]]) - } - - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val arrayClass = classOf[GenericArrayData].getName - nullSafeCodeGen(ctx, ev, (str, pattern) => - // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. - s"""${ev.primitive} = new $arrayClass($str.split($pattern, -1));""") - } - - override def prettyName: String = "split" -} - object Substring { def subStringBinarySQL(bytes: Array[Byte], pos: Int, len: Int): Array[Byte] = { if (pos > bytes.length) { @@ -1048,163 +881,6 @@ case class Encode(value: Expression, charset: Expression) } } -/** - * Replace all substrings of str that match regexp with rep. - * - * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. - */ -case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expression) - extends TernaryExpression with ImplicitCastInputTypes { - - // last regex in string, we will update the pattern iff regexp value changed. - @transient private var lastRegex: UTF8String = _ - // last regex pattern, we cache it for performance concern - @transient private var pattern: Pattern = _ - // last replacement string, we don't want to convert a UTF8String => java.langString every time. - @transient private var lastReplacement: String = _ - @transient private var lastReplacementInUTF8: UTF8String = _ - // result buffer write by Matcher - @transient private val result: StringBuffer = new StringBuffer - - override def nullSafeEval(s: Any, p: Any, r: Any): Any = { - if (!p.equals(lastRegex)) { - // regex value changed - lastRegex = p.asInstanceOf[UTF8String].clone() - pattern = Pattern.compile(lastRegex.toString) - } - if (!r.equals(lastReplacementInUTF8)) { - // replacement string changed - lastReplacementInUTF8 = r.asInstanceOf[UTF8String].clone() - lastReplacement = lastReplacementInUTF8.toString - } - val m = pattern.matcher(s.toString()) - result.delete(0, result.length()) - - while (m.find) { - m.appendReplacement(result, lastReplacement) - } - m.appendTail(result) - - UTF8String.fromString(result.toString) - } - - override def dataType: DataType = StringType - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, StringType) - override def children: Seq[Expression] = subject :: regexp :: rep :: Nil - override def prettyName: String = "regexp_replace" - - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val termLastRegex = ctx.freshName("lastRegex") - val termPattern = ctx.freshName("pattern") - - val termLastReplacement = ctx.freshName("lastReplacement") - val termLastReplacementInUTF8 = ctx.freshName("lastReplacementInUTF8") - - val termResult = ctx.freshName("result") - - val classNamePattern = classOf[Pattern].getCanonicalName - val classNameStringBuffer = classOf[java.lang.StringBuffer].getCanonicalName - - ctx.addMutableState("UTF8String", - termLastRegex, s"${termLastRegex} = null;") - ctx.addMutableState(classNamePattern, - termPattern, s"${termPattern} = null;") - ctx.addMutableState("String", - termLastReplacement, s"${termLastReplacement} = null;") - ctx.addMutableState("UTF8String", - termLastReplacementInUTF8, s"${termLastReplacementInUTF8} = null;") - ctx.addMutableState(classNameStringBuffer, - termResult, s"${termResult} = new $classNameStringBuffer();") - - nullSafeCodeGen(ctx, ev, (subject, regexp, rep) => { - s""" - if (!$regexp.equals(${termLastRegex})) { - // regex value changed - ${termLastRegex} = $regexp.clone(); - ${termPattern} = ${classNamePattern}.compile(${termLastRegex}.toString()); - } - if (!$rep.equals(${termLastReplacementInUTF8})) { - // replacement string changed - ${termLastReplacementInUTF8} = $rep.clone(); - ${termLastReplacement} = ${termLastReplacementInUTF8}.toString(); - } - ${termResult}.delete(0, ${termResult}.length()); - java.util.regex.Matcher m = ${termPattern}.matcher($subject.toString()); - - while (m.find()) { - m.appendReplacement(${termResult}, ${termLastReplacement}); - } - m.appendTail(${termResult}); - ${ev.primitive} = UTF8String.fromString(${termResult}.toString()); - ${ev.isNull} = false; - """ - }) - } -} - -/** - * Extract a specific(idx) group identified by a Java regex. - * - * NOTE: this expression is not THREAD-SAFE, as it has some internal mutable status. - */ -case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) - extends TernaryExpression with ImplicitCastInputTypes { - def this(s: Expression, r: Expression) = this(s, r, Literal(1)) - - // last regex in string, we will update the pattern iff regexp value changed. - @transient private var lastRegex: UTF8String = _ - // last regex pattern, we cache it for performance concern - @transient private var pattern: Pattern = _ - - override def nullSafeEval(s: Any, p: Any, r: Any): Any = { - if (!p.equals(lastRegex)) { - // regex value changed - lastRegex = p.asInstanceOf[UTF8String].clone() - pattern = Pattern.compile(lastRegex.toString) - } - val m = pattern.matcher(s.toString()) - if (m.find) { - val mr: MatchResult = m.toMatchResult - UTF8String.fromString(mr.group(r.asInstanceOf[Int])) - } else { - UTF8String.EMPTY_UTF8 - } - } - - override def dataType: DataType = StringType - override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType, IntegerType) - override def children: Seq[Expression] = subject :: regexp :: idx :: Nil - override def prettyName: String = "regexp_extract" - - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val termLastRegex = ctx.freshName("lastRegex") - val termPattern = ctx.freshName("pattern") - val classNamePattern = classOf[Pattern].getCanonicalName - - ctx.addMutableState("UTF8String", termLastRegex, s"${termLastRegex} = null;") - ctx.addMutableState(classNamePattern, termPattern, s"${termPattern} = null;") - - nullSafeCodeGen(ctx, ev, (subject, regexp, idx) => { - s""" - if (!$regexp.equals(${termLastRegex})) { - // regex value changed - ${termLastRegex} = $regexp.clone(); - ${termPattern} = ${classNamePattern}.compile(${termLastRegex}.toString()); - } - java.util.regex.Matcher m = - ${termPattern}.matcher($subject.toString()); - if (m.find()) { - java.util.regex.MatchResult mr = m.toMatchResult(); - ${ev.primitive} = UTF8String.fromString(mr.group($idx)); - ${ev.isNull} = false; - } else { - ${ev.primitive} = UTF8String.EMPTY_UTF8; - ${ev.isNull} = false; - }""" - }) - } -} - /** * Formats the number X to a format like '#,###,###.##', rounded to D decimal places, * and returns the result as a string. If D is 0, the result has no decimal point or diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 42457d5318b48..854463dd11c74 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -372,7 +372,7 @@ object NullPropagation extends Rule[LogicalPlan] { case _ => e } - case e: StringComparison => e.children match { + case e: StringPredicate => e.children match { case Literal(null, _) :: right :: Nil => Literal.create(null, e.dataType) case left :: Literal(null, _) :: Nil => Literal.create(null, e.dataType) case _ => e diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index 426dc272471ae..99e3b13ce8c97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -673,7 +673,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Length(Literal.create(null, BinaryType)), null, create_row(bytes)) } - test("number format") { + test("format_number / FormatNumber") { checkEvaluation(FormatNumber(Literal(4.asInstanceOf[Byte]), Literal(3)), "4.000") checkEvaluation(FormatNumber(Literal(4.asInstanceOf[Short]), Literal(3)), "4.000") checkEvaluation(FormatNumber(Literal(4.0f), Literal(3)), "4.000") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 1f270560d7bc1..fc4d0938c533a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -56,6 +56,7 @@ class UDFRegistration private[sql] (sqlContext: SQLContext) extends Logging { /** * Register a user-defined aggregate function (UDAF). + * * @param name the name of the UDAF. * @param udaf the UDAF needs to be registered. * @return the registered UDAF. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala index 7619f3ec9f0a7..d43d3dd9ffaae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala @@ -304,7 +304,7 @@ private[sql] case class ScalaUDAF( override def nullable: Boolean = true - override def dataType: DataType = udaf.returnDataType + override def dataType: DataType = udaf.dataType override def deterministic: Boolean = udaf.deterministic diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 5180871585f25..258afadc76951 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.expressions -import org.apache.spark.sql.catalyst.expressions.ScalaUDF import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, AggregateExpression2} import org.apache.spark.sql.execution.aggregate.ScalaUDAF import org.apache.spark.sql.{Column, Row} @@ -26,7 +25,7 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * The abstract class for implementing user-defined aggregate functions. + * The base class for implementing user-defined aggregate functions (UDAF). */ @Experimental abstract class UserDefinedAggregateFunction extends Serializable { @@ -67,22 +66,35 @@ abstract class UserDefinedAggregateFunction extends Serializable { /** * The [[DataType]] of the returned value of this [[UserDefinedAggregateFunction]]. */ - def returnDataType: DataType + def dataType: DataType - /** Indicates if this function is deterministic. */ + /** + * Returns true iff this function is deterministic, i.e. given the same input, + * always return the same output. + */ def deterministic: Boolean /** - * Initializes the given aggregation buffer. Initial values set by this method should satisfy - * the condition that when merging two buffers with initial values, the new buffer - * still store initial values. + * Initializes the given aggregation buffer, i.e. the zero value of the aggregation buffer. + * + * The contract should be that applying the merge function on two initial buffers should just + * return the initial buffer itself, i.e. + * `merge(initialBuffer, initialBuffer)` should equal `initialBuffer`. */ def initialize(buffer: MutableAggregationBuffer): Unit - /** Updates the given aggregation buffer `buffer` with new input data from `input`. */ + /** + * Updates the given aggregation buffer `buffer` with new input data from `input`. + * + * This is called once per input row. + */ def update(buffer: MutableAggregationBuffer, input: Row): Unit - /** Merges two aggregation buffers and stores the updated buffer values back to `buffer1`. */ + /** + * Merges two aggregation buffers and stores the updated buffer values back to `buffer1`. + * + * This is called when we merge two partially aggregated data together. + */ def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit /** @@ -92,7 +104,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { def evaluate(buffer: Row): Any /** - * Creates a [[Column]] for this UDAF with given [[Column]]s as arguments. + * Creates a [[Column]] for this UDAF using given [[Column]]s as input arguments. */ @scala.annotation.varargs def apply(exprs: Column*): Column = { @@ -105,16 +117,16 @@ abstract class UserDefinedAggregateFunction extends Serializable { } /** - * Creates a [[Column]] for this UDAF with given [[Column]]s as arguments. - * If `isDistinct` is true, this UDAF is working on distinct input values. + * Creates a [[Column]] for this UDAF using the distinct values of the given + * [[Column]]s as input arguments. */ @scala.annotation.varargs - def apply(isDistinct: Boolean, exprs: Column*): Column = { + def distinct(exprs: Column*): Column = { val aggregateExpression = AggregateExpression2( ScalaUDAF(exprs.map(_.expr), this), Complete, - isDistinct = isDistinct) + isDistinct = true) Column(aggregateExpression) } } @@ -122,9 +134,11 @@ abstract class UserDefinedAggregateFunction extends Serializable { /** * :: Experimental :: * A [[Row]] representing an mutable aggregation buffer. + * + * This is not meant to be extended outside of Spark. */ @Experimental -trait MutableAggregationBuffer extends Row { +abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ def update(i: Int, value: Any): Unit diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java index 21b053f07a3ba..a30dfa554eabc 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -92,7 +92,7 @@ public void testUDAF() { DataFrame aggregatedDF = df.groupBy() .agg( - udaf.apply(true, col("value")), + udaf.distinct(col("value")), udaf.apply(col("value")), registeredUDAF.apply(col("value")), callUDF("mydoublesum", col("value"))); diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java index a2247e3da1554..2961b803f14aa 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java +++ b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java @@ -65,7 +65,7 @@ public MyDoubleAvg() { return _bufferSchema; } - @Override public DataType returnDataType() { + @Override public DataType dataType() { return _returnDataType; } diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java index da29e24d267dd..c71882a6e7bed 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java +++ b/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java @@ -60,7 +60,7 @@ public MyDoubleSum() { return _bufferSchema; } - @Override public DataType returnDataType() { + @Override public DataType dataType() { return _returnDataType; } From 1f29d502e7ecd6faa185d70dc714f9ea3922fb6d Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 19 Aug 2015 18:36:01 -0700 Subject: [PATCH 023/232] [SPARK-9812] [STREAMING] Fix Python 3 compatibility issue in PySpark Streaming and some docs This PR includes the following fixes: 1. Use `range` instead of `xrange` in `queue_stream.py` to support Python 3. 2. Fix the issue that `utf8_decoder` will return `bytes` rather than `str` when receiving an empty `bytes` in Python 3. 3. Fix the commands in docs so that the user can copy them directly to the command line. The previous commands was broken in the middle of a path, so when copying to the command line, the path would be split to two parts by the extra spaces, which forces the user to fix it manually. Author: zsxwing Closes #8315 from zsxwing/SPARK-9812. --- .../src/main/python/streaming/direct_kafka_wordcount.py | 6 +++--- examples/src/main/python/streaming/flume_wordcount.py | 5 +++-- examples/src/main/python/streaming/kafka_wordcount.py | 5 +++-- examples/src/main/python/streaming/mqtt_wordcount.py | 5 +++-- examples/src/main/python/streaming/queue_stream.py | 4 ++-- python/pyspark/streaming/flume.py | 4 +++- python/pyspark/streaming/kafka.py | 4 +++- python/pyspark/streaming/kinesis.py | 4 +++- 8 files changed, 23 insertions(+), 14 deletions(-) diff --git a/examples/src/main/python/streaming/direct_kafka_wordcount.py b/examples/src/main/python/streaming/direct_kafka_wordcount.py index 6ef188a220c51..ea20678b9acad 100644 --- a/examples/src/main/python/streaming/direct_kafka_wordcount.py +++ b/examples/src/main/python/streaming/direct_kafka_wordcount.py @@ -23,8 +23,8 @@ http://kafka.apache.org/documentation.html#quickstart and then run the example - `$ bin/spark-submit --jars external/kafka-assembly/target/scala-*/\ - spark-streaming-kafka-assembly-*.jar \ + `$ bin/spark-submit --jars \ + external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ examples/src/main/python/streaming/direct_kafka_wordcount.py \ localhost:9092 test` """ @@ -37,7 +37,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: direct_kafka_wordcount.py " + print("Usage: direct_kafka_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingDirectKafkaWordCount") diff --git a/examples/src/main/python/streaming/flume_wordcount.py b/examples/src/main/python/streaming/flume_wordcount.py index 091b64d8c4af4..d75bc6daac138 100644 --- a/examples/src/main/python/streaming/flume_wordcount.py +++ b/examples/src/main/python/streaming/flume_wordcount.py @@ -23,8 +23,9 @@ https://flume.apache.org/documentation.html and then run the example - `$ bin/spark-submit --jars external/flume-assembly/target/scala-*/\ - spark-streaming-flume-assembly-*.jar examples/src/main/python/streaming/flume_wordcount.py \ + `$ bin/spark-submit --jars \ + external/flume-assembly/target/scala-*/spark-streaming-flume-assembly-*.jar \ + examples/src/main/python/streaming/flume_wordcount.py \ localhost 12345 """ from __future__ import print_function diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index b178e7899b5e1..8d697f620f467 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -23,8 +23,9 @@ http://kafka.apache.org/documentation.html#quickstart and then run the example - `$ bin/spark-submit --jars external/kafka-assembly/target/scala-*/\ - spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ + `$ bin/spark-submit --jars \ + external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ + examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ from __future__ import print_function diff --git a/examples/src/main/python/streaming/mqtt_wordcount.py b/examples/src/main/python/streaming/mqtt_wordcount.py index 617ce5ea6775e..abf9c0e21d307 100644 --- a/examples/src/main/python/streaming/mqtt_wordcount.py +++ b/examples/src/main/python/streaming/mqtt_wordcount.py @@ -26,8 +26,9 @@ http://www.eclipse.org/paho/#getting-started and then run the example - `$ bin/spark-submit --jars external/mqtt-assembly/target/scala-*/\ - spark-streaming-mqtt-assembly-*.jar examples/src/main/python/streaming/mqtt_wordcount.py \ + `$ bin/spark-submit --jars \ + external/mqtt-assembly/target/scala-*/spark-streaming-mqtt-assembly-*.jar \ + examples/src/main/python/streaming/mqtt_wordcount.py \ tcp://localhost:1883 foo` """ diff --git a/examples/src/main/python/streaming/queue_stream.py b/examples/src/main/python/streaming/queue_stream.py index dcd6a0fc6ff91..b3808907f74a6 100644 --- a/examples/src/main/python/streaming/queue_stream.py +++ b/examples/src/main/python/streaming/queue_stream.py @@ -36,8 +36,8 @@ # Create the queue through which RDDs can be pushed to # a QueueInputDStream rddQueue = [] - for i in xrange(5): - rddQueue += [ssc.sparkContext.parallelize([j for j in xrange(1, 1001)], 10)] + for i in range(5): + rddQueue += [ssc.sparkContext.parallelize([j for j in range(1, 1001)], 10)] # Create the QueueInputDStream and use it do some processing inputStream = ssc.queueStream(rddQueue) diff --git a/python/pyspark/streaming/flume.py b/python/pyspark/streaming/flume.py index cbb573f226bbe..c0cdc50d8d423 100644 --- a/python/pyspark/streaming/flume.py +++ b/python/pyspark/streaming/flume.py @@ -31,7 +31,9 @@ def utf8_decoder(s): """ Decode the unicode as UTF-8 """ - return s and s.decode('utf-8') + if s is None: + return None + return s.decode('utf-8') class FlumeUtils(object): diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index dc5b7fd878aef..8a814c64c0423 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -29,7 +29,9 @@ def utf8_decoder(s): """ Decode the unicode as UTF-8 """ - return s and s.decode('utf-8') + if s is None: + return None + return s.decode('utf-8') class KafkaUtils(object): diff --git a/python/pyspark/streaming/kinesis.py b/python/pyspark/streaming/kinesis.py index bcfe2703fecf9..34be5880e1708 100644 --- a/python/pyspark/streaming/kinesis.py +++ b/python/pyspark/streaming/kinesis.py @@ -26,7 +26,9 @@ def utf8_decoder(s): """ Decode the unicode as UTF-8 """ - return s and s.decode('utf-8') + if s is None: + return None + return s.decode('utf-8') class KinesisUtils(object): From affc8a887ede9fdc2ca6051833954cd10918c869 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 19 Aug 2015 19:43:09 -0700 Subject: [PATCH 024/232] [SPARK-10125] [STREAMING] Fix a potential deadlock in JobGenerator.stop Because `lazy val` uses `this` lock, if JobGenerator.stop and JobGenerator.doCheckpoint (JobGenerator.shouldCheckpoint has not yet been initialized) run at the same time, it may hang. Here are the stack traces for the deadlock: ```Java "pool-1-thread-1-ScalaTest-running-StreamingListenerSuite" #11 prio=5 os_prio=31 tid=0x00007fd35d094800 nid=0x5703 in Object.wait() [0x000000012ecaf000] java.lang.Thread.State: WAITING (on object monitor) at java.lang.Object.wait(Native Method) at java.lang.Thread.join(Thread.java:1245) - locked <0x00000007b5d8d7f8> (a org.apache.spark.util.EventLoop$$anon$1) at java.lang.Thread.join(Thread.java:1319) at org.apache.spark.util.EventLoop.stop(EventLoop.scala:81) at org.apache.spark.streaming.scheduler.JobGenerator.stop(JobGenerator.scala:155) - locked <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator) at org.apache.spark.streaming.scheduler.JobScheduler.stop(JobScheduler.scala:95) - locked <0x00000007b5d8ced8> (a org.apache.spark.streaming.scheduler.JobScheduler) at org.apache.spark.streaming.StreamingContext.stop(StreamingContext.scala:687) "JobGenerator" #67 daemon prio=5 os_prio=31 tid=0x00007fd35c3b9800 nid=0x9f03 waiting for monitor entry [0x0000000139e4a000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint$lzycompute(JobGenerator.scala:63) - waiting to lock <0x00000007b5d8cea0> (a org.apache.spark.streaming.scheduler.JobGenerator) at org.apache.spark.streaming.scheduler.JobGenerator.shouldCheckpoint(JobGenerator.scala:63) at org.apache.spark.streaming.scheduler.JobGenerator.doCheckpoint(JobGenerator.scala:290) at org.apache.spark.streaming.scheduler.JobGenerator.org$apache$spark$streaming$scheduler$JobGenerator$$processEvent(JobGenerator.scala:182) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:83) at org.apache.spark.streaming.scheduler.JobGenerator$$anon$1.onReceive(JobGenerator.scala:82) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) ``` I can use this patch to produce this deadlock: https://github.com/zsxwing/spark/commit/8a88f28d1331003a65fabef48ae3d22a7c21f05f And a timeout build in Jenkins due to this deadlock: https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1654/ This PR initializes `checkpointWriter` before `eventLoop` uses it to avoid this deadlock. Author: zsxwing Closes #8326 from zsxwing/SPARK-10125. --- .../org/apache/spark/streaming/scheduler/JobGenerator.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 9f2117ada61c0..2de035d166e7b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -79,6 +79,10 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { def start(): Unit = synchronized { if (eventLoop != null) return // generator has already been started + // Call checkpointWriter here to initialize it before eventLoop uses it to avoid a deadlock. + // See SPARK-10125 + checkpointWriter + eventLoop = new EventLoop[JobGeneratorEvent]("JobGenerator") { override protected def onReceive(event: JobGeneratorEvent): Unit = processEvent(event) From 73431d8afb41b93888d2642a1ce2d011f03fb740 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Wed, 19 Aug 2015 19:43:26 -0700 Subject: [PATCH 025/232] [SPARK-10124] [MESOS] Fix removing queued driver in mesos cluster mode. Currently the spark applications can be queued to the Mesos cluster dispatcher, but when multiple jobs are in queue we don't handle removing jobs from the buffer correctly while iterating and causes null pointer exception. This patch copies the buffer before iterating them, so exceptions aren't thrown when the jobs are removed. Author: Timothy Chen Closes #8322 from tnachen/fix_cluster_mode. --- .../cluster/mesos/MesosClusterScheduler.scala | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 64ec2b8e3db15..1206f184fbc82 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -507,14 +507,16 @@ private[spark] class MesosClusterScheduler( val driversToRetry = pendingRetryDrivers.filter { d => d.retryState.get.nextRetry.before(currentTime) } + scheduleTasks( - driversToRetry, + copyBuffer(driversToRetry), removeFromPendingRetryDrivers, currentOffers, tasks) + // Then we walk through the queued drivers and try to schedule them. scheduleTasks( - queuedDrivers, + copyBuffer(queuedDrivers), removeFromQueuedDrivers, currentOffers, tasks) @@ -527,13 +529,14 @@ private[spark] class MesosClusterScheduler( .foreach(o => driver.declineOffer(o.getId)) } + private def copyBuffer( + buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { + val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) + buffer.copyToBuffer(newBuffer) + newBuffer + } + def getSchedulerState(): MesosClusterSchedulerState = { - def copyBuffer( - buffer: ArrayBuffer[MesosDriverDescription]): ArrayBuffer[MesosDriverDescription] = { - val newBuffer = new ArrayBuffer[MesosDriverDescription](buffer.size) - buffer.copyToBuffer(newBuffer) - newBuffer - } stateLock.synchronized { new MesosClusterSchedulerState( frameworkId, From b762f9920f7587d3c08493c49dd2fede62110b88 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 19 Aug 2015 21:15:58 -0700 Subject: [PATCH 026/232] [SPARK-10128] [STREAMING] Used correct classloader to deserialize WAL data Recovering Kinesis sequence numbers from WAL leads to classnotfoundexception because the ObjectInputStream does not use the correct classloader and the SequenceNumberRanges class (in streaming-kinesis-asl package) cannot be found (added through spark-submit) while deserializing. The solution is to use `Thread.currentThread().getContextClassLoader` while deserializing. Author: Tathagata Das Closes #8328 from tdas/SPARK-10128 and squashes the following commits: f19b1c2 [Tathagata Das] Used correct classloader to deserialize WAL data --- .../spark/streaming/scheduler/ReceivedBlockTracker.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 7720259a5d794..53b96d51c9180 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.streaming.Time import org.apache.spark.streaming.util.{WriteAheadLog, WriteAheadLogUtils} import org.apache.spark.util.{Clock, Utils} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf} /** Trait representing any event in the ReceivedBlockTracker that updates its state. */ private[streaming] sealed trait ReceivedBlockTrackerLogEvent @@ -199,7 +199,8 @@ private[streaming] class ReceivedBlockTracker( import scala.collection.JavaConversions._ writeAheadLog.readAll().foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) - Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { + Utils.deserialize[ReceivedBlockTrackerLogEvent]( + byteBuffer.array, Thread.currentThread().getContextClassLoader) match { case BlockAdditionEvent(receivedBlockInfo) => insertAddedBlock(receivedBlockInfo) case BatchAllocationEvent(time, allocatedBlocks) => From 43e0135421b2262cbb0e06aae53523f663b4f959 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 20 Aug 2015 15:30:31 +0800 Subject: [PATCH 027/232] [SPARK-10092] [SQL] Multi-DB support follow up. https://issues.apache.org/jira/browse/SPARK-10092 This pr is a follow-up one for Multi-DB support. It has the following changes: * `HiveContext.refreshTable` now accepts `dbName.tableName`. * `HiveContext.analyze` now accepts `dbName.tableName`. * `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateTempTableUsing`, `CreateTempTableUsingAsSelect`, `CreateMetastoreDataSource`, and `CreateMetastoreDataSourceAsSelect` all take `TableIdentifier` instead of the string representation of table name. * When you call `saveAsTable` with a specified database, the data will be saved to the correct location. * Explicitly do not allow users to create a temporary with a specified database name (users cannot do it before). * When we save table to metastore, we also check if db name and table name can be accepted by hive (using `MetaStoreUtils.validateName`). Author: Yin Huai Closes #8324 from yhuai/saveAsTableDB. --- .../spark/sql/catalyst/TableIdentifier.scala | 4 +- .../spark/sql/catalyst/analysis/Catalog.scala | 63 +++++-- .../apache/spark/sql/DataFrameWriter.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 15 +- .../spark/sql/execution/SparkStrategies.scala | 10 +- .../sql/execution/datasources/DDLParser.scala | 32 ++-- .../spark/sql/execution/datasources/ddl.scala | 22 +-- .../sql/execution/datasources/rules.scala | 8 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 35 ++++ .../apache/spark/sql/hive/HiveContext.scala | 14 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 22 ++- .../spark/sql/hive/HiveStrategies.scala | 12 +- .../spark/sql/hive/execution/commands.scala | 54 ++++-- .../spark/sql/hive/ListTablesSuite.scala | 6 - .../spark/sql/hive/MultiDatabaseSuite.scala | 158 +++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 35 ++++ 16 files changed, 398 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala index aebcdeb9d070f..d701559bf2d9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala @@ -25,7 +25,9 @@ private[sql] case class TableIdentifier(table: String, database: Option[String] def toSeq: Seq[String] = database.toSeq :+ table - override def toString: String = toSeq.map("`" + _ + "`").mkString(".") + override def toString: String = quotedString + + def quotedString: String = toSeq.map("`" + _ + "`").mkString(".") def unquotedString: String = toSeq.mkString(".") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 5766e6a2dd51a..503c4f4b20f38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{TableIdentifier, CatalystConf, EmptyConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} @@ -55,12 +56,15 @@ trait Catalog { def refreshTable(tableIdent: TableIdentifier): Unit + // TODO: Refactor it in the work of SPARK-10104 def registerTable(tableIdentifier: Seq[String], plan: LogicalPlan): Unit + // TODO: Refactor it in the work of SPARK-10104 def unregisterTable(tableIdentifier: Seq[String]): Unit def unregisterAllTables(): Unit + // TODO: Refactor it in the work of SPARK-10104 protected def processTableIdentifier(tableIdentifier: Seq[String]): Seq[String] = { if (conf.caseSensitiveAnalysis) { tableIdentifier @@ -69,6 +73,7 @@ trait Catalog { } } + // TODO: Refactor it in the work of SPARK-10104 protected def getDbTableName(tableIdent: Seq[String]): String = { val size = tableIdent.size if (size <= 2) { @@ -78,9 +83,22 @@ trait Catalog { } } + // TODO: Refactor it in the work of SPARK-10104 protected def getDBTable(tableIdent: Seq[String]) : (Option[String], String) = { (tableIdent.lift(tableIdent.size - 2), tableIdent.last) } + + /** + * It is not allowed to specifiy database name for tables stored in [[SimpleCatalog]]. + * We use this method to check it. + */ + protected def checkTableIdentifier(tableIdentifier: Seq[String]): Unit = { + if (tableIdentifier.length > 1) { + throw new AnalysisException("Specifying database name or other qualifiers are not allowed " + + "for temporary tables. If the table name has dots (.) in it, please quote the " + + "table name with backticks (`).") + } + } } class SimpleCatalog(val conf: CatalystConf) extends Catalog { @@ -89,11 +107,13 @@ class SimpleCatalog(val conf: CatalystConf) extends Catalog { override def registerTable( tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { + checkTableIdentifier(tableIdentifier) val tableIdent = processTableIdentifier(tableIdentifier) tables.put(getDbTableName(tableIdent), plan) } override def unregisterTable(tableIdentifier: Seq[String]): Unit = { + checkTableIdentifier(tableIdentifier) val tableIdent = processTableIdentifier(tableIdentifier) tables.remove(getDbTableName(tableIdent)) } @@ -103,6 +123,7 @@ class SimpleCatalog(val conf: CatalystConf) extends Catalog { } override def tableExists(tableIdentifier: Seq[String]): Boolean = { + checkTableIdentifier(tableIdentifier) val tableIdent = processTableIdentifier(tableIdentifier) tables.containsKey(getDbTableName(tableIdent)) } @@ -110,6 +131,7 @@ class SimpleCatalog(val conf: CatalystConf) extends Catalog { override def lookupRelation( tableIdentifier: Seq[String], alias: Option[String] = None): LogicalPlan = { + checkTableIdentifier(tableIdentifier) val tableIdent = processTableIdentifier(tableIdentifier) val tableFullName = getDbTableName(tableIdent) val table = tables.get(tableFullName) @@ -149,7 +171,13 @@ trait OverrideCatalog extends Catalog { abstract override def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) - overrides.get(getDBTable(tableIdent)) match { + // A temporary tables only has a single part in the tableIdentifier. + val overriddenTable = if (tableIdentifier.length > 1) { + None: Option[LogicalPlan] + } else { + overrides.get(getDBTable(tableIdent)) + } + overriddenTable match { case Some(_) => true case None => super.tableExists(tableIdentifier) } @@ -159,7 +187,12 @@ trait OverrideCatalog extends Catalog { tableIdentifier: Seq[String], alias: Option[String] = None): LogicalPlan = { val tableIdent = processTableIdentifier(tableIdentifier) - val overriddenTable = overrides.get(getDBTable(tableIdent)) + // A temporary tables only has a single part in the tableIdentifier. + val overriddenTable = if (tableIdentifier.length > 1) { + None: Option[LogicalPlan] + } else { + overrides.get(getDBTable(tableIdent)) + } val tableWithQualifers = overriddenTable.map(r => Subquery(tableIdent.last, r)) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are @@ -171,20 +204,8 @@ trait OverrideCatalog extends Catalog { } abstract override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { - val dbName = if (conf.caseSensitiveAnalysis) { - databaseName - } else { - if (databaseName.isDefined) Some(databaseName.get.toLowerCase) else None - } - - val temporaryTables = overrides.filter { - // If a temporary table does not have an associated database, we should return its name. - case ((None, _), _) => true - // If a temporary table does have an associated database, we should return it if the database - // matches the given database name. - case ((db: Some[String], _), _) if db == dbName => true - case _ => false - }.map { + // We always return all temporary tables. + val temporaryTables = overrides.map { case ((_, tableName), _) => (tableName, true) }.toSeq @@ -194,13 +215,19 @@ trait OverrideCatalog extends Catalog { override def registerTable( tableIdentifier: Seq[String], plan: LogicalPlan): Unit = { + checkTableIdentifier(tableIdentifier) val tableIdent = processTableIdentifier(tableIdentifier) overrides.put(getDBTable(tableIdent), plan) } override def unregisterTable(tableIdentifier: Seq[String]): Unit = { - val tableIdent = processTableIdentifier(tableIdentifier) - overrides.remove(getDBTable(tableIdent)) + // A temporary tables only has a single part in the tableIdentifier. + // If tableIdentifier has more than one parts, it is not a temporary table + // and we do not need to do anything at here. + if (tableIdentifier.length == 1) { + val tableIdent = processTableIdentifier(tableIdentifier) + overrides.remove(getDBTable(tableIdent)) + } } override def unregisterAllTables(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index f0bf1be506411..ce8744b53175b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -218,7 +218,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { case _ => val cmd = CreateTableUsingAsSelect( - tableIdent.unquotedString, + tableIdent, source, temporary = false, partitioningColumns.map(_.toArray).getOrElse(Array.empty[String]), 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 58fe75b59f418..126c9c6f839c7 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 @@ -584,9 +584,10 @@ class SQLContext(@transient val sparkContext: SparkContext) tableName: String, source: String, options: Map[String, String]): DataFrame = { + val tableIdent = new SqlParser().parseTableIdentifier(tableName) val cmd = CreateTableUsing( - tableName, + tableIdent, userSpecifiedSchema = None, source, temporary = false, @@ -594,7 +595,7 @@ class SQLContext(@transient val sparkContext: SparkContext) allowExisting = false, managedIfNoPath = false) executePlan(cmd).toRdd - table(tableName) + table(tableIdent) } /** @@ -629,9 +630,10 @@ class SQLContext(@transient val sparkContext: SparkContext) source: String, schema: StructType, options: Map[String, String]): DataFrame = { + val tableIdent = new SqlParser().parseTableIdentifier(tableName) val cmd = CreateTableUsing( - tableName, + tableIdent, userSpecifiedSchema = Some(schema), source, temporary = false, @@ -639,7 +641,7 @@ class SQLContext(@transient val sparkContext: SparkContext) allowExisting = false, managedIfNoPath = false) executePlan(cmd).toRdd - table(tableName) + table(tableIdent) } /** @@ -724,7 +726,10 @@ class SQLContext(@transient val sparkContext: SparkContext) * @since 1.3.0 */ def table(tableName: String): DataFrame = { - val tableIdent = new SqlParser().parseTableIdentifier(tableName) + table(new SqlParser().parseTableIdentifier(tableName)) + } + + private def table(tableIdent: TableIdentifier): DataFrame = { DataFrame(this, catalog.lookupRelation(tableIdent.toSeq)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 1fc870d44b578..4df53687a0731 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -395,22 +395,22 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object DDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case CreateTableUsing(tableName, userSpecifiedSchema, provider, true, opts, false, _) => + case CreateTableUsing(tableIdent, userSpecifiedSchema, provider, true, opts, false, _) => ExecutedCommand( CreateTempTableUsing( - tableName, userSpecifiedSchema, provider, opts)) :: Nil + tableIdent, userSpecifiedSchema, provider, opts)) :: Nil case c: CreateTableUsing if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") case c: CreateTableUsing if c.temporary && c.allowExisting => sys.error("allowExisting should be set to false when creating a temporary table.") - case CreateTableUsingAsSelect(tableName, provider, true, partitionsCols, mode, opts, query) + case CreateTableUsingAsSelect(tableIdent, provider, true, partitionsCols, mode, opts, query) if partitionsCols.nonEmpty => sys.error("Cannot create temporary partitioned table.") - case CreateTableUsingAsSelect(tableName, provider, true, _, mode, opts, query) => + case CreateTableUsingAsSelect(tableIdent, provider, true, _, mode, opts, query) => val cmd = CreateTempTableUsingAsSelect( - tableName, provider, Array.empty[String], mode, opts, query) + tableIdent, provider, Array.empty[String], mode, opts, query) ExecutedCommand(cmd) :: Nil case c: CreateTableUsingAsSelect if !c.temporary => sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala index 6c462fa30461b..f7a88b98c0b48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DDLParser.scala @@ -80,9 +80,9 @@ class DDLParser(parseQuery: String => LogicalPlan) */ protected lazy val createTable: Parser[LogicalPlan] = { // TODO: Support database.table. - (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ ident ~ + (CREATE ~> TEMPORARY.? <~ TABLE) ~ (IF ~> NOT <~ EXISTS).? ~ tableIdentifier ~ tableCols.? ~ (USING ~> className) ~ (OPTIONS ~> options).? ~ (AS ~> restInput).? ^^ { - case temp ~ allowExisting ~ tableName ~ columns ~ provider ~ opts ~ query => + case temp ~ allowExisting ~ tableIdent ~ columns ~ provider ~ opts ~ query => if (temp.isDefined && allowExisting.isDefined) { throw new DDLException( "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.") @@ -104,7 +104,7 @@ class DDLParser(parseQuery: String => LogicalPlan) } val queryPlan = parseQuery(query.get) - CreateTableUsingAsSelect(tableName, + CreateTableUsingAsSelect(tableIdent, provider, temp.isDefined, Array.empty[String], @@ -114,7 +114,7 @@ class DDLParser(parseQuery: String => LogicalPlan) } else { val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) CreateTableUsing( - tableName, + tableIdent, userSpecifiedSchema, provider, temp.isDefined, @@ -125,6 +125,12 @@ class DDLParser(parseQuery: String => LogicalPlan) } } + // This is the same as tableIdentifier in SqlParser. + protected lazy val tableIdentifier: Parser[TableIdentifier] = + (ident <~ ".").? ~ ident ^^ { + case maybeDbName ~ tableName => TableIdentifier(tableName, maybeDbName) + } + protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")" /* @@ -132,21 +138,15 @@ class DDLParser(parseQuery: String => LogicalPlan) * This will display all columns of table `avroTable` includes column_name,column_type,comment */ protected lazy val describeTable: Parser[LogicalPlan] = - (DESCRIBE ~> opt(EXTENDED)) ~ (ident <~ ".").? ~ ident ^^ { - case e ~ db ~ tbl => - val tblIdentifier = db match { - case Some(dbName) => - Seq(dbName, tbl) - case None => - Seq(tbl) - } - DescribeCommand(UnresolvedRelation(tblIdentifier, None), e.isDefined) + (DESCRIBE ~> opt(EXTENDED)) ~ tableIdentifier ^^ { + case e ~ tableIdent => + DescribeCommand(UnresolvedRelation(tableIdent.toSeq, None), e.isDefined) } protected lazy val refreshTable: Parser[LogicalPlan] = - REFRESH ~> TABLE ~> (ident <~ ".").? ~ ident ^^ { - case maybeDatabaseName ~ tableName => - RefreshTable(TableIdentifier(tableName, maybeDatabaseName)) + REFRESH ~> TABLE ~> tableIdentifier ^^ { + case tableIndet => + RefreshTable(tableIndet) } protected lazy val options: Parser[Map[String, String]] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index ecd304c30cdee..31d6b75e13477 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -53,7 +53,7 @@ case class DescribeCommand( * If it is false, an exception will be thrown */ case class CreateTableUsing( - tableName: String, + tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], provider: String, temporary: Boolean, @@ -71,8 +71,9 @@ case class CreateTableUsing( * can analyze the logical plan that will be used to populate the table. * So, [[PreWriteCheck]] can detect cases that are not allowed. */ +// TODO: Use TableIdentifier instead of String for tableName (SPARK-10104). case class CreateTableUsingAsSelect( - tableName: String, + tableIdent: TableIdentifier, provider: String, temporary: Boolean, partitionColumns: Array[String], @@ -80,12 +81,10 @@ case class CreateTableUsingAsSelect( options: Map[String, String], child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = Seq.empty[Attribute] - // TODO: Override resolved after we support databaseName. - // override lazy val resolved = databaseName != None && childrenResolved } case class CreateTempTableUsing( - tableName: String, + tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], provider: String, options: Map[String, String]) extends RunnableCommand { @@ -93,14 +92,16 @@ case class CreateTempTableUsing( def run(sqlContext: SQLContext): Seq[Row] = { val resolved = ResolvedDataSource( sqlContext, userSpecifiedSchema, Array.empty[String], provider, options) - sqlContext.registerDataFrameAsTable( - DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) + sqlContext.catalog.registerTable( + tableIdent.toSeq, + DataFrame(sqlContext, LogicalRelation(resolved.relation)).logicalPlan) + Seq.empty[Row] } } case class CreateTempTableUsingAsSelect( - tableName: String, + tableIdent: TableIdentifier, provider: String, partitionColumns: Array[String], mode: SaveMode, @@ -110,8 +111,9 @@ case class CreateTempTableUsingAsSelect( override def run(sqlContext: SQLContext): Seq[Row] = { val df = DataFrame(sqlContext, query) val resolved = ResolvedDataSource(sqlContext, provider, partitionColumns, mode, options, df) - sqlContext.registerDataFrameAsTable( - DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) + sqlContext.catalog.registerTable( + tableIdent.toSeq, + DataFrame(sqlContext, LogicalRelation(resolved.relation)).logicalPlan) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 9d3d35692ffcc..16c9138419fa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -140,12 +140,12 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => // OK } - case CreateTableUsingAsSelect(tableName, _, _, partitionColumns, mode, _, query) => + case CreateTableUsingAsSelect(tableIdent, _, _, partitionColumns, mode, _, query) => // When the SaveMode is Overwrite, we need to check if the table is an input table of // the query. If so, we will throw an AnalysisException to let users know it is not allowed. - if (mode == SaveMode.Overwrite && catalog.tableExists(Seq(tableName))) { + if (mode == SaveMode.Overwrite && catalog.tableExists(tableIdent.toSeq)) { // Need to remove SubQuery operator. - EliminateSubQueries(catalog.lookupRelation(Seq(tableName))) match { + EliminateSubQueries(catalog.lookupRelation(tableIdent.toSeq)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). case l @ LogicalRelation(dest: BaseRelation) => @@ -155,7 +155,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } if (srcRelations.contains(dest)) { failAnalysis( - s"Cannot overwrite table $tableName that is also being read from.") + s"Cannot overwrite table $tableIdent that is also being read from.") } else { // OK } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 141468ca00d67..da50aec17c89e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1644,4 +1644,39 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select count(num) from 1one"), Row(10)) } } + + test("specifying database name for a temporary table is not allowed") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = + sqlContext.sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + df + .write + .format("parquet") + .save(path) + + val message = intercept[AnalysisException] { + sqlContext.sql( + s""" + |CREATE TEMPORARY TABLE db.t + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + }.getMessage + assert(message.contains("Specifying database name or other qualifiers are not allowed")) + + // If you use backticks to quote the name of a temporary table having dot in it. + sqlContext.sql( + s""" + |CREATE TEMPORARY TABLE `db.t` + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + checkAnswer(sqlContext.table("`db.t`"), df) + } + } } 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 index 17762649fd70d..17cc83087fb1d 100644 --- 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 @@ -43,7 +43,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ import org.apache.spark.sql.SQLConf.SQLConfEntry import org.apache.spark.sql.SQLConf.SQLConfEntry._ -import org.apache.spark.sql.catalyst.{TableIdentifier, ParserDialect} +import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier, ParserDialect} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUDFs, SetCommand} @@ -189,6 +189,10 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { // We instantiate a HiveConf here to read in the hive-site.xml file and then pass the options // into the isolated client loader val metadataConf = new HiveConf() + + val defaltWarehouseLocation = metadataConf.get("hive.metastore.warehouse.dir") + logInfo("defalt warehouse location is " + defaltWarehouseLocation) + // `configure` goes second to override other settings. val allConfig = metadataConf.iterator.map(e => e.getKey -> e.getValue).toMap ++ configure @@ -288,12 +292,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { * @since 1.3.0 */ def refreshTable(tableName: String): Unit = { - val tableIdent = TableIdentifier(tableName).withDatabase(catalog.client.currentDatabase) + val tableIdent = new SqlParser().parseTableIdentifier(tableName) catalog.refreshTable(tableIdent) } protected[hive] def invalidateTable(tableName: String): Unit = { - catalog.invalidateTable(catalog.client.currentDatabase, tableName) + val tableIdent = new SqlParser().parseTableIdentifier(tableName) + catalog.invalidateTable(tableIdent) } /** @@ -307,7 +312,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { */ @Experimental def analyze(tableName: String) { - val relation = EliminateSubQueries(catalog.lookupRelation(Seq(tableName))) + val tableIdent = new SqlParser().parseTableIdentifier(tableName) + val relation = EliminateSubQueries(catalog.lookupRelation(tableIdent.toSeq)) relation match { case relation: MetastoreRelation => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 6770462bb0ad3..bbe8c1911bf86 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -174,10 +174,13 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive // it is better at here to invalidate the cache to avoid confusing waring logs from the // cache loader (e.g. cannot find data source provider, which is only defined for // data source table.). - invalidateTable(tableIdent.database.getOrElse(client.currentDatabase), tableIdent.table) + invalidateTable(tableIdent) } - def invalidateTable(databaseName: String, tableName: String): Unit = { + def invalidateTable(tableIdent: TableIdentifier): Unit = { + val databaseName = tableIdent.database.getOrElse(client.currentDatabase) + val tableName = tableIdent.table + cachedDataSourceTables.invalidate(QualifiedTableName(databaseName, tableName).toLowerCase) } @@ -187,6 +190,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive * Creates a data source table (a table created with USING clause) in Hive's metastore. * Returns true when the table has been created. Otherwise, false. */ + // TODO: Remove this in SPARK-10104. def createDataSourceTable( tableName: String, userSpecifiedSchema: Option[StructType], @@ -203,7 +207,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive isExternal) } - private def createDataSourceTable( + def createDataSourceTable( tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], partitionColumns: Array[String], @@ -371,10 +375,16 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive } def hiveDefaultTableFilePath(tableName: String): String = { + hiveDefaultTableFilePath(new SqlParser().parseTableIdentifier(tableName)) + } + + def hiveDefaultTableFilePath(tableIdent: TableIdentifier): String = { // Code based on: hiveWarehouse.getTablePath(currentDatabase, tableName) + val database = tableIdent.database.getOrElse(client.currentDatabase) + new Path( - new Path(client.getDatabase(client.currentDatabase).location), - tableName.toLowerCase).toString + new Path(client.getDatabase(database).location), + tableIdent.table.toLowerCase).toString } def tableExists(tableIdentifier: Seq[String]): Boolean = { @@ -635,7 +645,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( - desc.name, + TableIdentifier(desc.name), hive.conf.defaultDataSourceName, temporary = false, Array.empty[String], diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index cd6cd322c94ed..d38ad9127327d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -83,14 +83,16 @@ private[hive] trait HiveStrategies { object HiveDDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case CreateTableUsing( - tableName, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => - ExecutedCommand( + tableIdent, userSpecifiedSchema, provider, false, opts, allowExisting, managedIfNoPath) => + val cmd = CreateMetastoreDataSource( - tableName, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath)) :: Nil + tableIdent, userSpecifiedSchema, provider, opts, allowExisting, managedIfNoPath) + ExecutedCommand(cmd) :: Nil - case CreateTableUsingAsSelect(tableName, provider, false, partitionCols, mode, opts, query) => + case CreateTableUsingAsSelect( + tableIdent, provider, false, partitionCols, mode, opts, query) => val cmd = - CreateMetastoreDataSourceAsSelect(tableName, provider, partitionCols, mode, opts, query) + CreateMetastoreDataSourceAsSelect(tableIdent, provider, partitionCols, mode, opts, query) ExecutedCommand(cmd) :: Nil case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 05a78930afe3d..d1699dd536817 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.hive.execution +import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.{TableIdentifier, SqlParser} import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -120,9 +122,10 @@ case class AddFile(path: String) extends RunnableCommand { } } +// TODO: Use TableIdentifier instead of String for tableName (SPARK-10104). private[hive] case class CreateMetastoreDataSource( - tableName: String, + tableIdent: TableIdentifier, userSpecifiedSchema: Option[StructType], provider: String, options: Map[String, String], @@ -130,9 +133,24 @@ case class CreateMetastoreDataSource( managedIfNoPath: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { + // Since we are saving metadata to metastore, we need to check if metastore supports + // the table name and database name we have for this query. MetaStoreUtils.validateName + // is the method used by Hive to check if a table name or a database name is valid for + // the metastore. + if (!MetaStoreUtils.validateName(tableIdent.table)) { + throw new AnalysisException(s"Table name ${tableIdent.table} is not a valid name for " + + s"metastore. Metastore only accepts table name containing characters, numbers and _.") + } + if (tableIdent.database.isDefined && !MetaStoreUtils.validateName(tableIdent.database.get)) { + throw new AnalysisException(s"Database name ${tableIdent.database.get} is not a valid name " + + s"for metastore. Metastore only accepts database name containing " + + s"characters, numbers and _.") + } + + val tableName = tableIdent.unquotedString val hiveContext = sqlContext.asInstanceOf[HiveContext] - if (hiveContext.catalog.tableExists(tableName :: Nil)) { + if (hiveContext.catalog.tableExists(tableIdent.toSeq)) { if (allowExisting) { return Seq.empty[Row] } else { @@ -144,13 +162,13 @@ case class CreateMetastoreDataSource( val optionsWithPath = if (!options.contains("path") && managedIfNoPath) { isExternal = false - options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableName)) + options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } hiveContext.catalog.createDataSourceTable( - tableName, + tableIdent, userSpecifiedSchema, Array.empty[String], provider, @@ -161,9 +179,10 @@ case class CreateMetastoreDataSource( } } +// TODO: Use TableIdentifier instead of String for tableName (SPARK-10104). private[hive] case class CreateMetastoreDataSourceAsSelect( - tableName: String, + tableIdent: TableIdentifier, provider: String, partitionColumns: Array[String], mode: SaveMode, @@ -171,19 +190,34 @@ case class CreateMetastoreDataSourceAsSelect( query: LogicalPlan) extends RunnableCommand { override def run(sqlContext: SQLContext): Seq[Row] = { + // Since we are saving metadata to metastore, we need to check if metastore supports + // the table name and database name we have for this query. MetaStoreUtils.validateName + // is the method used by Hive to check if a table name or a database name is valid for + // the metastore. + if (!MetaStoreUtils.validateName(tableIdent.table)) { + throw new AnalysisException(s"Table name ${tableIdent.table} is not a valid name for " + + s"metastore. Metastore only accepts table name containing characters, numbers and _.") + } + if (tableIdent.database.isDefined && !MetaStoreUtils.validateName(tableIdent.database.get)) { + throw new AnalysisException(s"Database name ${tableIdent.database.get} is not a valid name " + + s"for metastore. Metastore only accepts database name containing " + + s"characters, numbers and _.") + } + + val tableName = tableIdent.unquotedString val hiveContext = sqlContext.asInstanceOf[HiveContext] var createMetastoreTable = false var isExternal = true val optionsWithPath = if (!options.contains("path")) { isExternal = false - options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableName)) + options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableIdent)) } else { options } var existingSchema = None: Option[StructType] - if (sqlContext.catalog.tableExists(Seq(tableName))) { + if (sqlContext.catalog.tableExists(tableIdent.toSeq)) { // Check if we need to throw an exception or just return. mode match { case SaveMode.ErrorIfExists => @@ -200,7 +234,7 @@ case class CreateMetastoreDataSourceAsSelect( val resolved = ResolvedDataSource( sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) - EliminateSubQueries(sqlContext.table(tableName).logicalPlan) match { + EliminateSubQueries(sqlContext.catalog.lookupRelation(tableIdent.toSeq)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation) => if (l.relation != createdRelation.relation) { val errorDescription = @@ -249,7 +283,7 @@ case class CreateMetastoreDataSourceAsSelect( // the schema of df). It is important since the nullability may be changed by the relation // provider (for example, see org.apache.spark.sql.parquet.DefaultSource). hiveContext.catalog.createDataSourceTable( - tableName, + tableIdent, Some(resolved.relation.schema), partitionColumns, provider, @@ -258,7 +292,7 @@ case class CreateMetastoreDataSourceAsSelect( } // Refresh the cache of the table in the catalog. - hiveContext.refreshTable(tableName) + hiveContext.catalog.refreshTable(tableIdent) Seq.empty[Row] } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala index 1c15997ea8e6d..d3388a9429e41 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ListTablesSuite.scala @@ -34,7 +34,6 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { override def beforeAll(): Unit = { // The catalog in HiveContext is a case insensitive one. catalog.registerTable(Seq("ListTablesSuiteTable"), df.logicalPlan) - catalog.registerTable(Seq("ListTablesSuiteDB", "InDBListTablesSuiteTable"), df.logicalPlan) sql("CREATE TABLE HiveListTablesSuiteTable (key int, value string)") sql("CREATE DATABASE IF NOT EXISTS ListTablesSuiteDB") sql("CREATE TABLE ListTablesSuiteDB.HiveInDBListTablesSuiteTable (key int, value string)") @@ -42,7 +41,6 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { override def afterAll(): Unit = { catalog.unregisterTable(Seq("ListTablesSuiteTable")) - catalog.unregisterTable(Seq("ListTablesSuiteDB", "InDBListTablesSuiteTable")) sql("DROP TABLE IF EXISTS HiveListTablesSuiteTable") sql("DROP TABLE IF EXISTS ListTablesSuiteDB.HiveInDBListTablesSuiteTable") sql("DROP DATABASE IF EXISTS ListTablesSuiteDB") @@ -55,7 +53,6 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) - assert(allTables.filter("tableName = 'indblisttablessuitetable'").count() === 0) checkAnswer( allTables.filter("tableName = 'hivelisttablessuitetable'"), Row("hivelisttablessuitetable", false)) @@ -69,9 +66,6 @@ class ListTablesSuite extends QueryTest with BeforeAndAfterAll { checkAnswer( allTables.filter("tableName = 'listtablessuitetable'"), Row("listtablessuitetable", true)) - checkAnswer( - allTables.filter("tableName = 'indblisttablessuitetable'"), - Row("indblisttablessuitetable", true)) assert(allTables.filter("tableName = 'hivelisttablessuitetable'").count() === 0) checkAnswer( allTables.filter("tableName = 'hiveindblisttablessuitetable'"), 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 417e8b07917cc..997c667ec0d1b 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 @@ -19,14 +19,22 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.{QueryTest, SQLContext, SaveMode} +import org.apache.spark.sql.{AnalysisException, QueryTest, SQLContext, SaveMode} class MultiDatabaseSuite extends QueryTest with SQLTestUtils { - override val _sqlContext: SQLContext = TestHive + override val _sqlContext: HiveContext = TestHive private val sqlContext = _sqlContext private val df = sqlContext.range(10).coalesce(1) + private def checkTablePath(dbName: String, tableName: String): Unit = { + // val hiveContext = sqlContext.asInstanceOf[HiveContext] + val metastoreTable = sqlContext.catalog.client.getTable(dbName, tableName) + val expectedPath = sqlContext.catalog.client.getDatabase(dbName).location + "/" + tableName + + assert(metastoreTable.serdeProperties("path") === expectedPath) + } + test(s"saveAsTable() to non-default database - with USE - Overwrite") { withTempDatabase { db => activateDatabase(db) { @@ -37,6 +45,8 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { assert(sqlContext.tableNames(db).contains("t")) checkAnswer(sqlContext.table(s"$db.t"), df) + + checkTablePath(db, "t") } } @@ -45,6 +55,58 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { df.write.mode(SaveMode.Overwrite).saveAsTable(s"$db.t") assert(sqlContext.tableNames(db).contains("t")) checkAnswer(sqlContext.table(s"$db.t"), df) + + checkTablePath(db, "t") + } + } + + test(s"createExternalTable() 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) + + sqlContext.createExternalTable("t", path, "parquet") + assert(sqlContext.tableNames(db).contains("t")) + checkAnswer(sqlContext.table("t"), df) + + sql( + s""" + |CREATE TABLE t1 + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + assert(sqlContext.tableNames(db).contains("t1")) + checkAnswer(sqlContext.table("t1"), df) + } + } + } + } + + test(s"createExternalTable() to non-default database - without USE") { + withTempDatabase { db => + withTempPath { dir => + val path = dir.getCanonicalPath + df.write.format("parquet").mode(SaveMode.Overwrite).save(path) + sqlContext.createExternalTable(s"$db.t", path, "parquet") + + assert(sqlContext.tableNames(db).contains("t")) + checkAnswer(sqlContext.table(s"$db.t"), df) + + sql( + s""" + |CREATE TABLE $db.t1 + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + assert(sqlContext.tableNames(db).contains("t1")) + checkAnswer(sqlContext.table(s"$db.t1"), df) + } } } @@ -59,6 +121,8 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { assert(sqlContext.tableNames(db).contains("t")) checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + + checkTablePath(db, "t") } } @@ -68,6 +132,8 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { df.write.mode(SaveMode.Append).saveAsTable(s"$db.t") assert(sqlContext.tableNames(db).contains("t")) checkAnswer(sqlContext.table(s"$db.t"), df.unionAll(df)) + + checkTablePath(db, "t") } } @@ -130,7 +196,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { } } - test("Refreshes a table in a non-default database") { + test("Refreshes a table in a non-default database - with USE") { import org.apache.spark.sql.functions.lit withTempDatabase { db => @@ -151,8 +217,94 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils { sql("ALTER TABLE t ADD PARTITION (p=1)") sql("REFRESH TABLE t") checkAnswer(sqlContext.table("t"), df.withColumn("p", lit(1))) + + df.write.parquet(s"$path/p=2") + sql("ALTER TABLE t ADD PARTITION (p=2)") + sqlContext.refreshTable("t") + checkAnswer( + sqlContext.table("t"), + df.withColumn("p", lit(1)).unionAll(df.withColumn("p", lit(2)))) } } } } + + test("Refreshes a table in a non-default database - without USE") { + import org.apache.spark.sql.functions.lit + + withTempDatabase { db => + withTempPath { dir => + val path = dir.getCanonicalPath + + sql( + s"""CREATE EXTERNAL TABLE $db.t (id BIGINT) + |PARTITIONED BY (p INT) + |STORED AS PARQUET + |LOCATION '$path' + """.stripMargin) + + checkAnswer(sqlContext.table(s"$db.t"), sqlContext.emptyDataFrame) + + df.write.parquet(s"$path/p=1") + sql(s"ALTER TABLE $db.t ADD PARTITION (p=1)") + sql(s"REFRESH TABLE $db.t") + checkAnswer(sqlContext.table(s"$db.t"), df.withColumn("p", lit(1))) + + df.write.parquet(s"$path/p=2") + sql(s"ALTER TABLE $db.t ADD PARTITION (p=2)") + sqlContext.refreshTable(s"$db.t") + checkAnswer( + sqlContext.table(s"$db.t"), + df.withColumn("p", lit(1)).unionAll(df.withColumn("p", lit(2)))) + } + } + } + + test("invalid database name and table names") { + { + val message = intercept[AnalysisException] { + df.write.format("parquet").saveAsTable("`d:b`.`t:a`") + }.getMessage + assert(message.contains("is not a valid name for metastore")) + } + + { + val message = intercept[AnalysisException] { + df.write.format("parquet").saveAsTable("`d:b`.`table`") + }.getMessage + assert(message.contains("is not a valid name for metastore")) + } + + withTempPath { dir => + val path = dir.getCanonicalPath + + { + val message = intercept[AnalysisException] { + sql( + s""" + |CREATE TABLE `d:b`.`t:a` (a int) + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + }.getMessage + assert(message.contains("is not a valid name for metastore")) + } + + { + val message = intercept[AnalysisException] { + sql( + s""" + |CREATE TABLE `d:b`.`table` (a int) + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + }.getMessage + assert(message.contains("is not a valid name for metastore")) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 8b8f520776e70..55ecbd5b5f21d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1138,4 +1138,39 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils { Row(CalendarInterval.fromString( "interval 4 minutes 59 seconds 889 milliseconds 987 microseconds"))) } + + test("specifying database name for a temporary table is not allowed") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = + sqlContext.sparkContext.parallelize(1 to 10).map(i => (i, i.toString)).toDF("num", "str") + df + .write + .format("parquet") + .save(path) + + val message = intercept[AnalysisException] { + sqlContext.sql( + s""" + |CREATE TEMPORARY TABLE db.t + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + }.getMessage + assert(message.contains("Specifying database name or other qualifiers are not allowed")) + + // If you use backticks to quote the name of a temporary table having dot in it. + sqlContext.sql( + s""" + |CREATE TEMPORARY TABLE `db.t` + |USING parquet + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + checkAnswer(sqlContext.table("`db.t`"), df) + } + } } From b4f4e91c395cb69ced61d9ff1492d1b814f96828 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 20 Aug 2015 07:53:27 -0700 Subject: [PATCH 028/232] [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation. This improves performance by ~ 20 - 30% in one of my local test and should fix the performance regression from 1.4 to 1.5 on ss_max. Author: Reynold Xin Closes #8332 from rxin/SPARK-10100. --- .../aggregate/TungstenAggregate.scala | 2 +- .../TungstenAggregationIterator.scala | 30 +++++++++++++------ 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 99f51ba5b6935..ba379d358d206 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -104,7 +104,7 @@ case class TungstenAggregate( } else { // This is a grouped aggregate and the input iterator is empty, // so return an empty iterator. - Iterator[UnsafeRow]() + Iterator.empty } } else { aggregationIterator.start(parentIterator) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index af7e0fcedbe4e..26fdbc83ef50b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -357,18 +357,30 @@ class TungstenAggregationIterator( // sort-based aggregation (by calling switchToSortBasedAggregation). private def processInputs(): Unit = { assert(inputIter != null, "attempted to process input when iterator was null") - while (!sortBased && inputIter.hasNext) { - val newInput = inputIter.next() - numInputRows += 1 - val groupingKey = groupProjection.apply(newInput) + if (groupingExpressions.isEmpty) { + // If there is no grouping expressions, we can just reuse the same buffer over and over again. + // Note that it would be better to eliminate the hash map entirely in the future. + val groupingKey = groupProjection.apply(null) val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) - if (buffer == null) { - // buffer == null means that we could not allocate more memory. - // Now, we need to spill the map and switch to sort-based aggregation. - switchToSortBasedAggregation(groupingKey, newInput) - } else { + while (inputIter.hasNext) { + val newInput = inputIter.next() + numInputRows += 1 processRow(buffer, newInput) } + } else { + while (!sortBased && inputIter.hasNext) { + val newInput = inputIter.next() + numInputRows += 1 + val groupingKey = groupProjection.apply(newInput) + val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) + if (buffer == null) { + // buffer == null means that we could not allocate more memory. + // Now, we need to spill the map and switch to sort-based aggregation. + switchToSortBasedAggregation(groupingKey, newInput) + } else { + processRow(buffer, newInput) + } + } } } From 52c60537a274af5414f6b0340a4bd7488ef35280 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 20 Aug 2015 10:05:31 -0700 Subject: [PATCH 029/232] [MINOR] [SQL] Fix sphinx warnings in PySpark SQL Author: MechCoder Closes #8171 from MechCoder/sql_sphinx. --- python/pyspark/context.py | 8 ++++---- python/pyspark/sql/types.py | 4 +++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index eb5b0bbbdac4b..1b2a52ad64114 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -302,10 +302,10 @@ def applicationId(self): """ A unique identifier for the Spark application. Its format depends on the scheduler implementation. - (i.e. - in case of local spark app something like 'local-1433865536131' - in case of YARN something like 'application_1433865536131_34483' - ) + + * in case of local spark app something like 'local-1433865536131' + * in case of YARN something like 'application_1433865536131_34483' + >>> sc.applicationId # doctest: +ELLIPSIS u'local-...' """ diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index c083bf89905bf..ed4e5b594bd61 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -467,9 +467,11 @@ def add(self, field, data_type=None, nullable=True, metadata=None): """ Construct a StructType by adding new elements to it to define the schema. The method accepts either: + a) A single parameter which is a StructField object. b) Between 2 and 4 parameters as (name, data_type, nullable (optional), - metadata(optional). The data_type parameter may be either a String or a DataType object + metadata(optional). The data_type parameter may be either a String or a + DataType object. >>> struct1 = StructType().add("f1", StringType(), True).add("f2", StringType(), True, None) >>> struct2 = StructType([StructField("f1", StringType(), True),\ From 39e91fe2fd43044cc734d55625a3c03284b69f09 Mon Sep 17 00:00:00 2001 From: Alex Shkurenko Date: Thu, 20 Aug 2015 10:16:38 -0700 Subject: [PATCH 030/232] [SPARK-9982] [SPARKR] SparkR DataFrame fail to return data of Decimal type Author: Alex Shkurenko Closes #8239 from ashkurenko/master. --- core/src/main/scala/org/apache/spark/api/r/SerDe.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index d5b4260bf4529..3c89f24473744 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -181,6 +181,7 @@ private[spark] object SerDe { // Boolean -> logical // Float -> double // Double -> double + // Decimal -> double // Long -> double // Array[Byte] -> raw // Date -> Date @@ -219,6 +220,10 @@ private[spark] object SerDe { case "float" | "java.lang.Float" => writeType(dos, "double") writeDouble(dos, value.asInstanceOf[Float].toDouble) + case "decimal" | "java.math.BigDecimal" => + writeType(dos, "double") + val javaDecimal = value.asInstanceOf[java.math.BigDecimal] + writeDouble(dos, scala.math.BigDecimal(javaDecimal).toDouble) case "double" | "java.lang.Double" => writeType(dos, "double") writeDouble(dos, value.asInstanceOf[Double]) From 85f9a61357994da5023b08b0a8a2eb09388ce7f8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 20 Aug 2015 11:00:24 -0700 Subject: [PATCH 031/232] [SPARK-10136] [SQL] Fixes Parquet support for Avro array of primitive array I caught SPARK-10136 while adding more test cases to `ParquetAvroCompatibilitySuite`. Actual bug fix code lies in `CatalystRowConverter.scala`. Author: Cheng Lian Closes #8341 from liancheng/spark-10136/parquet-avro-nested-primitive-array. --- .../parquet/CatalystReadSupport.scala | 1 - .../parquet/CatalystRowConverter.scala | 24 +- sql/core/src/test/avro/parquet-compat.avdl | 19 +- sql/core/src/test/avro/parquet-compat.avpr | 54 +- .../parquet/test/avro/AvroArrayOfArray.java | 142 +++ .../parquet/test/avro/AvroMapOfArray.java | 142 +++ .../test/avro/AvroNonNullableArrays.java | 196 +++++ .../test/avro/AvroOptionalPrimitives.java | 466 ++++++++++ .../parquet/test/avro/AvroPrimitives.java | 461 ++++++++++ .../parquet/test/avro/CompatibilityTest.java | 2 +- .../parquet/test/avro/ParquetAvroCompat.java | 821 +----------------- .../ParquetAvroCompatibilitySuite.scala | 227 +++-- .../parquet/ParquetCompatibilityTest.scala | 7 + 13 files changed, 1718 insertions(+), 844 deletions(-) create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroOptionalPrimitives.java create mode 100644 sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroPrimitives.java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index a4679bb2f6389..3f8353af6e2ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -61,7 +61,6 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with | |Parquet form: |$parquetRequestedSchema - | |Catalyst form: |$catalystRequestedSchema """.stripMargin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index 18c5b500209e6..d2c2db51769ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -25,11 +25,12 @@ import scala.collection.mutable.ArrayBuffer import org.apache.parquet.column.Dictionary import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} -import org.apache.parquet.schema.OriginalType.{LIST, INT_32, UTF8} +import org.apache.parquet.schema.OriginalType.{INT_32, LIST, UTF8} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE import org.apache.parquet.schema.Type.Repetition import org.apache.parquet.schema.{GroupType, MessageType, PrimitiveType, Type} +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -145,7 +146,16 @@ private[parquet] class CatalystRowConverter( parquetType: GroupType, catalystType: StructType, updater: ParentContainerUpdater) - extends CatalystGroupConverter(updater) { + extends CatalystGroupConverter(updater) with Logging { + + logDebug( + s"""Building row converter for the following schema: + | + |Parquet form: + |$parquetType + |Catalyst form: + |${catalystType.prettyJson} + """.stripMargin) /** * Updater used together with field converters within a [[CatalystRowConverter]]. It propagates @@ -464,9 +474,15 @@ private[parquet] class CatalystRowConverter( override def getConverter(fieldIndex: Int): Converter = converter - override def end(): Unit = currentArray += currentElement + override def end(): Unit = { + converter.updater.end() + currentArray += currentElement + } - override def start(): Unit = currentElement = null + override def start(): Unit = { + converter.updater.start() + currentElement = null + } } } diff --git a/sql/core/src/test/avro/parquet-compat.avdl b/sql/core/src/test/avro/parquet-compat.avdl index 8070d0a9170a3..c5eb5b5164cf4 100644 --- a/sql/core/src/test/avro/parquet-compat.avdl +++ b/sql/core/src/test/avro/parquet-compat.avdl @@ -34,7 +34,7 @@ protocol CompatibilityTest { string nested_string_column; } - record ParquetAvroCompat { + record AvroPrimitives { boolean bool_column; int int_column; long long_column; @@ -42,7 +42,9 @@ protocol CompatibilityTest { double double_column; bytes binary_column; string string_column; + } + record AvroOptionalPrimitives { union { null, boolean } maybe_bool_column; union { null, int } maybe_int_column; union { null, long } maybe_long_column; @@ -50,7 +52,22 @@ protocol CompatibilityTest { union { null, double } maybe_double_column; union { null, bytes } maybe_binary_column; union { null, string } maybe_string_column; + } + + record AvroNonNullableArrays { + array strings_column; + union { null, array } maybe_ints_column; + } + record AvroArrayOfArray { + array> int_arrays_column; + } + + record AvroMapOfArray { + map> string_to_ints_column; + } + + record ParquetAvroCompat { array strings_column; map string_to_int_column; map> complex_column; diff --git a/sql/core/src/test/avro/parquet-compat.avpr b/sql/core/src/test/avro/parquet-compat.avpr index 060391765034b..9ad315b74fb41 100644 --- a/sql/core/src/test/avro/parquet-compat.avpr +++ b/sql/core/src/test/avro/parquet-compat.avpr @@ -27,7 +27,7 @@ } ] }, { "type" : "record", - "name" : "ParquetAvroCompat", + "name" : "AvroPrimitives", "fields" : [ { "name" : "bool_column", "type" : "boolean" @@ -49,7 +49,11 @@ }, { "name" : "string_column", "type" : "string" - }, { + } ] + }, { + "type" : "record", + "name" : "AvroOptionalPrimitives", + "fields" : [ { "name" : "maybe_bool_column", "type" : [ "null", "boolean" ] }, { @@ -70,7 +74,53 @@ }, { "name" : "maybe_string_column", "type" : [ "null", "string" ] + } ] + }, { + "type" : "record", + "name" : "AvroNonNullableArrays", + "fields" : [ { + "name" : "strings_column", + "type" : { + "type" : "array", + "items" : "string" + } }, { + "name" : "maybe_ints_column", + "type" : [ "null", { + "type" : "array", + "items" : "int" + } ] + } ] + }, { + "type" : "record", + "name" : "AvroArrayOfArray", + "fields" : [ { + "name" : "int_arrays_column", + "type" : { + "type" : "array", + "items" : { + "type" : "array", + "items" : "int" + } + } + } ] + }, { + "type" : "record", + "name" : "AvroMapOfArray", + "fields" : [ { + "name" : "string_to_ints_column", + "type" : { + "type" : "map", + "values" : { + "type" : "array", + "items" : "int" + } + } + } ] + }, { + "type" : "record", + "name" : "ParquetAvroCompat", + "fields" : [ { "name" : "strings_column", "type" : { "type" : "array", diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java new file mode 100644 index 0000000000000..ee327827903e5 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroArrayOfArray.java @@ -0,0 +1,142 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.execution.datasources.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class AvroArrayOfArray extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroArrayOfArray\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"int_arrays_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"array\",\"items\":\"int\"}}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.util.List> int_arrays_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroArrayOfArray() {} + + /** + * All-args constructor. + */ + public AvroArrayOfArray(java.util.List> int_arrays_column) { + this.int_arrays_column = int_arrays_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return int_arrays_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: int_arrays_column = (java.util.List>)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'int_arrays_column' field. + */ + public java.util.List> getIntArraysColumn() { + return int_arrays_column; + } + + /** + * Sets the value of the 'int_arrays_column' field. + * @param value the value to set. + */ + public void setIntArraysColumn(java.util.List> value) { + this.int_arrays_column = value; + } + + /** Creates a new AvroArrayOfArray RecordBuilder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder newBuilder() { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder(); + } + + /** Creates a new AvroArrayOfArray RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder(other); + } + + /** Creates a new AvroArrayOfArray RecordBuilder by copying an existing AvroArrayOfArray instance */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder(other); + } + + /** + * RecordBuilder for AvroArrayOfArray instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.util.List> int_arrays_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder other) { + super(other); + if (isValidValue(fields()[0], other.int_arrays_column)) { + this.int_arrays_column = data().deepCopy(fields()[0].schema(), other.int_arrays_column); + fieldSetFlags()[0] = true; + } + } + + /** Creates a Builder by copying an existing AvroArrayOfArray instance */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray other) { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.SCHEMA$); + if (isValidValue(fields()[0], other.int_arrays_column)) { + this.int_arrays_column = data().deepCopy(fields()[0].schema(), other.int_arrays_column); + fieldSetFlags()[0] = true; + } + } + + /** Gets the value of the 'int_arrays_column' field */ + public java.util.List> getIntArraysColumn() { + return int_arrays_column; + } + + /** Sets the value of the 'int_arrays_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder setIntArraysColumn(java.util.List> value) { + validate(fields()[0], value); + this.int_arrays_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'int_arrays_column' field has been set */ + public boolean hasIntArraysColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'int_arrays_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroArrayOfArray.Builder clearIntArraysColumn() { + int_arrays_column = null; + fieldSetFlags()[0] = false; + return this; + } + + @Override + public AvroArrayOfArray build() { + try { + AvroArrayOfArray record = new AvroArrayOfArray(); + record.int_arrays_column = fieldSetFlags()[0] ? this.int_arrays_column : (java.util.List>) defaultValue(fields()[0]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java new file mode 100644 index 0000000000000..727f6a7bf733e --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroMapOfArray.java @@ -0,0 +1,142 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.execution.datasources.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class AvroMapOfArray extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroMapOfArray\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"string_to_ints_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"int\"},\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.util.Map> string_to_ints_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroMapOfArray() {} + + /** + * All-args constructor. + */ + public AvroMapOfArray(java.util.Map> string_to_ints_column) { + this.string_to_ints_column = string_to_ints_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return string_to_ints_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: string_to_ints_column = (java.util.Map>)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'string_to_ints_column' field. + */ + public java.util.Map> getStringToIntsColumn() { + return string_to_ints_column; + } + + /** + * Sets the value of the 'string_to_ints_column' field. + * @param value the value to set. + */ + public void setStringToIntsColumn(java.util.Map> value) { + this.string_to_ints_column = value; + } + + /** Creates a new AvroMapOfArray RecordBuilder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder newBuilder() { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder(); + } + + /** Creates a new AvroMapOfArray RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder(other); + } + + /** Creates a new AvroMapOfArray RecordBuilder by copying an existing AvroMapOfArray instance */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder(other); + } + + /** + * RecordBuilder for AvroMapOfArray instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.util.Map> string_to_ints_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder other) { + super(other); + if (isValidValue(fields()[0], other.string_to_ints_column)) { + this.string_to_ints_column = data().deepCopy(fields()[0].schema(), other.string_to_ints_column); + fieldSetFlags()[0] = true; + } + } + + /** Creates a Builder by copying an existing AvroMapOfArray instance */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray other) { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.SCHEMA$); + if (isValidValue(fields()[0], other.string_to_ints_column)) { + this.string_to_ints_column = data().deepCopy(fields()[0].schema(), other.string_to_ints_column); + fieldSetFlags()[0] = true; + } + } + + /** Gets the value of the 'string_to_ints_column' field */ + public java.util.Map> getStringToIntsColumn() { + return string_to_ints_column; + } + + /** Sets the value of the 'string_to_ints_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder setStringToIntsColumn(java.util.Map> value) { + validate(fields()[0], value); + this.string_to_ints_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'string_to_ints_column' field has been set */ + public boolean hasStringToIntsColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'string_to_ints_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroMapOfArray.Builder clearStringToIntsColumn() { + string_to_ints_column = null; + fieldSetFlags()[0] = false; + return this; + } + + @Override + public AvroMapOfArray build() { + try { + AvroMapOfArray record = new AvroMapOfArray(); + record.string_to_ints_column = fieldSetFlags()[0] ? this.string_to_ints_column : (java.util.Map>) defaultValue(fields()[0]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java new file mode 100644 index 0000000000000..934793f42f9c9 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroNonNullableArrays.java @@ -0,0 +1,196 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.execution.datasources.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class AvroNonNullableArrays extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroNonNullableArrays\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"maybe_ints_column\",\"type\":[\"null\",{\"type\":\"array\",\"items\":\"int\"}]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.util.List strings_column; + @Deprecated public java.util.List maybe_ints_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroNonNullableArrays() {} + + /** + * All-args constructor. + */ + public AvroNonNullableArrays(java.util.List strings_column, java.util.List maybe_ints_column) { + this.strings_column = strings_column; + this.maybe_ints_column = maybe_ints_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return strings_column; + case 1: return maybe_ints_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: strings_column = (java.util.List)value$; break; + case 1: maybe_ints_column = (java.util.List)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'strings_column' field. + */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** + * Sets the value of the 'strings_column' field. + * @param value the value to set. + */ + public void setStringsColumn(java.util.List value) { + this.strings_column = value; + } + + /** + * Gets the value of the 'maybe_ints_column' field. + */ + public java.util.List getMaybeIntsColumn() { + return maybe_ints_column; + } + + /** + * Sets the value of the 'maybe_ints_column' field. + * @param value the value to set. + */ + public void setMaybeIntsColumn(java.util.List value) { + this.maybe_ints_column = value; + } + + /** Creates a new AvroNonNullableArrays RecordBuilder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder newBuilder() { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder(); + } + + /** Creates a new AvroNonNullableArrays RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder(other); + } + + /** Creates a new AvroNonNullableArrays RecordBuilder by copying an existing AvroNonNullableArrays instance */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder(other); + } + + /** + * RecordBuilder for AvroNonNullableArrays instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.util.List strings_column; + private java.util.List maybe_ints_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder other) { + super(other); + if (isValidValue(fields()[0], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[0].schema(), other.strings_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.maybe_ints_column)) { + this.maybe_ints_column = data().deepCopy(fields()[1].schema(), other.maybe_ints_column); + fieldSetFlags()[1] = true; + } + } + + /** Creates a Builder by copying an existing AvroNonNullableArrays instance */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays other) { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.SCHEMA$); + if (isValidValue(fields()[0], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[0].schema(), other.strings_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.maybe_ints_column)) { + this.maybe_ints_column = data().deepCopy(fields()[1].schema(), other.maybe_ints_column); + fieldSetFlags()[1] = true; + } + } + + /** Gets the value of the 'strings_column' field */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** Sets the value of the 'strings_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder setStringsColumn(java.util.List value) { + validate(fields()[0], value); + this.strings_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'strings_column' field has been set */ + public boolean hasStringsColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'strings_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder clearStringsColumn() { + strings_column = null; + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'maybe_ints_column' field */ + public java.util.List getMaybeIntsColumn() { + return maybe_ints_column; + } + + /** Sets the value of the 'maybe_ints_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder setMaybeIntsColumn(java.util.List value) { + validate(fields()[1], value); + this.maybe_ints_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'maybe_ints_column' field has been set */ + public boolean hasMaybeIntsColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'maybe_ints_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroNonNullableArrays.Builder clearMaybeIntsColumn() { + maybe_ints_column = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + public AvroNonNullableArrays build() { + try { + AvroNonNullableArrays record = new AvroNonNullableArrays(); + record.strings_column = fieldSetFlags()[0] ? this.strings_column : (java.util.List) defaultValue(fields()[0]); + record.maybe_ints_column = fieldSetFlags()[1] ? this.maybe_ints_column : (java.util.List) defaultValue(fields()[1]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroOptionalPrimitives.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroOptionalPrimitives.java new file mode 100644 index 0000000000000..e4d1ead8dd15f --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroOptionalPrimitives.java @@ -0,0 +1,466 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.execution.datasources.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class AvroOptionalPrimitives extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroOptionalPrimitives\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.lang.Boolean maybe_bool_column; + @Deprecated public java.lang.Integer maybe_int_column; + @Deprecated public java.lang.Long maybe_long_column; + @Deprecated public java.lang.Float maybe_float_column; + @Deprecated public java.lang.Double maybe_double_column; + @Deprecated public java.nio.ByteBuffer maybe_binary_column; + @Deprecated public java.lang.String maybe_string_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroOptionalPrimitives() {} + + /** + * All-args constructor. + */ + public AvroOptionalPrimitives(java.lang.Boolean maybe_bool_column, java.lang.Integer maybe_int_column, java.lang.Long maybe_long_column, java.lang.Float maybe_float_column, java.lang.Double maybe_double_column, java.nio.ByteBuffer maybe_binary_column, java.lang.String maybe_string_column) { + this.maybe_bool_column = maybe_bool_column; + this.maybe_int_column = maybe_int_column; + this.maybe_long_column = maybe_long_column; + this.maybe_float_column = maybe_float_column; + this.maybe_double_column = maybe_double_column; + this.maybe_binary_column = maybe_binary_column; + this.maybe_string_column = maybe_string_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return maybe_bool_column; + case 1: return maybe_int_column; + case 2: return maybe_long_column; + case 3: return maybe_float_column; + case 4: return maybe_double_column; + case 5: return maybe_binary_column; + case 6: return maybe_string_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: maybe_bool_column = (java.lang.Boolean)value$; break; + case 1: maybe_int_column = (java.lang.Integer)value$; break; + case 2: maybe_long_column = (java.lang.Long)value$; break; + case 3: maybe_float_column = (java.lang.Float)value$; break; + case 4: maybe_double_column = (java.lang.Double)value$; break; + case 5: maybe_binary_column = (java.nio.ByteBuffer)value$; break; + case 6: maybe_string_column = (java.lang.String)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'maybe_bool_column' field. + */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** + * Sets the value of the 'maybe_bool_column' field. + * @param value the value to set. + */ + public void setMaybeBoolColumn(java.lang.Boolean value) { + this.maybe_bool_column = value; + } + + /** + * Gets the value of the 'maybe_int_column' field. + */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** + * Sets the value of the 'maybe_int_column' field. + * @param value the value to set. + */ + public void setMaybeIntColumn(java.lang.Integer value) { + this.maybe_int_column = value; + } + + /** + * Gets the value of the 'maybe_long_column' field. + */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** + * Sets the value of the 'maybe_long_column' field. + * @param value the value to set. + */ + public void setMaybeLongColumn(java.lang.Long value) { + this.maybe_long_column = value; + } + + /** + * Gets the value of the 'maybe_float_column' field. + */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** + * Sets the value of the 'maybe_float_column' field. + * @param value the value to set. + */ + public void setMaybeFloatColumn(java.lang.Float value) { + this.maybe_float_column = value; + } + + /** + * Gets the value of the 'maybe_double_column' field. + */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** + * Sets the value of the 'maybe_double_column' field. + * @param value the value to set. + */ + public void setMaybeDoubleColumn(java.lang.Double value) { + this.maybe_double_column = value; + } + + /** + * Gets the value of the 'maybe_binary_column' field. + */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** + * Sets the value of the 'maybe_binary_column' field. + * @param value the value to set. + */ + public void setMaybeBinaryColumn(java.nio.ByteBuffer value) { + this.maybe_binary_column = value; + } + + /** + * Gets the value of the 'maybe_string_column' field. + */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** + * Sets the value of the 'maybe_string_column' field. + * @param value the value to set. + */ + public void setMaybeStringColumn(java.lang.String value) { + this.maybe_string_column = value; + } + + /** Creates a new AvroOptionalPrimitives RecordBuilder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder newBuilder() { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder(); + } + + /** Creates a new AvroOptionalPrimitives RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder(other); + } + + /** Creates a new AvroOptionalPrimitives RecordBuilder by copying an existing AvroOptionalPrimitives instance */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder(other); + } + + /** + * RecordBuilder for AvroOptionalPrimitives instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.lang.Boolean maybe_bool_column; + private java.lang.Integer maybe_int_column; + private java.lang.Long maybe_long_column; + private java.lang.Float maybe_float_column; + private java.lang.Double maybe_double_column; + private java.nio.ByteBuffer maybe_binary_column; + private java.lang.String maybe_string_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder other) { + super(other); + if (isValidValue(fields()[0], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[0].schema(), other.maybe_bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[1].schema(), other.maybe_int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[2].schema(), other.maybe_long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[3].schema(), other.maybe_float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[4].schema(), other.maybe_double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[5].schema(), other.maybe_binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[6].schema(), other.maybe_string_column); + fieldSetFlags()[6] = true; + } + } + + /** Creates a Builder by copying an existing AvroOptionalPrimitives instance */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives other) { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.SCHEMA$); + if (isValidValue(fields()[0], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[0].schema(), other.maybe_bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[1].schema(), other.maybe_int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[2].schema(), other.maybe_long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[3].schema(), other.maybe_float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[4].schema(), other.maybe_double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[5].schema(), other.maybe_binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[6].schema(), other.maybe_string_column); + fieldSetFlags()[6] = true; + } + } + + /** Gets the value of the 'maybe_bool_column' field */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** Sets the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeBoolColumn(java.lang.Boolean value) { + validate(fields()[0], value); + this.maybe_bool_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'maybe_bool_column' field has been set */ + public boolean hasMaybeBoolColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeBoolColumn() { + maybe_bool_column = null; + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'maybe_int_column' field */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** Sets the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeIntColumn(java.lang.Integer value) { + validate(fields()[1], value); + this.maybe_int_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'maybe_int_column' field has been set */ + public boolean hasMaybeIntColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeIntColumn() { + maybe_int_column = null; + fieldSetFlags()[1] = false; + return this; + } + + /** Gets the value of the 'maybe_long_column' field */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** Sets the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeLongColumn(java.lang.Long value) { + validate(fields()[2], value); + this.maybe_long_column = value; + fieldSetFlags()[2] = true; + return this; + } + + /** Checks whether the 'maybe_long_column' field has been set */ + public boolean hasMaybeLongColumn() { + return fieldSetFlags()[2]; + } + + /** Clears the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeLongColumn() { + maybe_long_column = null; + fieldSetFlags()[2] = false; + return this; + } + + /** Gets the value of the 'maybe_float_column' field */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** Sets the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeFloatColumn(java.lang.Float value) { + validate(fields()[3], value); + this.maybe_float_column = value; + fieldSetFlags()[3] = true; + return this; + } + + /** Checks whether the 'maybe_float_column' field has been set */ + public boolean hasMaybeFloatColumn() { + return fieldSetFlags()[3]; + } + + /** Clears the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeFloatColumn() { + maybe_float_column = null; + fieldSetFlags()[3] = false; + return this; + } + + /** Gets the value of the 'maybe_double_column' field */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** Sets the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeDoubleColumn(java.lang.Double value) { + validate(fields()[4], value); + this.maybe_double_column = value; + fieldSetFlags()[4] = true; + return this; + } + + /** Checks whether the 'maybe_double_column' field has been set */ + public boolean hasMaybeDoubleColumn() { + return fieldSetFlags()[4]; + } + + /** Clears the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeDoubleColumn() { + maybe_double_column = null; + fieldSetFlags()[4] = false; + return this; + } + + /** Gets the value of the 'maybe_binary_column' field */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** Sets the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[5], value); + this.maybe_binary_column = value; + fieldSetFlags()[5] = true; + return this; + } + + /** Checks whether the 'maybe_binary_column' field has been set */ + public boolean hasMaybeBinaryColumn() { + return fieldSetFlags()[5]; + } + + /** Clears the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeBinaryColumn() { + maybe_binary_column = null; + fieldSetFlags()[5] = false; + return this; + } + + /** Gets the value of the 'maybe_string_column' field */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** Sets the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder setMaybeStringColumn(java.lang.String value) { + validate(fields()[6], value); + this.maybe_string_column = value; + fieldSetFlags()[6] = true; + return this; + } + + /** Checks whether the 'maybe_string_column' field has been set */ + public boolean hasMaybeStringColumn() { + return fieldSetFlags()[6]; + } + + /** Clears the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroOptionalPrimitives.Builder clearMaybeStringColumn() { + maybe_string_column = null; + fieldSetFlags()[6] = false; + return this; + } + + @Override + public AvroOptionalPrimitives build() { + try { + AvroOptionalPrimitives record = new AvroOptionalPrimitives(); + record.maybe_bool_column = fieldSetFlags()[0] ? this.maybe_bool_column : (java.lang.Boolean) defaultValue(fields()[0]); + record.maybe_int_column = fieldSetFlags()[1] ? this.maybe_int_column : (java.lang.Integer) defaultValue(fields()[1]); + record.maybe_long_column = fieldSetFlags()[2] ? this.maybe_long_column : (java.lang.Long) defaultValue(fields()[2]); + record.maybe_float_column = fieldSetFlags()[3] ? this.maybe_float_column : (java.lang.Float) defaultValue(fields()[3]); + record.maybe_double_column = fieldSetFlags()[4] ? this.maybe_double_column : (java.lang.Double) defaultValue(fields()[4]); + record.maybe_binary_column = fieldSetFlags()[5] ? this.maybe_binary_column : (java.nio.ByteBuffer) defaultValue(fields()[5]); + record.maybe_string_column = fieldSetFlags()[6] ? this.maybe_string_column : (java.lang.String) defaultValue(fields()[6]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroPrimitives.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroPrimitives.java new file mode 100644 index 0000000000000..1c2afed16781e --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/AvroPrimitives.java @@ -0,0 +1,461 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.execution.datasources.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class AvroPrimitives extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"AvroPrimitives\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public boolean bool_column; + @Deprecated public int int_column; + @Deprecated public long long_column; + @Deprecated public float float_column; + @Deprecated public double double_column; + @Deprecated public java.nio.ByteBuffer binary_column; + @Deprecated public java.lang.String string_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public AvroPrimitives() {} + + /** + * All-args constructor. + */ + public AvroPrimitives(java.lang.Boolean bool_column, java.lang.Integer int_column, java.lang.Long long_column, java.lang.Float float_column, java.lang.Double double_column, java.nio.ByteBuffer binary_column, java.lang.String string_column) { + this.bool_column = bool_column; + this.int_column = int_column; + this.long_column = long_column; + this.float_column = float_column; + this.double_column = double_column; + this.binary_column = binary_column; + this.string_column = string_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return bool_column; + case 1: return int_column; + case 2: return long_column; + case 3: return float_column; + case 4: return double_column; + case 5: return binary_column; + case 6: return string_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: bool_column = (java.lang.Boolean)value$; break; + case 1: int_column = (java.lang.Integer)value$; break; + case 2: long_column = (java.lang.Long)value$; break; + case 3: float_column = (java.lang.Float)value$; break; + case 4: double_column = (java.lang.Double)value$; break; + case 5: binary_column = (java.nio.ByteBuffer)value$; break; + case 6: string_column = (java.lang.String)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'bool_column' field. + */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** + * Sets the value of the 'bool_column' field. + * @param value the value to set. + */ + public void setBoolColumn(java.lang.Boolean value) { + this.bool_column = value; + } + + /** + * Gets the value of the 'int_column' field. + */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** + * Sets the value of the 'int_column' field. + * @param value the value to set. + */ + public void setIntColumn(java.lang.Integer value) { + this.int_column = value; + } + + /** + * Gets the value of the 'long_column' field. + */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** + * Sets the value of the 'long_column' field. + * @param value the value to set. + */ + public void setLongColumn(java.lang.Long value) { + this.long_column = value; + } + + /** + * Gets the value of the 'float_column' field. + */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** + * Sets the value of the 'float_column' field. + * @param value the value to set. + */ + public void setFloatColumn(java.lang.Float value) { + this.float_column = value; + } + + /** + * Gets the value of the 'double_column' field. + */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** + * Sets the value of the 'double_column' field. + * @param value the value to set. + */ + public void setDoubleColumn(java.lang.Double value) { + this.double_column = value; + } + + /** + * Gets the value of the 'binary_column' field. + */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** + * Sets the value of the 'binary_column' field. + * @param value the value to set. + */ + public void setBinaryColumn(java.nio.ByteBuffer value) { + this.binary_column = value; + } + + /** + * Gets the value of the 'string_column' field. + */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** + * Sets the value of the 'string_column' field. + * @param value the value to set. + */ + public void setStringColumn(java.lang.String value) { + this.string_column = value; + } + + /** Creates a new AvroPrimitives RecordBuilder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder newBuilder() { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder(); + } + + /** Creates a new AvroPrimitives RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder(other); + } + + /** Creates a new AvroPrimitives RecordBuilder by copying an existing AvroPrimitives instance */ + public static org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder newBuilder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives other) { + return new org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder(other); + } + + /** + * RecordBuilder for AvroPrimitives instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_column; + private int int_column; + private long long_column; + private float float_column; + private double double_column; + private java.nio.ByteBuffer binary_column; + private java.lang.String string_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + } + + /** Creates a Builder by copying an existing AvroPrimitives instance */ + private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives other) { + super(org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.SCHEMA$); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + } + + /** Gets the value of the 'bool_column' field */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** Sets the value of the 'bool_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setBoolColumn(boolean value) { + validate(fields()[0], value); + this.bool_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'bool_column' field has been set */ + public boolean hasBoolColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'bool_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearBoolColumn() { + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'int_column' field */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** Sets the value of the 'int_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setIntColumn(int value) { + validate(fields()[1], value); + this.int_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'int_column' field has been set */ + public boolean hasIntColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'int_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearIntColumn() { + fieldSetFlags()[1] = false; + return this; + } + + /** Gets the value of the 'long_column' field */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** Sets the value of the 'long_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setLongColumn(long value) { + validate(fields()[2], value); + this.long_column = value; + fieldSetFlags()[2] = true; + return this; + } + + /** Checks whether the 'long_column' field has been set */ + public boolean hasLongColumn() { + return fieldSetFlags()[2]; + } + + /** Clears the value of the 'long_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearLongColumn() { + fieldSetFlags()[2] = false; + return this; + } + + /** Gets the value of the 'float_column' field */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** Sets the value of the 'float_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setFloatColumn(float value) { + validate(fields()[3], value); + this.float_column = value; + fieldSetFlags()[3] = true; + return this; + } + + /** Checks whether the 'float_column' field has been set */ + public boolean hasFloatColumn() { + return fieldSetFlags()[3]; + } + + /** Clears the value of the 'float_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearFloatColumn() { + fieldSetFlags()[3] = false; + return this; + } + + /** Gets the value of the 'double_column' field */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** Sets the value of the 'double_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setDoubleColumn(double value) { + validate(fields()[4], value); + this.double_column = value; + fieldSetFlags()[4] = true; + return this; + } + + /** Checks whether the 'double_column' field has been set */ + public boolean hasDoubleColumn() { + return fieldSetFlags()[4]; + } + + /** Clears the value of the 'double_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearDoubleColumn() { + fieldSetFlags()[4] = false; + return this; + } + + /** Gets the value of the 'binary_column' field */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** Sets the value of the 'binary_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[5], value); + this.binary_column = value; + fieldSetFlags()[5] = true; + return this; + } + + /** Checks whether the 'binary_column' field has been set */ + public boolean hasBinaryColumn() { + return fieldSetFlags()[5]; + } + + /** Clears the value of the 'binary_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearBinaryColumn() { + binary_column = null; + fieldSetFlags()[5] = false; + return this; + } + + /** Gets the value of the 'string_column' field */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** Sets the value of the 'string_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder setStringColumn(java.lang.String value) { + validate(fields()[6], value); + this.string_column = value; + fieldSetFlags()[6] = true; + return this; + } + + /** Checks whether the 'string_column' field has been set */ + public boolean hasStringColumn() { + return fieldSetFlags()[6]; + } + + /** Clears the value of the 'string_column' field */ + public org.apache.spark.sql.execution.datasources.parquet.test.avro.AvroPrimitives.Builder clearStringColumn() { + string_column = null; + fieldSetFlags()[6] = false; + return this; + } + + @Override + public AvroPrimitives build() { + try { + AvroPrimitives record = new AvroPrimitives(); + record.bool_column = fieldSetFlags()[0] ? this.bool_column : (java.lang.Boolean) defaultValue(fields()[0]); + record.int_column = fieldSetFlags()[1] ? this.int_column : (java.lang.Integer) defaultValue(fields()[1]); + record.long_column = fieldSetFlags()[2] ? this.long_column : (java.lang.Long) defaultValue(fields()[2]); + record.float_column = fieldSetFlags()[3] ? this.float_column : (java.lang.Float) defaultValue(fields()[3]); + record.double_column = fieldSetFlags()[4] ? this.double_column : (java.lang.Double) defaultValue(fields()[4]); + record.binary_column = fieldSetFlags()[5] ? this.binary_column : (java.nio.ByteBuffer) defaultValue(fields()[5]); + record.string_column = fieldSetFlags()[6] ? this.string_column : (java.lang.String) defaultValue(fields()[6]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/CompatibilityTest.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/CompatibilityTest.java index 2368323cb36b9..28fdc1dfb911c 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/CompatibilityTest.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/CompatibilityTest.java @@ -8,7 +8,7 @@ @SuppressWarnings("all") @org.apache.avro.specific.AvroGenerated public interface CompatibilityTest { - public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("{\"protocol\":\"CompatibilityTest\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"types\":[{\"type\":\"enum\",\"name\":\"Suit\",\"symbols\":[\"SPADES\",\"HEARTS\",\"DIAMONDS\",\"CLUBS\"]},{\"type\":\"record\",\"name\":\"ParquetEnum\",\"fields\":[{\"name\":\"suit\",\"type\":\"Suit\"}]},{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"Nested\"},\"avro.java.string\":\"String\"}}]}],\"messages\":{}}"); + public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("{\"protocol\":\"CompatibilityTest\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"types\":[{\"type\":\"enum\",\"name\":\"Suit\",\"symbols\":[\"SPADES\",\"HEARTS\",\"DIAMONDS\",\"CLUBS\"]},{\"type\":\"record\",\"name\":\"ParquetEnum\",\"fields\":[{\"name\":\"suit\",\"type\":\"Suit\"}]},{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"AvroPrimitives\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"AvroOptionalPrimitives\",\"fields\":[{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]}]},{\"type\":\"record\",\"name\":\"AvroNonNullableArrays\",\"fields\":[{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"maybe_ints_column\",\"type\":[\"null\",{\"type\":\"array\",\"items\":\"int\"}]}]},{\"type\":\"record\",\"name\":\"AvroArrayOfArray\",\"fields\":[{\"name\":\"int_arrays_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"array\",\"items\":\"int\"}}}]},{\"type\":\"record\",\"name\":\"AvroMapOfArray\",\"fields\":[{\"name\":\"string_to_ints_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"int\"},\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"fields\":[{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"Nested\"},\"avro.java.string\":\"String\"}}]}],\"messages\":{}}"); @SuppressWarnings("all") public interface Callback extends CompatibilityTest { diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java index 681cacbd12c7c..ef12d193f916c 100644 --- a/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/execution/datasources/parquet/test/avro/ParquetAvroCompat.java @@ -7,22 +7,8 @@ @SuppressWarnings("all") @org.apache.avro.specific.AvroGenerated public class ParquetAvroCompat extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}},\"avro.java.string\":\"String\"}}]}"); + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"namespace\":\"org.apache.spark.sql.execution.datasources.parquet.test.avro\",\"fields\":[{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}},\"avro.java.string\":\"String\"}}]}"); public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } - @Deprecated public boolean bool_column; - @Deprecated public int int_column; - @Deprecated public long long_column; - @Deprecated public float float_column; - @Deprecated public double double_column; - @Deprecated public java.nio.ByteBuffer binary_column; - @Deprecated public java.lang.String string_column; - @Deprecated public java.lang.Boolean maybe_bool_column; - @Deprecated public java.lang.Integer maybe_int_column; - @Deprecated public java.lang.Long maybe_long_column; - @Deprecated public java.lang.Float maybe_float_column; - @Deprecated public java.lang.Double maybe_double_column; - @Deprecated public java.nio.ByteBuffer maybe_binary_column; - @Deprecated public java.lang.String maybe_string_column; @Deprecated public java.util.List strings_column; @Deprecated public java.util.Map string_to_int_column; @Deprecated public java.util.Map> complex_column; @@ -37,21 +23,7 @@ public ParquetAvroCompat() {} /** * All-args constructor. */ - public ParquetAvroCompat(java.lang.Boolean bool_column, java.lang.Integer int_column, java.lang.Long long_column, java.lang.Float float_column, java.lang.Double double_column, java.nio.ByteBuffer binary_column, java.lang.String string_column, java.lang.Boolean maybe_bool_column, java.lang.Integer maybe_int_column, java.lang.Long maybe_long_column, java.lang.Float maybe_float_column, java.lang.Double maybe_double_column, java.nio.ByteBuffer maybe_binary_column, java.lang.String maybe_string_column, java.util.List strings_column, java.util.Map string_to_int_column, java.util.Map> complex_column) { - this.bool_column = bool_column; - this.int_column = int_column; - this.long_column = long_column; - this.float_column = float_column; - this.double_column = double_column; - this.binary_column = binary_column; - this.string_column = string_column; - this.maybe_bool_column = maybe_bool_column; - this.maybe_int_column = maybe_int_column; - this.maybe_long_column = maybe_long_column; - this.maybe_float_column = maybe_float_column; - this.maybe_double_column = maybe_double_column; - this.maybe_binary_column = maybe_binary_column; - this.maybe_string_column = maybe_string_column; + public ParquetAvroCompat(java.util.List strings_column, java.util.Map string_to_int_column, java.util.Map> complex_column) { this.strings_column = strings_column; this.string_to_int_column = string_to_int_column; this.complex_column = complex_column; @@ -61,23 +33,9 @@ public ParquetAvroCompat(java.lang.Boolean bool_column, java.lang.Integer int_co // Used by DatumWriter. Applications should not call. public java.lang.Object get(int field$) { switch (field$) { - case 0: return bool_column; - case 1: return int_column; - case 2: return long_column; - case 3: return float_column; - case 4: return double_column; - case 5: return binary_column; - case 6: return string_column; - case 7: return maybe_bool_column; - case 8: return maybe_int_column; - case 9: return maybe_long_column; - case 10: return maybe_float_column; - case 11: return maybe_double_column; - case 12: return maybe_binary_column; - case 13: return maybe_string_column; - case 14: return strings_column; - case 15: return string_to_int_column; - case 16: return complex_column; + case 0: return strings_column; + case 1: return string_to_int_column; + case 2: return complex_column; default: throw new org.apache.avro.AvroRuntimeException("Bad index"); } } @@ -85,237 +43,13 @@ public java.lang.Object get(int field$) { @SuppressWarnings(value="unchecked") public void put(int field$, java.lang.Object value$) { switch (field$) { - case 0: bool_column = (java.lang.Boolean)value$; break; - case 1: int_column = (java.lang.Integer)value$; break; - case 2: long_column = (java.lang.Long)value$; break; - case 3: float_column = (java.lang.Float)value$; break; - case 4: double_column = (java.lang.Double)value$; break; - case 5: binary_column = (java.nio.ByteBuffer)value$; break; - case 6: string_column = (java.lang.String)value$; break; - case 7: maybe_bool_column = (java.lang.Boolean)value$; break; - case 8: maybe_int_column = (java.lang.Integer)value$; break; - case 9: maybe_long_column = (java.lang.Long)value$; break; - case 10: maybe_float_column = (java.lang.Float)value$; break; - case 11: maybe_double_column = (java.lang.Double)value$; break; - case 12: maybe_binary_column = (java.nio.ByteBuffer)value$; break; - case 13: maybe_string_column = (java.lang.String)value$; break; - case 14: strings_column = (java.util.List)value$; break; - case 15: string_to_int_column = (java.util.Map)value$; break; - case 16: complex_column = (java.util.Map>)value$; break; + case 0: strings_column = (java.util.List)value$; break; + case 1: string_to_int_column = (java.util.Map)value$; break; + case 2: complex_column = (java.util.Map>)value$; break; default: throw new org.apache.avro.AvroRuntimeException("Bad index"); } } - /** - * Gets the value of the 'bool_column' field. - */ - public java.lang.Boolean getBoolColumn() { - return bool_column; - } - - /** - * Sets the value of the 'bool_column' field. - * @param value the value to set. - */ - public void setBoolColumn(java.lang.Boolean value) { - this.bool_column = value; - } - - /** - * Gets the value of the 'int_column' field. - */ - public java.lang.Integer getIntColumn() { - return int_column; - } - - /** - * Sets the value of the 'int_column' field. - * @param value the value to set. - */ - public void setIntColumn(java.lang.Integer value) { - this.int_column = value; - } - - /** - * Gets the value of the 'long_column' field. - */ - public java.lang.Long getLongColumn() { - return long_column; - } - - /** - * Sets the value of the 'long_column' field. - * @param value the value to set. - */ - public void setLongColumn(java.lang.Long value) { - this.long_column = value; - } - - /** - * Gets the value of the 'float_column' field. - */ - public java.lang.Float getFloatColumn() { - return float_column; - } - - /** - * Sets the value of the 'float_column' field. - * @param value the value to set. - */ - public void setFloatColumn(java.lang.Float value) { - this.float_column = value; - } - - /** - * Gets the value of the 'double_column' field. - */ - public java.lang.Double getDoubleColumn() { - return double_column; - } - - /** - * Sets the value of the 'double_column' field. - * @param value the value to set. - */ - public void setDoubleColumn(java.lang.Double value) { - this.double_column = value; - } - - /** - * Gets the value of the 'binary_column' field. - */ - public java.nio.ByteBuffer getBinaryColumn() { - return binary_column; - } - - /** - * Sets the value of the 'binary_column' field. - * @param value the value to set. - */ - public void setBinaryColumn(java.nio.ByteBuffer value) { - this.binary_column = value; - } - - /** - * Gets the value of the 'string_column' field. - */ - public java.lang.String getStringColumn() { - return string_column; - } - - /** - * Sets the value of the 'string_column' field. - * @param value the value to set. - */ - public void setStringColumn(java.lang.String value) { - this.string_column = value; - } - - /** - * Gets the value of the 'maybe_bool_column' field. - */ - public java.lang.Boolean getMaybeBoolColumn() { - return maybe_bool_column; - } - - /** - * Sets the value of the 'maybe_bool_column' field. - * @param value the value to set. - */ - public void setMaybeBoolColumn(java.lang.Boolean value) { - this.maybe_bool_column = value; - } - - /** - * Gets the value of the 'maybe_int_column' field. - */ - public java.lang.Integer getMaybeIntColumn() { - return maybe_int_column; - } - - /** - * Sets the value of the 'maybe_int_column' field. - * @param value the value to set. - */ - public void setMaybeIntColumn(java.lang.Integer value) { - this.maybe_int_column = value; - } - - /** - * Gets the value of the 'maybe_long_column' field. - */ - public java.lang.Long getMaybeLongColumn() { - return maybe_long_column; - } - - /** - * Sets the value of the 'maybe_long_column' field. - * @param value the value to set. - */ - public void setMaybeLongColumn(java.lang.Long value) { - this.maybe_long_column = value; - } - - /** - * Gets the value of the 'maybe_float_column' field. - */ - public java.lang.Float getMaybeFloatColumn() { - return maybe_float_column; - } - - /** - * Sets the value of the 'maybe_float_column' field. - * @param value the value to set. - */ - public void setMaybeFloatColumn(java.lang.Float value) { - this.maybe_float_column = value; - } - - /** - * Gets the value of the 'maybe_double_column' field. - */ - public java.lang.Double getMaybeDoubleColumn() { - return maybe_double_column; - } - - /** - * Sets the value of the 'maybe_double_column' field. - * @param value the value to set. - */ - public void setMaybeDoubleColumn(java.lang.Double value) { - this.maybe_double_column = value; - } - - /** - * Gets the value of the 'maybe_binary_column' field. - */ - public java.nio.ByteBuffer getMaybeBinaryColumn() { - return maybe_binary_column; - } - - /** - * Sets the value of the 'maybe_binary_column' field. - * @param value the value to set. - */ - public void setMaybeBinaryColumn(java.nio.ByteBuffer value) { - this.maybe_binary_column = value; - } - - /** - * Gets the value of the 'maybe_string_column' field. - */ - public java.lang.String getMaybeStringColumn() { - return maybe_string_column; - } - - /** - * Sets the value of the 'maybe_string_column' field. - * @param value the value to set. - */ - public void setMaybeStringColumn(java.lang.String value) { - this.maybe_string_column = value; - } - /** * Gets the value of the 'strings_column' field. */ @@ -382,20 +116,6 @@ public static org.apache.spark.sql.execution.datasources.parquet.test.avro.Parqu public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase implements org.apache.avro.data.RecordBuilder { - private boolean bool_column; - private int int_column; - private long long_column; - private float float_column; - private double double_column; - private java.nio.ByteBuffer binary_column; - private java.lang.String string_column; - private java.lang.Boolean maybe_bool_column; - private java.lang.Integer maybe_int_column; - private java.lang.Long maybe_long_column; - private java.lang.Float maybe_float_column; - private java.lang.Double maybe_double_column; - private java.nio.ByteBuffer maybe_binary_column; - private java.lang.String maybe_string_column; private java.util.List strings_column; private java.util.Map string_to_int_column; private java.util.Map> complex_column; @@ -408,492 +128,35 @@ private Builder() { /** Creates a Builder by copying an existing Builder */ private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder other) { super(other); - if (isValidValue(fields()[0], other.bool_column)) { - this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + if (isValidValue(fields()[0], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[0].schema(), other.strings_column); fieldSetFlags()[0] = true; } - if (isValidValue(fields()[1], other.int_column)) { - this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + if (isValidValue(fields()[1], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[1].schema(), other.string_to_int_column); fieldSetFlags()[1] = true; } - if (isValidValue(fields()[2], other.long_column)) { - this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + if (isValidValue(fields()[2], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[2].schema(), other.complex_column); fieldSetFlags()[2] = true; } - if (isValidValue(fields()[3], other.float_column)) { - this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); - fieldSetFlags()[3] = true; - } - if (isValidValue(fields()[4], other.double_column)) { - this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); - fieldSetFlags()[4] = true; - } - if (isValidValue(fields()[5], other.binary_column)) { - this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); - fieldSetFlags()[5] = true; - } - if (isValidValue(fields()[6], other.string_column)) { - this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); - fieldSetFlags()[6] = true; - } - if (isValidValue(fields()[7], other.maybe_bool_column)) { - this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); - fieldSetFlags()[7] = true; - } - if (isValidValue(fields()[8], other.maybe_int_column)) { - this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); - fieldSetFlags()[8] = true; - } - if (isValidValue(fields()[9], other.maybe_long_column)) { - this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); - fieldSetFlags()[9] = true; - } - if (isValidValue(fields()[10], other.maybe_float_column)) { - this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); - fieldSetFlags()[10] = true; - } - if (isValidValue(fields()[11], other.maybe_double_column)) { - this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); - fieldSetFlags()[11] = true; - } - if (isValidValue(fields()[12], other.maybe_binary_column)) { - this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); - fieldSetFlags()[12] = true; - } - if (isValidValue(fields()[13], other.maybe_string_column)) { - this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); - fieldSetFlags()[13] = true; - } - if (isValidValue(fields()[14], other.strings_column)) { - this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); - fieldSetFlags()[14] = true; - } - if (isValidValue(fields()[15], other.string_to_int_column)) { - this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); - fieldSetFlags()[15] = true; - } - if (isValidValue(fields()[16], other.complex_column)) { - this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); - fieldSetFlags()[16] = true; - } } /** Creates a Builder by copying an existing ParquetAvroCompat instance */ private Builder(org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat other) { super(org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.SCHEMA$); - if (isValidValue(fields()[0], other.bool_column)) { - this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + if (isValidValue(fields()[0], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[0].schema(), other.strings_column); fieldSetFlags()[0] = true; } - if (isValidValue(fields()[1], other.int_column)) { - this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + if (isValidValue(fields()[1], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[1].schema(), other.string_to_int_column); fieldSetFlags()[1] = true; } - if (isValidValue(fields()[2], other.long_column)) { - this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + if (isValidValue(fields()[2], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[2].schema(), other.complex_column); fieldSetFlags()[2] = true; } - if (isValidValue(fields()[3], other.float_column)) { - this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); - fieldSetFlags()[3] = true; - } - if (isValidValue(fields()[4], other.double_column)) { - this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); - fieldSetFlags()[4] = true; - } - if (isValidValue(fields()[5], other.binary_column)) { - this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); - fieldSetFlags()[5] = true; - } - if (isValidValue(fields()[6], other.string_column)) { - this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); - fieldSetFlags()[6] = true; - } - if (isValidValue(fields()[7], other.maybe_bool_column)) { - this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); - fieldSetFlags()[7] = true; - } - if (isValidValue(fields()[8], other.maybe_int_column)) { - this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); - fieldSetFlags()[8] = true; - } - if (isValidValue(fields()[9], other.maybe_long_column)) { - this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); - fieldSetFlags()[9] = true; - } - if (isValidValue(fields()[10], other.maybe_float_column)) { - this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); - fieldSetFlags()[10] = true; - } - if (isValidValue(fields()[11], other.maybe_double_column)) { - this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); - fieldSetFlags()[11] = true; - } - if (isValidValue(fields()[12], other.maybe_binary_column)) { - this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); - fieldSetFlags()[12] = true; - } - if (isValidValue(fields()[13], other.maybe_string_column)) { - this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); - fieldSetFlags()[13] = true; - } - if (isValidValue(fields()[14], other.strings_column)) { - this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); - fieldSetFlags()[14] = true; - } - if (isValidValue(fields()[15], other.string_to_int_column)) { - this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); - fieldSetFlags()[15] = true; - } - if (isValidValue(fields()[16], other.complex_column)) { - this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); - fieldSetFlags()[16] = true; - } - } - - /** Gets the value of the 'bool_column' field */ - public java.lang.Boolean getBoolColumn() { - return bool_column; - } - - /** Sets the value of the 'bool_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setBoolColumn(boolean value) { - validate(fields()[0], value); - this.bool_column = value; - fieldSetFlags()[0] = true; - return this; - } - - /** Checks whether the 'bool_column' field has been set */ - public boolean hasBoolColumn() { - return fieldSetFlags()[0]; - } - - /** Clears the value of the 'bool_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearBoolColumn() { - fieldSetFlags()[0] = false; - return this; - } - - /** Gets the value of the 'int_column' field */ - public java.lang.Integer getIntColumn() { - return int_column; - } - - /** Sets the value of the 'int_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setIntColumn(int value) { - validate(fields()[1], value); - this.int_column = value; - fieldSetFlags()[1] = true; - return this; - } - - /** Checks whether the 'int_column' field has been set */ - public boolean hasIntColumn() { - return fieldSetFlags()[1]; - } - - /** Clears the value of the 'int_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearIntColumn() { - fieldSetFlags()[1] = false; - return this; - } - - /** Gets the value of the 'long_column' field */ - public java.lang.Long getLongColumn() { - return long_column; - } - - /** Sets the value of the 'long_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setLongColumn(long value) { - validate(fields()[2], value); - this.long_column = value; - fieldSetFlags()[2] = true; - return this; - } - - /** Checks whether the 'long_column' field has been set */ - public boolean hasLongColumn() { - return fieldSetFlags()[2]; - } - - /** Clears the value of the 'long_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearLongColumn() { - fieldSetFlags()[2] = false; - return this; - } - - /** Gets the value of the 'float_column' field */ - public java.lang.Float getFloatColumn() { - return float_column; - } - - /** Sets the value of the 'float_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setFloatColumn(float value) { - validate(fields()[3], value); - this.float_column = value; - fieldSetFlags()[3] = true; - return this; - } - - /** Checks whether the 'float_column' field has been set */ - public boolean hasFloatColumn() { - return fieldSetFlags()[3]; - } - - /** Clears the value of the 'float_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearFloatColumn() { - fieldSetFlags()[3] = false; - return this; - } - - /** Gets the value of the 'double_column' field */ - public java.lang.Double getDoubleColumn() { - return double_column; - } - - /** Sets the value of the 'double_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setDoubleColumn(double value) { - validate(fields()[4], value); - this.double_column = value; - fieldSetFlags()[4] = true; - return this; - } - - /** Checks whether the 'double_column' field has been set */ - public boolean hasDoubleColumn() { - return fieldSetFlags()[4]; - } - - /** Clears the value of the 'double_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearDoubleColumn() { - fieldSetFlags()[4] = false; - return this; - } - - /** Gets the value of the 'binary_column' field */ - public java.nio.ByteBuffer getBinaryColumn() { - return binary_column; - } - - /** Sets the value of the 'binary_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setBinaryColumn(java.nio.ByteBuffer value) { - validate(fields()[5], value); - this.binary_column = value; - fieldSetFlags()[5] = true; - return this; - } - - /** Checks whether the 'binary_column' field has been set */ - public boolean hasBinaryColumn() { - return fieldSetFlags()[5]; - } - - /** Clears the value of the 'binary_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearBinaryColumn() { - binary_column = null; - fieldSetFlags()[5] = false; - return this; - } - - /** Gets the value of the 'string_column' field */ - public java.lang.String getStringColumn() { - return string_column; - } - - /** Sets the value of the 'string_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setStringColumn(java.lang.String value) { - validate(fields()[6], value); - this.string_column = value; - fieldSetFlags()[6] = true; - return this; - } - - /** Checks whether the 'string_column' field has been set */ - public boolean hasStringColumn() { - return fieldSetFlags()[6]; - } - - /** Clears the value of the 'string_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearStringColumn() { - string_column = null; - fieldSetFlags()[6] = false; - return this; - } - - /** Gets the value of the 'maybe_bool_column' field */ - public java.lang.Boolean getMaybeBoolColumn() { - return maybe_bool_column; - } - - /** Sets the value of the 'maybe_bool_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBoolColumn(java.lang.Boolean value) { - validate(fields()[7], value); - this.maybe_bool_column = value; - fieldSetFlags()[7] = true; - return this; - } - - /** Checks whether the 'maybe_bool_column' field has been set */ - public boolean hasMaybeBoolColumn() { - return fieldSetFlags()[7]; - } - - /** Clears the value of the 'maybe_bool_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBoolColumn() { - maybe_bool_column = null; - fieldSetFlags()[7] = false; - return this; - } - - /** Gets the value of the 'maybe_int_column' field */ - public java.lang.Integer getMaybeIntColumn() { - return maybe_int_column; - } - - /** Sets the value of the 'maybe_int_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeIntColumn(java.lang.Integer value) { - validate(fields()[8], value); - this.maybe_int_column = value; - fieldSetFlags()[8] = true; - return this; - } - - /** Checks whether the 'maybe_int_column' field has been set */ - public boolean hasMaybeIntColumn() { - return fieldSetFlags()[8]; - } - - /** Clears the value of the 'maybe_int_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeIntColumn() { - maybe_int_column = null; - fieldSetFlags()[8] = false; - return this; - } - - /** Gets the value of the 'maybe_long_column' field */ - public java.lang.Long getMaybeLongColumn() { - return maybe_long_column; - } - - /** Sets the value of the 'maybe_long_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeLongColumn(java.lang.Long value) { - validate(fields()[9], value); - this.maybe_long_column = value; - fieldSetFlags()[9] = true; - return this; - } - - /** Checks whether the 'maybe_long_column' field has been set */ - public boolean hasMaybeLongColumn() { - return fieldSetFlags()[9]; - } - - /** Clears the value of the 'maybe_long_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeLongColumn() { - maybe_long_column = null; - fieldSetFlags()[9] = false; - return this; - } - - /** Gets the value of the 'maybe_float_column' field */ - public java.lang.Float getMaybeFloatColumn() { - return maybe_float_column; - } - - /** Sets the value of the 'maybe_float_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeFloatColumn(java.lang.Float value) { - validate(fields()[10], value); - this.maybe_float_column = value; - fieldSetFlags()[10] = true; - return this; - } - - /** Checks whether the 'maybe_float_column' field has been set */ - public boolean hasMaybeFloatColumn() { - return fieldSetFlags()[10]; - } - - /** Clears the value of the 'maybe_float_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeFloatColumn() { - maybe_float_column = null; - fieldSetFlags()[10] = false; - return this; - } - - /** Gets the value of the 'maybe_double_column' field */ - public java.lang.Double getMaybeDoubleColumn() { - return maybe_double_column; - } - - /** Sets the value of the 'maybe_double_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeDoubleColumn(java.lang.Double value) { - validate(fields()[11], value); - this.maybe_double_column = value; - fieldSetFlags()[11] = true; - return this; - } - - /** Checks whether the 'maybe_double_column' field has been set */ - public boolean hasMaybeDoubleColumn() { - return fieldSetFlags()[11]; - } - - /** Clears the value of the 'maybe_double_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeDoubleColumn() { - maybe_double_column = null; - fieldSetFlags()[11] = false; - return this; - } - - /** Gets the value of the 'maybe_binary_column' field */ - public java.nio.ByteBuffer getMaybeBinaryColumn() { - return maybe_binary_column; - } - - /** Sets the value of the 'maybe_binary_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBinaryColumn(java.nio.ByteBuffer value) { - validate(fields()[12], value); - this.maybe_binary_column = value; - fieldSetFlags()[12] = true; - return this; - } - - /** Checks whether the 'maybe_binary_column' field has been set */ - public boolean hasMaybeBinaryColumn() { - return fieldSetFlags()[12]; - } - - /** Clears the value of the 'maybe_binary_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBinaryColumn() { - maybe_binary_column = null; - fieldSetFlags()[12] = false; - return this; - } - - /** Gets the value of the 'maybe_string_column' field */ - public java.lang.String getMaybeStringColumn() { - return maybe_string_column; - } - - /** Sets the value of the 'maybe_string_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setMaybeStringColumn(java.lang.String value) { - validate(fields()[13], value); - this.maybe_string_column = value; - fieldSetFlags()[13] = true; - return this; - } - - /** Checks whether the 'maybe_string_column' field has been set */ - public boolean hasMaybeStringColumn() { - return fieldSetFlags()[13]; - } - - /** Clears the value of the 'maybe_string_column' field */ - public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeStringColumn() { - maybe_string_column = null; - fieldSetFlags()[13] = false; - return this; } /** Gets the value of the 'strings_column' field */ @@ -903,21 +166,21 @@ public java.util.List getStringsColumn() { /** Sets the value of the 'strings_column' field */ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setStringsColumn(java.util.List value) { - validate(fields()[14], value); + validate(fields()[0], value); this.strings_column = value; - fieldSetFlags()[14] = true; + fieldSetFlags()[0] = true; return this; } /** Checks whether the 'strings_column' field has been set */ public boolean hasStringsColumn() { - return fieldSetFlags()[14]; + return fieldSetFlags()[0]; } /** Clears the value of the 'strings_column' field */ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearStringsColumn() { strings_column = null; - fieldSetFlags()[14] = false; + fieldSetFlags()[0] = false; return this; } @@ -928,21 +191,21 @@ public java.util.Map getStringToIntColumn() /** Sets the value of the 'string_to_int_column' field */ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder setStringToIntColumn(java.util.Map value) { - validate(fields()[15], value); + validate(fields()[1], value); this.string_to_int_column = value; - fieldSetFlags()[15] = true; + fieldSetFlags()[1] = true; return this; } /** Checks whether the 'string_to_int_column' field has been set */ public boolean hasStringToIntColumn() { - return fieldSetFlags()[15]; + return fieldSetFlags()[1]; } /** Clears the value of the 'string_to_int_column' field */ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearStringToIntColumn() { string_to_int_column = null; - fieldSetFlags()[15] = false; + fieldSetFlags()[1] = false; return this; } @@ -953,21 +216,21 @@ public java.util.Map> value) { - validate(fields()[16], value); + validate(fields()[2], value); this.complex_column = value; - fieldSetFlags()[16] = true; + fieldSetFlags()[2] = true; return this; } /** Checks whether the 'complex_column' field has been set */ public boolean hasComplexColumn() { - return fieldSetFlags()[16]; + return fieldSetFlags()[2]; } /** Clears the value of the 'complex_column' field */ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroCompat.Builder clearComplexColumn() { complex_column = null; - fieldSetFlags()[16] = false; + fieldSetFlags()[2] = false; return this; } @@ -975,23 +238,9 @@ public org.apache.spark.sql.execution.datasources.parquet.test.avro.ParquetAvroC public ParquetAvroCompat build() { try { ParquetAvroCompat record = new ParquetAvroCompat(); - record.bool_column = fieldSetFlags()[0] ? this.bool_column : (java.lang.Boolean) defaultValue(fields()[0]); - record.int_column = fieldSetFlags()[1] ? this.int_column : (java.lang.Integer) defaultValue(fields()[1]); - record.long_column = fieldSetFlags()[2] ? this.long_column : (java.lang.Long) defaultValue(fields()[2]); - record.float_column = fieldSetFlags()[3] ? this.float_column : (java.lang.Float) defaultValue(fields()[3]); - record.double_column = fieldSetFlags()[4] ? this.double_column : (java.lang.Double) defaultValue(fields()[4]); - record.binary_column = fieldSetFlags()[5] ? this.binary_column : (java.nio.ByteBuffer) defaultValue(fields()[5]); - record.string_column = fieldSetFlags()[6] ? this.string_column : (java.lang.String) defaultValue(fields()[6]); - record.maybe_bool_column = fieldSetFlags()[7] ? this.maybe_bool_column : (java.lang.Boolean) defaultValue(fields()[7]); - record.maybe_int_column = fieldSetFlags()[8] ? this.maybe_int_column : (java.lang.Integer) defaultValue(fields()[8]); - record.maybe_long_column = fieldSetFlags()[9] ? this.maybe_long_column : (java.lang.Long) defaultValue(fields()[9]); - record.maybe_float_column = fieldSetFlags()[10] ? this.maybe_float_column : (java.lang.Float) defaultValue(fields()[10]); - record.maybe_double_column = fieldSetFlags()[11] ? this.maybe_double_column : (java.lang.Double) defaultValue(fields()[11]); - record.maybe_binary_column = fieldSetFlags()[12] ? this.maybe_binary_column : (java.nio.ByteBuffer) defaultValue(fields()[12]); - record.maybe_string_column = fieldSetFlags()[13] ? this.maybe_string_column : (java.lang.String) defaultValue(fields()[13]); - record.strings_column = fieldSetFlags()[14] ? this.strings_column : (java.util.List) defaultValue(fields()[14]); - record.string_to_int_column = fieldSetFlags()[15] ? this.string_to_int_column : (java.util.Map) defaultValue(fields()[15]); - record.complex_column = fieldSetFlags()[16] ? this.complex_column : (java.util.Map>) defaultValue(fields()[16]); + record.strings_column = fieldSetFlags()[0] ? this.strings_column : (java.util.List) defaultValue(fields()[0]); + record.string_to_int_column = fieldSetFlags()[1] ? this.string_to_int_column : (java.util.Map) defaultValue(fields()[1]); + record.complex_column = fieldSetFlags()[2] ? this.complex_column : (java.util.Map>) defaultValue(fields()[2]); return record; } catch (Exception e) { throw new org.apache.avro.AvroRuntimeException(e); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala index 82d40e2b61a10..45db619567a22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.nio.ByteBuffer import java.util.{List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters.seqAsJavaListConverter +import scala.collection.JavaConverters.mapAsJavaMapConverter import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord @@ -32,48 +33,196 @@ import org.apache.spark.sql.execution.datasources.parquet.test.avro._ import org.apache.spark.sql.test.SharedSQLContext class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with SharedSQLContext { - import ParquetCompatibilityTest._ - private def withWriter[T <: IndexedRecord] (path: String, schema: Schema) (f: AvroParquetWriter[T] => Unit): Unit = { + logInfo( + s"""Writing Avro records with the following Avro schema into Parquet file: + | + |${schema.toString(true)} + """.stripMargin) + val writer = new AvroParquetWriter[T](new Path(path), schema) try f(writer) finally writer.close() } - test("Read Parquet file generated by parquet-avro") { + test("required primitives") { withTempPath { dir => val path = dir.getCanonicalPath - withWriter[ParquetAvroCompat](path, ParquetAvroCompat.getClassSchema) { writer => - (0 until 10).foreach(i => writer.write(makeParquetAvroCompat(i))) + withWriter[AvroPrimitives](path, AvroPrimitives.getClassSchema) { writer => + (0 until 10).foreach { i => + writer.write( + AvroPrimitives.newBuilder() + .setBoolColumn(i % 2 == 0) + .setIntColumn(i) + .setLongColumn(i.toLong * 10) + .setFloatColumn(i.toFloat + 0.1f) + .setDoubleColumn(i.toDouble + 0.2d) + .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8"))) + .setStringColumn(s"val_$i") + .build()) + } } - logInfo( - s"""Schema of the Parquet file written by parquet-avro: - |${readParquetSchema(path)} - """.stripMargin) + logParquetSchema(path) checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => - def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) - Row( i % 2 == 0, i, i.toLong * 10, i.toFloat + 0.1f, i.toDouble + 0.2d, - s"val_$i".getBytes, - s"val_$i", + s"val_$i".getBytes("UTF-8"), + s"val_$i") + }) + } + } + + test("optional primitives") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withWriter[AvroOptionalPrimitives](path, AvroOptionalPrimitives.getClassSchema) { writer => + (0 until 10).foreach { i => + val record = if (i % 3 == 0) { + AvroOptionalPrimitives.newBuilder() + .setMaybeBoolColumn(null) + .setMaybeIntColumn(null) + .setMaybeLongColumn(null) + .setMaybeFloatColumn(null) + .setMaybeDoubleColumn(null) + .setMaybeBinaryColumn(null) + .setMaybeStringColumn(null) + .build() + } else { + AvroOptionalPrimitives.newBuilder() + .setMaybeBoolColumn(i % 2 == 0) + .setMaybeIntColumn(i) + .setMaybeLongColumn(i.toLong * 10) + .setMaybeFloatColumn(i.toFloat + 0.1f) + .setMaybeDoubleColumn(i.toDouble + 0.2d) + .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes("UTF-8"))) + .setMaybeStringColumn(s"val_$i") + .build() + } + + writer.write(record) + } + } + + logParquetSchema(path) + + checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + if (i % 3 == 0) { + Row.apply(Seq.fill(7)(null): _*) + } else { + Row( + i % 2 == 0, + i, + i.toLong * 10, + i.toFloat + 0.1f, + i.toDouble + 0.2d, + s"val_$i".getBytes("UTF-8"), + s"val_$i") + } + }) + } + } + + test("non-nullable arrays") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withWriter[AvroNonNullableArrays](path, AvroNonNullableArrays.getClassSchema) { writer => + (0 until 10).foreach { i => + val record = { + val builder = + AvroNonNullableArrays.newBuilder() + .setStringsColumn(Seq.tabulate(3)(i => s"val_$i").asJava) + + if (i % 3 == 0) { + builder.setMaybeIntsColumn(null).build() + } else { + builder.setMaybeIntsColumn(Seq.tabulate(3)(Int.box).asJava).build() + } + } + + writer.write(record) + } + } + + logParquetSchema(path) + + checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + Row( + Seq.tabulate(3)(i => s"val_$i"), + if (i % 3 == 0) null else Seq.tabulate(3)(identity)) + }) + } + } + + ignore("nullable arrays (parquet-avro 1.7.0 does not properly support this)") { + // TODO Complete this test case after upgrading to parquet-mr 1.8+ + } + + test("SPARK-10136 array of primitive array") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withWriter[AvroArrayOfArray](path, AvroArrayOfArray.getClassSchema) { writer => + (0 until 10).foreach { i => + writer.write(AvroArrayOfArray.newBuilder() + .setIntArraysColumn( + Seq.tabulate(3, 3)((i, j) => i * 3 + j: Integer).map(_.asJava).asJava) + .build()) + } + } - nullable(i % 2 == 0: java.lang.Boolean), - nullable(i: Integer), - nullable(i.toLong: java.lang.Long), - nullable(i.toFloat + 0.1f: java.lang.Float), - nullable(i.toDouble + 0.2d: java.lang.Double), - nullable(s"val_$i".getBytes), - nullable(s"val_$i"), + logParquetSchema(path) + checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + Row(Seq.tabulate(3, 3)((i, j) => i * 3 + j)) + }) + } + } + + test("map of primitive array") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withWriter[AvroMapOfArray](path, AvroMapOfArray.getClassSchema) { writer => + (0 until 10).foreach { i => + writer.write(AvroMapOfArray.newBuilder() + .setStringToIntsColumn( + Seq.tabulate(3) { i => + i.toString -> Seq.tabulate(3)(j => i + j: Integer).asJava + }.toMap.asJava) + .build()) + } + } + + logParquetSchema(path) + + checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + Row(Seq.tabulate(3)(i => i.toString -> Seq.tabulate(3)(j => i + j)).toMap) + }) + } + } + + test("various complex types") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withWriter[ParquetAvroCompat](path, ParquetAvroCompat.getClassSchema) { writer => + (0 until 10).foreach(i => writer.write(makeParquetAvroCompat(i))) + } + + logParquetSchema(path) + + checkAnswer(sqlContext.read.parquet(path), (0 until 10).map { i => + Row( Seq.tabulate(3)(n => s"arr_${i + n}"), Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap, Seq.tabulate(3) { n => @@ -86,47 +235,27 @@ class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest with Shared } def makeParquetAvroCompat(i: Int): ParquetAvroCompat = { - def nullable[T <: AnyRef] = makeNullable[T](i) _ - def makeComplexColumn(i: Int): JMap[String, JList[Nested]] = { - mapAsJavaMap(Seq.tabulate(3) { n => - (i + n).toString -> seqAsJavaList(Seq.tabulate(3) { m => + Seq.tabulate(3) { n => + (i + n).toString -> Seq.tabulate(3) { m => Nested .newBuilder() - .setNestedIntsColumn(seqAsJavaList(Seq.tabulate(3)(j => i + j + m))) + .setNestedIntsColumn(Seq.tabulate(3)(j => i + j + m: Integer).asJava) .setNestedStringColumn(s"val_${i + m}") .build() - }) - }.toMap) + }.asJava + }.toMap.asJava } ParquetAvroCompat .newBuilder() - .setBoolColumn(i % 2 == 0) - .setIntColumn(i) - .setLongColumn(i.toLong * 10) - .setFloatColumn(i.toFloat + 0.1f) - .setDoubleColumn(i.toDouble + 0.2d) - .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes)) - .setStringColumn(s"val_$i") - - .setMaybeBoolColumn(nullable(i % 2 == 0: java.lang.Boolean)) - .setMaybeIntColumn(nullable(i: Integer)) - .setMaybeLongColumn(nullable(i.toLong: java.lang.Long)) - .setMaybeFloatColumn(nullable(i.toFloat + 0.1f: java.lang.Float)) - .setMaybeDoubleColumn(nullable(i.toDouble + 0.2d: java.lang.Double)) - .setMaybeBinaryColumn(nullable(ByteBuffer.wrap(s"val_$i".getBytes))) - .setMaybeStringColumn(nullable(s"val_$i")) - - .setStringsColumn(Seq.tabulate(3)(n => s"arr_${i + n}")) - .setStringToIntColumn( - mapAsJavaMap(Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap)) + .setStringsColumn(Seq.tabulate(3)(n => s"arr_${i + n}").asJava) + .setStringToIntColumn(Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap.asJava) .setComplexColumn(makeComplexColumn(i)) - .build() } - test("SPARK-9407 Don't push down predicates involving Parquet ENUM columns") { + test("SPARK-9407 Push down predicates involving Parquet ENUM columns") { import testImplicits._ withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala index b3406729fcc5e..d85c564e3e8d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala @@ -43,6 +43,13 @@ private[sql] abstract class ParquetCompatibilityTest extends QueryTest with Parq val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true) footers.head.getParquetMetadata.getFileMetaData.getSchema } + + protected def logParquetSchema(path: String): Unit = { + logInfo( + s"""Schema of the Parquet file written by parquet-avro: + |${readParquetSchema(path)} + """.stripMargin) + } } object ParquetCompatibilityTest { From 12de348332108f8c0c5bdad1d4cfac89b952b0f8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 20 Aug 2015 11:31:03 -0700 Subject: [PATCH 032/232] [SPARK-10126] [PROJECT INFRA] Fix typo in release-build.sh which broke snapshot publishing for Scala 2.11 The current `release-build.sh` has a typo which breaks snapshot publication for Scala 2.11. We should change the Scala version to 2.11 and clean before building a 2.11 snapshot. Author: Josh Rosen Closes #8325 from JoshRosen/fix-2.11-snapshots. --- dev/create-release/release-build.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 399c73e7bf6bc..d0b3a54dde1dc 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -225,9 +225,9 @@ if [[ "$1" == "publish-snapshot" ]]; then $MVN -DzincPort=$ZINC_PORT --settings $tmp_settings -DskipTests $PUBLISH_PROFILES \ -Phive-thriftserver deploy - ./dev/change-scala-version.sh 2.10 + ./dev/change-scala-version.sh 2.11 $MVN -DzincPort=$ZINC_PORT -Dscala-2.11 --settings $tmp_settings \ - -DskipTests $PUBLISH_PROFILES deploy + -DskipTests $PUBLISH_PROFILES clean deploy # Clean-up Zinc nailgun process /usr/sbin/lsof -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill From 907df2fce00d2cbc9fae371344f05f800e0d2726 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 20 Aug 2015 13:51:54 -0700 Subject: [PATCH 033/232] [SQL] [MINOR] remove unnecessary class This class is identical to `org.apache.spark.sql.execution.datasources.jdbc. DefaultSource` and is not needed. Author: Wenchen Fan Closes #8334 from cloud-fan/minor. --- .../execution/datasources/DefaultSource.scala | 64 ------------------- 1 file changed, 64 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DefaultSource.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DefaultSource.scala deleted file mode 100644 index 6e4cc4de7f651..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DefaultSource.scala +++ /dev/null @@ -1,64 +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.execution.datasources - -import java.util.Properties - -import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.execution.datasources.jdbc.{JDBCRelation, JDBCPartitioningInfo, DriverRegistry} -import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister, RelationProvider} - - -class DefaultSource extends RelationProvider with DataSourceRegister { - - override def shortName(): String = "jdbc" - - /** Returns a new base relation with the given parameters. */ - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) - val driver = parameters.getOrElse("driver", null) - val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) - val partitionColumn = parameters.getOrElse("partitionColumn", null) - val lowerBound = parameters.getOrElse("lowerBound", null) - val upperBound = parameters.getOrElse("upperBound", null) - val numPartitions = parameters.getOrElse("numPartitions", null) - - if (driver != null) DriverRegistry.register(driver) - - if (partitionColumn != null - && (lowerBound == null || upperBound == null || numPartitions == null)) { - sys.error("Partitioning incompletely specified") - } - - val partitionInfo = if (partitionColumn == null) { - null - } else { - JDBCPartitioningInfo( - partitionColumn, - lowerBound.toLong, - upperBound.toLong, - numPartitions.toInt) - } - val parts = JDBCRelation.columnPartition(partitionInfo) - val properties = new Properties() // Additional properties that we will pass to getConnection - parameters.foreach(kv => properties.setProperty(kv._1, kv._2)) - JDBCRelation(url, table, parts, properties)(sqlContext) - } -} From 2a3d98aae285aba39786e9809f96de412a130f39 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 20 Aug 2015 14:47:04 -0700 Subject: [PATCH 034/232] [SPARK-10138] [ML] move setters to MultilayerPerceptronClassifier and add Java test suite Otherwise, setters do not return self type. jkbradley avulanov Author: Xiangrui Meng Closes #8342 from mengxr/SPARK-10138. --- .../MultilayerPerceptronClassifier.scala | 54 +++++++------- ...vaMultilayerPerceptronClassifierSuite.java | 74 +++++++++++++++++++ 2 files changed, 101 insertions(+), 27 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index ccca4ecc004c3..1e5b0bc4453e4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -42,9 +42,6 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams ParamValidators.arrayLengthGt(1) ) - /** @group setParam */ - def setLayers(value: Array[Int]): this.type = set(layers, value) - /** @group getParam */ final def getLayers: Array[Int] = $(layers) @@ -61,33 +58,9 @@ private[ml] trait MultilayerPerceptronParams extends PredictorParams "it is adjusted to the size of this data. Recommended size is between 10 and 1000", ParamValidators.gt(0)) - /** @group setParam */ - def setBlockSize(value: Int): this.type = set(blockSize, value) - /** @group getParam */ final def getBlockSize: Int = $(blockSize) - /** - * Set the maximum number of iterations. - * Default is 100. - * @group setParam - */ - def setMaxIter(value: Int): this.type = set(maxIter, value) - - /** - * Set the convergence tolerance of iterations. - * Smaller value will lead to higher accuracy with the cost of more iterations. - * Default is 1E-4. - * @group setParam - */ - def setTol(value: Double): this.type = set(tol, value) - - /** - * Set the seed for weights initialization. - * @group setParam - */ - def setSeed(value: Long): this.type = set(seed, value) - setDefault(maxIter -> 100, tol -> 1e-4, layers -> Array(1, 1), blockSize -> 128) } @@ -136,6 +109,33 @@ class MultilayerPerceptronClassifier(override val uid: String) def this() = this(Identifiable.randomUID("mlpc")) + /** @group setParam */ + def setLayers(value: Array[Int]): this.type = set(layers, value) + + /** @group setParam */ + def setBlockSize(value: Int): this.type = set(blockSize, value) + + /** + * Set the maximum number of iterations. + * Default is 100. + * @group setParam + */ + def setMaxIter(value: Int): this.type = set(maxIter, value) + + /** + * Set the convergence tolerance of iterations. + * Smaller value will lead to higher accuracy with the cost of more iterations. + * Default is 1E-4. + * @group setParam + */ + def setTol(value: Double): this.type = set(tol, value) + + /** + * Set the seed for weights initialization. + * @group setParam + */ + def setSeed(value: Long): this.type = set(seed, value) + override def copy(extra: ParamMap): MultilayerPerceptronClassifier = defaultCopy(extra) /** diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java new file mode 100644 index 0000000000000..ec6b4bf3c0f8c --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaMultilayerPerceptronClassifierSuite.java @@ -0,0 +1,74 @@ +/* + * 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.ml.classification; + +import java.io.Serializable; +import java.util.Arrays; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; + +public class JavaMultilayerPerceptronClassifierSuite implements Serializable { + + private transient JavaSparkContext jsc; + private transient SQLContext sqlContext; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); + sqlContext = new SQLContext(jsc); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + sqlContext = null; + } + + @Test + public void testMLPC() { + DataFrame dataFrame = sqlContext.createDataFrame( + jsc.parallelize(Arrays.asList( + new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), + new LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), + new LabeledPoint(0.0, Vectors.dense(1.0, 1.0)))), + LabeledPoint.class); + MultilayerPerceptronClassifier mlpc = new MultilayerPerceptronClassifier() + .setLayers(new int[] {2, 5, 2}) + .setBlockSize(1) + .setSeed(11L) + .setMaxIter(100); + MultilayerPerceptronClassificationModel model = mlpc.fit(dataFrame); + DataFrame result = model.transform(dataFrame); + Row[] predictionAndLabels = result.select("prediction", "label").collect(); + for (Row r: predictionAndLabels) { + Assert.assertEquals((int) r.getDouble(0), (int) r.getDouble(1)); + } + } +} From 7cfc0750e14f2c1b3847e4720cc02150253525a9 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 20 Aug 2015 14:56:08 -0700 Subject: [PATCH 035/232] [SPARK-10108] Add since tags to mllib.feature Author: MechCoder Closes #8309 from MechCoder/tags_feature. --- .../spark/mllib/feature/ChiSqSelector.scala | 12 +++++++++--- .../mllib/feature/ElementwiseProduct.scala | 4 +++- .../spark/mllib/feature/HashingTF.scala | 11 ++++++++++- .../org/apache/spark/mllib/feature/IDF.scala | 8 +++++++- .../spark/mllib/feature/Normalizer.scala | 5 ++++- .../org/apache/spark/mllib/feature/PCA.scala | 9 ++++++++- .../spark/mllib/feature/StandardScaler.scala | 13 ++++++++++++- .../mllib/feature/VectorTransformer.scala | 6 +++++- .../apache/spark/mllib/feature/Word2Vec.scala | 19 ++++++++++++++++++- 9 files changed, 76 insertions(+), 11 deletions(-) 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 5f8c1dea237b4..fdd974d7a391e 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 @@ -19,7 +19,7 @@ package org.apache.spark.mllib.feature import scala.collection.mutable.ArrayBuilder -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.Statistics @@ -31,8 +31,10 @@ import org.apache.spark.rdd.RDD * * @param selectedFeatures list of indices to select (filter). Must be ordered asc */ +@Since("1.3.0") @Experimental -class ChiSqSelectorModel (val selectedFeatures: Array[Int]) extends VectorTransformer { +class ChiSqSelectorModel ( + @Since("1.3.0") val selectedFeatures: Array[Int]) extends VectorTransformer { require(isSorted(selectedFeatures), "Array has to be sorted asc") @@ -52,6 +54,7 @@ class ChiSqSelectorModel (val selectedFeatures: Array[Int]) extends VectorTransf * @param vector vector to be transformed. * @return transformed vector. */ + @Since("1.3.0") override def transform(vector: Vector): Vector = { compress(vector, selectedFeatures) } @@ -107,8 +110,10 @@ class ChiSqSelectorModel (val selectedFeatures: Array[Int]) extends VectorTransf * @param numTopFeatures number of features that selector will select * (ordered by statistic value descending) */ +@Since("1.3.0") @Experimental -class ChiSqSelector (val numTopFeatures: Int) extends Serializable { +class ChiSqSelector ( + @Since("1.3.0") val numTopFeatures: Int) extends Serializable { /** * Returns a ChiSquared feature selector. @@ -117,6 +122,7 @@ class ChiSqSelector (val numTopFeatures: Int) extends Serializable { * Real-valued features will be treated as categorical for each distinct value. * Apply feature discretizer before using this function. */ + @Since("1.3.0") def fit(data: RDD[LabeledPoint]): ChiSqSelectorModel = { val indices = Statistics.chiSqTest(data) .zipWithIndex.sortBy { case (res, _) => -res.statistic } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala index d67fe6c3ee4f8..33e2d17bb472e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg._ /** @@ -27,6 +27,7 @@ import org.apache.spark.mllib.linalg._ * multiplier. * @param scalingVec The values used to scale the reference vector's individual components. */ +@Since("1.4.0") @Experimental class ElementwiseProduct(val scalingVec: Vector) extends VectorTransformer { @@ -36,6 +37,7 @@ class ElementwiseProduct(val scalingVec: Vector) extends VectorTransformer { * @param vector vector to be transformed. * @return transformed vector. */ + @Since("1.4.0") override def transform(vector: Vector): Vector = { require(vector.size == scalingVec.size, s"vector sizes do not match: Expected ${scalingVec.size} but found ${vector.size}") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index c53475818395f..e47d524b61623 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -22,7 +22,7 @@ import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.rdd.RDD @@ -34,19 +34,25 @@ import org.apache.spark.util.Utils * * @param numFeatures number of features (default: 2^20^) */ +@Since("1.1.0") @Experimental class HashingTF(val numFeatures: Int) extends Serializable { + /** + */ + @Since("1.1.0") def this() = this(1 << 20) /** * Returns the index of the input term. */ + @Since("1.1.0") def indexOf(term: Any): Int = Utils.nonNegativeMod(term.##, numFeatures) /** * Transforms the input document into a sparse term frequency vector. */ + @Since("1.1.0") def transform(document: Iterable[_]): Vector = { val termFrequencies = mutable.HashMap.empty[Int, Double] document.foreach { term => @@ -59,6 +65,7 @@ class HashingTF(val numFeatures: Int) extends Serializable { /** * Transforms the input document into a sparse term frequency vector (Java version). */ + @Since("1.1.0") def transform(document: JavaIterable[_]): Vector = { transform(document.asScala) } @@ -66,6 +73,7 @@ class HashingTF(val numFeatures: Int) extends Serializable { /** * Transforms the input document to term frequency vectors. */ + @Since("1.1.0") def transform[D <: Iterable[_]](dataset: RDD[D]): RDD[Vector] = { dataset.map(this.transform) } @@ -73,6 +81,7 @@ class HashingTF(val numFeatures: Int) extends Serializable { /** * Transforms the input document to term frequency vectors (Java version). */ + @Since("1.1.0") def transform[D <: JavaIterable[_]](dataset: JavaRDD[D]): JavaRDD[Vector] = { dataset.rdd.map(this.transform).toJavaRDD() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 3fab7ea79befc..d5353ddd972e0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.feature import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.rdd.RDD @@ -37,6 +37,7 @@ import org.apache.spark.rdd.RDD * @param minDocFreq minimum of documents in which a term * should appear for filtering */ +@Since("1.1.0") @Experimental class IDF(val minDocFreq: Int) { @@ -48,6 +49,7 @@ class IDF(val minDocFreq: Int) { * Computes the inverse document frequency. * @param dataset an RDD of term frequency vectors */ + @Since("1.1.0") def fit(dataset: RDD[Vector]): IDFModel = { val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator( minDocFreq = minDocFreq))( @@ -61,6 +63,7 @@ class IDF(val minDocFreq: Int) { * Computes the inverse document frequency. * @param dataset a JavaRDD of term frequency vectors */ + @Since("1.1.0") def fit(dataset: JavaRDD[Vector]): IDFModel = { fit(dataset.rdd) } @@ -171,6 +174,7 @@ class IDFModel private[spark] (val idf: Vector) extends Serializable { * @param dataset an RDD of term frequency vectors * @return an RDD of TF-IDF vectors */ + @Since("1.1.0") def transform(dataset: RDD[Vector]): RDD[Vector] = { val bcIdf = dataset.context.broadcast(idf) dataset.mapPartitions(iter => iter.map(v => IDFModel.transform(bcIdf.value, v))) @@ -182,6 +186,7 @@ class IDFModel private[spark] (val idf: Vector) extends Serializable { * @param v a term frequency vector * @return a TF-IDF vector */ + @Since("1.3.0") def transform(v: Vector): Vector = IDFModel.transform(idf, v) /** @@ -189,6 +194,7 @@ class IDFModel private[spark] (val idf: Vector) extends Serializable { * @param dataset a JavaRDD of term frequency vectors * @return a JavaRDD of TF-IDF vectors */ + @Since("1.1.0") def transform(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { transform(dataset.rdd).toJavaRDD() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index 32848e039eb81..0e070257d9fb2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} /** @@ -31,9 +31,11 @@ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors * * @param p Normalization in L^p^ space, p = 2 by default. */ +@Since("1.1.0") @Experimental class Normalizer(p: Double) extends VectorTransformer { + @Since("1.1.0") def this() = this(2) require(p >= 1.0) @@ -44,6 +46,7 @@ class Normalizer(p: Double) extends VectorTransformer { * @param vector vector to be normalized. * @return normalized vector. If the norm of the input is zero, it will return the input vector. */ + @Since("1.1.0") override def transform(vector: Vector): Vector = { val norm = Vectors.norm(vector, p) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index 2a66263d8b7d6..a48b7bba665d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.feature +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.distributed.RowMatrix @@ -27,6 +28,7 @@ import org.apache.spark.rdd.RDD * * @param k number of principal components */ +@Since("1.4.0") class PCA(val k: Int) { require(k >= 1, s"PCA requires a number of principal components k >= 1 but was given $k") @@ -35,6 +37,7 @@ class PCA(val k: Int) { * * @param sources source vectors */ + @Since("1.4.0") def fit(sources: RDD[Vector]): PCAModel = { require(k <= sources.first().size, s"source vector size is ${sources.first().size} must be greater than k=$k") @@ -58,7 +61,10 @@ class PCA(val k: Int) { new PCAModel(k, pc) } - /** Java-friendly version of [[fit()]] */ + /** + * Java-friendly version of [[fit()]] + */ + @Since("1.4.0") def fit(sources: JavaRDD[Vector]): PCAModel = fit(sources.rdd) } @@ -76,6 +82,7 @@ class PCAModel private[spark] (val k: Int, val pc: DenseMatrix) extends VectorTr * Vector must be the same length as the source vectors given to [[PCA.fit()]]. * @return transformed vector. Vector will be of length k. */ + @Since("1.4.0") override def transform(vector: Vector): Vector = { vector match { case dv: DenseVector => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index c73b8f258060d..b95d5a899001e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.Logging -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD @@ -32,9 +32,11 @@ import org.apache.spark.rdd.RDD * dense output, so this does not work on sparse input and will raise an exception. * @param withStd True by default. Scales the data to unit standard deviation. */ +@Since("1.1.0") @Experimental class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { + @Since("1.1.0") def this() = this(false, true) if (!(withMean || withStd)) { @@ -47,6 +49,7 @@ class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { * @param data The data used to compute the mean and variance to build the transformation model. * @return a StandardScalarModel */ + @Since("1.1.0") def fit(data: RDD[Vector]): StandardScalerModel = { // TODO: skip computation if both withMean and withStd are false val summary = data.treeAggregate(new MultivariateOnlineSummarizer)( @@ -69,6 +72,7 @@ class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { * @param withStd whether to scale the data to have unit standard deviation * @param withMean whether to center the data before scaling */ +@Since("1.1.0") @Experimental class StandardScalerModel ( val std: Vector, @@ -76,6 +80,9 @@ class StandardScalerModel ( var withStd: Boolean, var withMean: Boolean) extends VectorTransformer { + /** + */ + @Since("1.3.0") def this(std: Vector, mean: Vector) { this(std, mean, withStd = std != null, withMean = mean != null) require(this.withStd || this.withMean, @@ -86,8 +93,10 @@ class StandardScalerModel ( } } + @Since("1.3.0") def this(std: Vector) = this(std, null) + @Since("1.3.0") @DeveloperApi def setWithMean(withMean: Boolean): this.type = { require(!(withMean && this.mean == null), "cannot set withMean to true while mean is null") @@ -95,6 +104,7 @@ class StandardScalerModel ( this } + @Since("1.3.0") @DeveloperApi def setWithStd(withStd: Boolean): this.type = { require(!(withStd && this.std == null), @@ -115,6 +125,7 @@ class StandardScalerModel ( * @return Standardized vector. If the std of a column is zero, it will return default `0.0` * for the column with zero std. */ + @Since("1.1.0") override def transform(vector: Vector): Vector = { require(mean.size == vector.size) if (withMean) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index 7358c1c84f79c..5778fd1d09254 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD * :: DeveloperApi :: * Trait for transformation of a vector */ +@Since("1.1.0") @DeveloperApi trait VectorTransformer extends Serializable { @@ -35,6 +36,7 @@ trait VectorTransformer extends Serializable { * @param vector vector to be transformed. * @return transformed vector. */ + @Since("1.1.0") def transform(vector: Vector): Vector /** @@ -43,6 +45,7 @@ trait VectorTransformer extends Serializable { * @param data RDD[Vector] to be transformed. * @return transformed RDD[Vector]. */ + @Since("1.1.0") def transform(data: RDD[Vector]): RDD[Vector] = { // Later in #1498 , all RDD objects are sent via broadcasting instead of akka. // So it should be no longer necessary to explicitly broadcast `this` object. @@ -55,6 +58,7 @@ trait VectorTransformer extends Serializable { * @param data JavaRDD[Vector] to be transformed. * @return transformed JavaRDD[Vector]. */ + @Since("1.1.0") def transform(data: JavaRDD[Vector]): JavaRDD[Vector] = { transform(data.rdd) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index cbbd2b0c8d060..e6f45ae4b01d5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -32,7 +32,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors, DenseMatrix, BLAS, DenseVector} import org.apache.spark.mllib.util.{Loader, Saveable} @@ -70,6 +70,7 @@ private case class VocabWord( * and * Distributed Representations of Words and Phrases and their Compositionality. */ +@Since("1.1.0") @Experimental class Word2Vec extends Serializable with Logging { @@ -83,6 +84,7 @@ class Word2Vec extends Serializable with Logging { /** * Sets vector size (default: 100). */ + @Since("1.1.0") def setVectorSize(vectorSize: Int): this.type = { this.vectorSize = vectorSize this @@ -91,6 +93,7 @@ class Word2Vec extends Serializable with Logging { /** * Sets initial learning rate (default: 0.025). */ + @Since("1.1.0") def setLearningRate(learningRate: Double): this.type = { this.learningRate = learningRate this @@ -99,6 +102,7 @@ class Word2Vec extends Serializable with Logging { /** * Sets number of partitions (default: 1). Use a small number for accuracy. */ + @Since("1.1.0") def setNumPartitions(numPartitions: Int): this.type = { require(numPartitions > 0, s"numPartitions must be greater than 0 but got $numPartitions") this.numPartitions = numPartitions @@ -109,6 +113,7 @@ class Word2Vec extends Serializable with Logging { * Sets number of iterations (default: 1), which should be smaller than or equal to number of * partitions. */ + @Since("1.1.0") def setNumIterations(numIterations: Int): this.type = { this.numIterations = numIterations this @@ -117,6 +122,7 @@ class Word2Vec extends Serializable with Logging { /** * Sets random seed (default: a random long integer). */ + @Since("1.1.0") def setSeed(seed: Long): this.type = { this.seed = seed this @@ -126,6 +132,7 @@ class Word2Vec extends Serializable with Logging { * Sets minCount, the minimum number of times a token must appear to be included in the word2vec * model's vocabulary (default: 5). */ + @Since("1.3.0") def setMinCount(minCount: Int): this.type = { this.minCount = minCount this @@ -263,6 +270,7 @@ class Word2Vec extends Serializable with Logging { * @param dataset an RDD of words * @return a Word2VecModel */ + @Since("1.1.0") def fit[S <: Iterable[String]](dataset: RDD[S]): Word2VecModel = { val words = dataset.flatMap(x => x) @@ -412,6 +420,7 @@ class Word2Vec extends Serializable with Logging { * @param dataset a JavaRDD of words * @return a Word2VecModel */ + @Since("1.1.0") def fit[S <: JavaIterable[String]](dataset: JavaRDD[S]): Word2VecModel = { fit(dataset.rdd.map(_.asScala)) } @@ -454,6 +463,7 @@ class Word2VecModel private[mllib] ( wordVecNorms } + @Since("1.5.0") def this(model: Map[String, Array[Float]]) = { this(Word2VecModel.buildWordIndex(model), Word2VecModel.buildWordVectors(model)) } @@ -469,6 +479,7 @@ class Word2VecModel private[mllib] ( override protected def formatVersion = "1.0" + @Since("1.4.0") def save(sc: SparkContext, path: String): Unit = { Word2VecModel.SaveLoadV1_0.save(sc, path, getVectors) } @@ -478,6 +489,7 @@ class Word2VecModel private[mllib] ( * @param word a word * @return vector representation of word */ + @Since("1.1.0") def transform(word: String): Vector = { wordIndex.get(word) match { case Some(ind) => @@ -494,6 +506,7 @@ class Word2VecModel private[mllib] ( * @param num number of synonyms to find * @return array of (word, cosineSimilarity) */ + @Since("1.1.0") def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) findSynonyms(vector, num) @@ -505,6 +518,7 @@ class Word2VecModel private[mllib] ( * @param num number of synonyms to find * @return array of (word, cosineSimilarity) */ + @Since("1.1.0") def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") // TODO: optimize top-k @@ -534,6 +548,7 @@ class Word2VecModel private[mllib] ( /** * Returns a map of words to their vector representations. */ + @Since("1.2.0") def getVectors: Map[String, Array[Float]] = { wordIndex.map { case (word, ind) => (word, wordVectors.slice(vectorSize * ind, vectorSize * ind + vectorSize)) @@ -541,6 +556,7 @@ class Word2VecModel private[mllib] ( } } +@Since("1.4.0") @Experimental object Word2VecModel extends Loader[Word2VecModel] { @@ -600,6 +616,7 @@ object Word2VecModel extends Loader[Word2VecModel] { } } + @Since("1.4.0") override def load(sc: SparkContext, path: String): Word2VecModel = { val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) From eaafe139f881d6105996373c9b11f2ccd91b5b3e Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 20 Aug 2015 15:01:31 -0700 Subject: [PATCH 036/232] [SPARK-9245] [MLLIB] LDA topic assignments For each (document, term) pair, return top topic. Note that instances of (doc, term) pairs within a document (a.k.a. "tokens") are exchangeable, so we should provide an estimate per document-term, rather than per token. CC: rotationsymmetry mengxr Author: Joseph K. Bradley Closes #8329 from jkbradley/lda-topic-assignments. --- .../spark/mllib/clustering/LDAModel.scala | 51 +++++++++++++++++-- .../spark/mllib/clustering/LDAOptimizer.scala | 2 +- .../spark/mllib/clustering/JavaLDASuite.java | 7 +++ .../spark/mllib/clustering/LDASuite.scala | 21 +++++++- 4 files changed, 74 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index b70e380c0393e..6bc68a4c18b99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argtopk, normalize, sum} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax, argtopk, normalize, sum} import breeze.numerics.{exp, lgamma} import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats @@ -438,7 +438,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { Loader.checkSchema[Data](dataFrame.schema) val topics = dataFrame.collect() val vocabSize = topics(0).getAs[Vector](0).size - val k = topics.size + val k = topics.length val brzTopics = BDM.zeros[Double](vocabSize, k) topics.foreach { case Row(vec: Vector, ind: Int) => @@ -610,6 +610,50 @@ class DistributedLDAModel private[clustering] ( } } + /** + * Return the top topic for each (doc, term) pair. I.e., for each document, what is the most + * likely topic generating each term? + * + * @return RDD of (doc ID, assignment of top topic index for each term), + * where the assignment is specified via a pair of zippable arrays + * (term indices, topic indices). Note that terms will be omitted if not present in + * the document. + */ + lazy val topicAssignments: RDD[(Long, Array[Int], Array[Int])] = { + // For reference, compare the below code with the core part of EMLDAOptimizer.next(). + val eta = topicConcentration + val W = vocabSize + val alpha = docConcentration(0) + val N_k = globalTopicTotals + val sendMsg: EdgeContext[TopicCounts, TokenCount, (Array[Int], Array[Int])] => Unit = + (edgeContext) => { + // E-STEP: Compute gamma_{wjk} (smoothed topic distributions). + val scaledTopicDistribution: TopicCounts = + computePTopic(edgeContext.srcAttr, edgeContext.dstAttr, N_k, W, eta, alpha) + // For this (doc j, term w), send top topic k to doc vertex. + val topTopic: Int = argmax(scaledTopicDistribution) + val term: Int = index2term(edgeContext.dstId) + edgeContext.sendToSrc((Array(term), Array(topTopic))) + } + val mergeMsg: ((Array[Int], Array[Int]), (Array[Int], Array[Int])) => (Array[Int], Array[Int]) = + (terms_topics0, terms_topics1) => { + (terms_topics0._1 ++ terms_topics1._1, terms_topics0._2 ++ terms_topics1._2) + } + // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. + val perDocAssignments = + graph.aggregateMessages[(Array[Int], Array[Int])](sendMsg, mergeMsg).filter(isDocumentVertex) + perDocAssignments.map { case (docID: Long, (terms: Array[Int], topics: Array[Int])) => + // TODO: Avoid zip, which is inefficient. + val (sortedTerms, sortedTopics) = terms.zip(topics).sortBy(_._1).unzip + (docID, sortedTerms.toArray, sortedTopics.toArray) + } + } + + /** Java-friendly version of [[topicAssignments]] */ + lazy val javaTopicAssignments: JavaRDD[(java.lang.Long, Array[Int], Array[Int])] = { + topicAssignments.asInstanceOf[RDD[(java.lang.Long, Array[Int], Array[Int])]].toJavaRDD() + } + // TODO // override def logLikelihood(documents: RDD[(Long, Vector)]): Double = ??? @@ -849,10 +893,9 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { val classNameV1_0 = SaveLoadV1_0.thisClassName val model = (loadedClassName, loadedVersion) match { - case (className, "1.0") if className == classNameV1_0 => { + case (className, "1.0") if className == classNameV1_0 => DistributedLDAModel.SaveLoadV1_0.load(sc, path, vocabSize, docConcentration, topicConcentration, iterationTimes.toArray, gammaShape) - } case _ => throw new Exception( s"DistributedLDAModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $loadedVersion). Supported: ($classNameV1_0, 1.0)") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 360241c8081ac..cb517f9689ade 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -167,7 +167,7 @@ final class EMLDAOptimizer extends LDAOptimizer { edgeContext.sendToDst((false, scaledTopicDistribution)) edgeContext.sendToSrc((false, scaledTopicDistribution)) } - // This is a hack to detect whether we could modify the values in-place. + // The Boolean is a hack to detect whether we could modify the values in-place. // TODO: Add zero/seqOp/combOp option to aggregateMessages. (SPARK-5438) val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = (m0, m1) => { diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index 6e91cde2eabb5..3fea359a3b46c 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -134,6 +134,13 @@ public Boolean call(Tuple2 tuple2) { double[] topicWeights = topTopics._3(); assertEquals(3, topicIndices.length); assertEquals(3, topicWeights.length); + + // Check: topTopicAssignments + Tuple3 topicAssignment = model.javaTopicAssignments().first(); + Long docId2 = topicAssignment._1(); + int[] termIndices2 = topicAssignment._2(); + int[] topicIndices2 = topicAssignment._3(); + assertEquals(termIndices2.length, topicIndices2.length); } @Test diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 99e28499fd316..8a714f9b79e02 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -135,17 +135,34 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { } // Top 3 documents per topic - model.topDocumentsPerTopic(3).zip(topDocsByTopicDistributions(3)).foreach {case (t1, t2) => + model.topDocumentsPerTopic(3).zip(topDocsByTopicDistributions(3)).foreach { case (t1, t2) => assert(t1._1 === t2._1) assert(t1._2 === t2._2) } // All documents per topic val q = tinyCorpus.length - model.topDocumentsPerTopic(q).zip(topDocsByTopicDistributions(q)).foreach {case (t1, t2) => + model.topDocumentsPerTopic(q).zip(topDocsByTopicDistributions(q)).foreach { case (t1, t2) => assert(t1._1 === t2._1) assert(t1._2 === t2._2) } + + // Check: topTopicAssignments + // Make sure it assigns a topic to each term appearing in each doc. + val topTopicAssignments: Map[Long, (Array[Int], Array[Int])] = + model.topicAssignments.collect().map(x => x._1 -> (x._2, x._3)).toMap + assert(topTopicAssignments.keys.max < tinyCorpus.length) + tinyCorpus.foreach { case (docID: Long, doc: Vector) => + if (topTopicAssignments.contains(docID)) { + val (inds, vals) = topTopicAssignments(docID) + assert(inds.length === doc.numNonzeros) + // For "term" in actual doc, + // check that it has a topic assigned. + doc.foreachActive((term, wcnt) => assert(wcnt === 0 || inds.contains(term))) + } else { + assert(doc.numNonzeros === 0) + } + } } test("vertex indexing") { From afe9f03fd964d1e8604d02feee8d6970efbe6009 Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Thu, 20 Aug 2015 15:10:13 -0700 Subject: [PATCH 037/232] [SPARK-9400] [SQL] codegen for StringLocate This is based on #7779 , thanks to tarekauel . Fix the conflict and nullability. Closes #7779 and #8274 . Author: Tarek Auel Author: Davies Liu Closes #8330 from davies/stringLocate. --- .../expressions/stringExpressions.scala | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 3c23f2ecfb57c..b60d318534a41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -409,13 +409,14 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: * in given string after position pos. */ case class StringLocate(substr: Expression, str: Expression, start: Expression) - extends TernaryExpression with ImplicitCastInputTypes with CodegenFallback { + extends TernaryExpression with ImplicitCastInputTypes { def this(substr: Expression, str: Expression) = { this(substr, str, Literal(0)) } override def children: Seq[Expression] = substr :: str :: start :: Nil + override def nullable: Boolean = substr.nullable || str.nullable override def dataType: DataType = IntegerType override def inputTypes: Seq[DataType] = Seq(StringType, StringType, IntegerType) @@ -441,6 +442,31 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) } } + override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val substrGen = substr.gen(ctx) + val strGen = str.gen(ctx) + val startGen = start.gen(ctx) + s""" + int ${ev.primitive} = 0; + boolean ${ev.isNull} = false; + ${startGen.code} + if (!${startGen.isNull}) { + ${substrGen.code} + if (!${substrGen.isNull}) { + ${strGen.code} + if (!${strGen.isNull}) { + ${ev.primitive} = ${strGen.primitive}.indexOf(${substrGen.primitive}, + ${startGen.primitive}) + 1; + } else { + ${ev.isNull} = true; + } + } else { + ${ev.isNull} = true; + } + } + """ + } + override def prettyName: String = "locate" } From cdd9a2bb10e20556003843a0f7aaa33acd55f6d2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 20 Aug 2015 20:01:13 -0700 Subject: [PATCH 038/232] [SPARK-10140] [DOC] add target fields to @Since so constructors parameters and public fields can be annotated. rxin MechCoder Author: Xiangrui Meng Closes #8344 from mengxr/SPARK-10140.2. --- core/src/main/scala/org/apache/spark/annotation/Since.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/annotation/Since.scala b/core/src/main/scala/org/apache/spark/annotation/Since.scala index fa59393c22476..af483e361e339 100644 --- a/core/src/main/scala/org/apache/spark/annotation/Since.scala +++ b/core/src/main/scala/org/apache/spark/annotation/Since.scala @@ -18,6 +18,7 @@ package org.apache.spark.annotation import scala.annotation.StaticAnnotation +import scala.annotation.meta._ /** * A Scala annotation that specifies the Spark version when a definition was added. @@ -25,4 +26,5 @@ import scala.annotation.StaticAnnotation * hence works for overridden methods that inherit API documentation directly from parents. * The limitation is that it does not show up in the generated Java API documentation. */ +@param @field @getter @setter @beanGetter @beanSetter private[spark] class Since(version: String) extends StaticAnnotation From dcfe0c5cde953b31c5bfeb6e41d1fc9b333241eb Mon Sep 17 00:00:00 2001 From: Alexander Ulanov Date: Thu, 20 Aug 2015 20:02:27 -0700 Subject: [PATCH 039/232] [SPARK-9846] [DOCS] User guide for Multilayer Perceptron Classifier Added user guide for multilayer perceptron classifier: - Simplified description of the multilayer perceptron classifier - Example code for Scala and Java Author: Alexander Ulanov Closes #8262 from avulanov/SPARK-9846-mlpc-docs. --- docs/ml-ann.md | 123 +++++++++++++++++++++++++++++++++++++++++++++++ docs/ml-guide.md | 1 + 2 files changed, 124 insertions(+) create mode 100644 docs/ml-ann.md diff --git a/docs/ml-ann.md b/docs/ml-ann.md new file mode 100644 index 0000000000000..d5ddd92af1e96 --- /dev/null +++ b/docs/ml-ann.md @@ -0,0 +1,123 @@ +--- +layout: global +title: Multilayer perceptron classifier - ML +displayTitle: ML - Multilayer perceptron classifier +--- + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + + +Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). +MLPC consists of multiple layers of nodes. +Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes maps inputs to the outputs +by performing linear combination of the inputs with the node's weights `$\wv$` and bias `$\bv$` and applying an activation function. +It can be written in matrix form for MLPC with `$K+1$` layers as follows: +`\[ +\mathrm{y}(\x) = \mathrm{f_K}(...\mathrm{f_2}(\wv_2^T\mathrm{f_1}(\wv_1^T \x+b_1)+b_2)...+b_K) +\]` +Nodes in intermediate layers use sigmoid (logistic) function: +`\[ +\mathrm{f}(z_i) = \frac{1}{1 + e^{-z_i}} +\]` +Nodes in the output layer use softmax function: +`\[ +\mathrm{f}(z_i) = \frac{e^{z_i}}{\sum_{k=1}^N e^{z_k}} +\]` +The number of nodes `$N$` in the output layer corresponds to the number of classes. + +MLPC employes backpropagation for learning the model. We use logistic loss function for optimization and L-BFGS as optimization routine. + +**Examples** + +
+ +
+ +{% highlight scala %} +import org.apache.spark.ml.classification.MultilayerPerceptronClassifier +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.sql.Row + +// Load training data +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_multiclass_classification_data.txt").toDF() +// Split the data into train and test +val splits = data.randomSplit(Array(0.6, 0.4), seed = 1234L) +val train = splits(0) +val test = splits(1) +// specify layers for the neural network: +// input layer of size 4 (features), two intermediate of size 5 and 4 and output of size 3 (classes) +val layers = Array[Int](4, 5, 4, 3) +// create the trainer and set its parameters +val trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(128) + .setSeed(1234L) + .setMaxIter(100) +// train the model +val model = trainer.fit(train) +// compute precision on the test set +val result = model.transform(test) +val predictionAndLabels = result.select("prediction", "label") +val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("precision") +println("Precision:" + evaluator.evaluate(predictionAndLabels)) +{% endhighlight %} + +
+ +
+ +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.classification.MultilayerPerceptronClassificationModel; +import org.apache.spark.ml.classification.MultilayerPerceptronClassifier; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; + +// Load training data +String path = "data/mllib/sample_multiclass_classification_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); +DataFrame dataFrame = sqlContext.createDataFrame(data, LabeledPoint.class); +// Split the data into train and test +DataFrame[] splits = dataFrame.randomSplit(new double[]{0.6, 0.4}, 1234L); +DataFrame train = splits[0]; +DataFrame test = splits[1]; +// specify layers for the neural network: +// input layer of size 4 (features), two intermediate of size 5 and 4 and output of size 3 (classes) +int[] layers = new int[] {4, 5, 4, 3}; +// create the trainer and set its parameters +MultilayerPerceptronClassifier trainer = new MultilayerPerceptronClassifier() + .setLayers(layers) + .setBlockSize(128) + .setSeed(1234L) + .setMaxIter(100); +// train the model +MultilayerPerceptronClassificationModel model = trainer.fit(train); +// compute precision on the test set +DataFrame result = model.transform(test); +DataFrame predictionAndLabels = result.select("prediction", "label"); +MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setMetricName("precision"); +System.out.println("Precision = " + evaluator.evaluate(predictionAndLabels)); +{% endhighlight %} +
+ +
diff --git a/docs/ml-guide.md b/docs/ml-guide.md index c64fff7c0315a..de8fead3529e4 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -179,6 +179,7 @@ There are now several algorithms in the Pipelines API which are not in the lower * [Decision Trees for Classification and Regression](ml-decision-tree.html) * [Ensembles](ml-ensembles.html) * [Linear methods with elastic net regularization](ml-linear-methods.html) +* [Multilayer perceptron classifier](ml-ann.html) # Code Examples From bb220f6570aa0b95598b30524224a3e82c1effbc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 21 Aug 2015 01:43:49 -0700 Subject: [PATCH 040/232] [SPARK-10040] [SQL] Use batch insert for JDBC writing JIRA: https://issues.apache.org/jira/browse/SPARK-10040 We should use batch insert instead of single row in JDBC. Author: Liang-Chi Hsieh Closes #8273 from viirya/jdbc-insert-batch. --- .../execution/datasources/jdbc/JdbcUtils.scala | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 2d0e736ee4766..26788b2a4fd69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -88,13 +88,15 @@ object JdbcUtils extends Logging { table: String, iterator: Iterator[Row], rddSchema: StructType, - nullTypes: Array[Int]): Iterator[Byte] = { + nullTypes: Array[Int], + batchSize: Int): Iterator[Byte] = { val conn = getConnection() var committed = false try { conn.setAutoCommit(false) // Everything in the same db transaction. val stmt = insertStatement(conn, table, rddSchema) try { + var rowCount = 0 while (iterator.hasNext) { val row = iterator.next() val numFields = rddSchema.fields.length @@ -122,7 +124,15 @@ object JdbcUtils extends Logging { } i = i + 1 } - stmt.executeUpdate() + stmt.addBatch() + rowCount += 1 + if (rowCount % batchSize == 0) { + stmt.executeBatch() + rowCount = 0 + } + } + if (rowCount > 0) { + stmt.executeBatch() } } finally { stmt.close() @@ -211,8 +221,9 @@ object JdbcUtils extends Logging { val rddSchema = df.schema val driver: String = DriverRegistry.getDriverClassName(url) val getConnection: () => Connection = JDBCRDD.getConnector(driver, url, properties) + val batchSize = properties.getProperty("batchsize", "1000").toInt df.foreachPartition { iterator => - savePartition(getConnection, table, iterator, rddSchema, nullTypes) + savePartition(getConnection, table, iterator, rddSchema, nullTypes, batchSize) } } From 708036c1de52d674ceff30ac465e1dcedeb8dde8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 21 Aug 2015 08:41:36 -0500 Subject: [PATCH 041/232] [SPARK-9439] [YARN] External shuffle service robust to NM restarts using leveldb https://issues.apache.org/jira/browse/SPARK-9439 In general, Yarn apps should be robust to NodeManager restarts. However, if you run spark with the external shuffle service on, after a NM restart all shuffles fail, b/c the shuffle service has lost some state with info on each executor. (Note the shuffle data is perfectly fine on disk across a NM restart, the problem is we've lost the small bit of state that lets us *find* those files.) The solution proposed here is that the external shuffle service can write out its state to leveldb (backed by a local file) every time an executor is added. When running with yarn, that file is in the NM's local dir. Whenever the service is started, it looks for that file, and if it exists, it reads the file and re-registers all executors there. Nothing is changed in non-yarn modes with this patch. The service is not given a place to save the state to, so it operates the same as before. This should make it easy to update other cluster managers as well, by just supplying the right file & the equivalent of yarn's `initializeApplication` -- I'm not familiar enough with those modes to know how to do that. Author: Imran Rashid Closes #7943 from squito/leveldb_external_shuffle_service_NM_restart and squashes the following commits: 0d285d3 [Imran Rashid] review feedback 70951d6 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart 5c71c8c [Imran Rashid] save executor to db before registering; style 2499c8c [Imran Rashid] explicit dependency on jackson-annotations 795d28f [Imran Rashid] review feedback 81f80e2 [Imran Rashid] Merge branch 'master' into leveldb_external_shuffle_service_NM_restart 594d520 [Imran Rashid] use json to serialize application executor info 1a7980b [Imran Rashid] version 8267d2a [Imran Rashid] style e9f99e8 [Imran Rashid] cleanup the handling of bad dbs a little 9378ba3 [Imran Rashid] fail gracefully on corrupt leveldb files acedb62 [Imran Rashid] switch to writing out one record per executor 79922b7 [Imran Rashid] rely on yarn to call stopApplication; assorted cleanup 12b6a35 [Imran Rashid] save registered executors when apps are removed; add tests c878fbe [Imran Rashid] better explanation of shuffle service port handling 694934c [Imran Rashid] only open leveldb connection once per service d596410 [Imran Rashid] store executor data in leveldb 59800b7 [Imran Rashid] Files.move in case renaming is unsupported 32fe5ae [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart d7450f0 [Imran Rashid] style f729e2b [Imran Rashid] debugging 4492835 [Imran Rashid] lol, dont use a PrintWriter b/c of scalastyle checks 0a39b98 [Imran Rashid] Merge branch 'master' into external_shuffle_service_NM_restart 55f49fc [Imran Rashid] make sure the service doesnt die if the registered executor file is corrupt; add tests 245db19 [Imran Rashid] style 62586a6 [Imran Rashid] just serialize the whole executors map bdbbf0d [Imran Rashid] comments, remove some unnecessary changes 857331a [Imran Rashid] better tests & comments bb9d1e6 [Imran Rashid] formatting bdc4b32 [Imran Rashid] rename 86e0cb9 [Imran Rashid] for tests, shuffle service finds an open port 23994ff [Imran Rashid] style 7504de8 [Imran Rashid] style a36729c [Imran Rashid] cleanup efb6195 [Imran Rashid] proper unit test, and no longer leak if apps stop during NM restart dd93dc0 [Imran Rashid] test for shuffle service w/ NM restarts d596969 [Imran Rashid] cleanup imports 0e9d69b [Imran Rashid] better names 9eae119 [Imran Rashid] cleanup lots of duplication 1136f44 [Imran Rashid] test needs to have an actual shuffle 0b588bd [Imran Rashid] more fixes ... ad122ef [Imran Rashid] more fixes 5e5a7c3 [Imran Rashid] fix build c69f46b [Imran Rashid] maybe working version, needs tests & cleanup ... bb3ba49 [Imran Rashid] minor cleanup 36127d3 [Imran Rashid] wip b9d2ced [Imran Rashid] incomplete setup for external shuffle service tests --- .../spark/deploy/ExternalShuffleService.scala | 2 +- .../mesos/MesosExternalShuffleService.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 14 +- .../spark/ExternalShuffleServiceSuite.scala | 2 +- network/shuffle/pom.xml | 16 ++ .../shuffle/ExternalShuffleBlockHandler.java | 37 ++- .../shuffle/ExternalShuffleBlockResolver.java | 225 +++++++++++++++-- .../shuffle/protocol/ExecutorShuffleInfo.java | 8 +- .../ExternalShuffleBlockResolverSuite.java | 35 ++- .../shuffle/ExternalShuffleCleanupSuite.java | 9 +- .../ExternalShuffleIntegrationSuite.java | 2 +- .../shuffle/ExternalShuffleSecuritySuite.java | 5 +- .../network/yarn/YarnShuffleService.java | 62 ++++- pom.xml | 5 + yarn/pom.xml | 6 + .../deploy/yarn/BaseYarnClusterSuite.scala | 193 +++++++++++++++ .../spark/deploy/yarn/YarnClusterSuite.scala | 173 +------------ .../yarn/YarnShuffleIntegrationSuite.scala | 109 ++++++++ .../network/shuffle/ShuffleTestAccessor.scala | 71 ++++++ .../yarn/YarnShuffleServiceSuite.scala | 233 ++++++++++++++++++ .../spark/network/yarn/YarnTestAccessor.scala | 37 +++ 21 files changed, 1031 insertions(+), 215 deletions(-) create mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala create mode 100644 yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala create mode 100644 yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala create mode 100644 yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala create mode 100644 yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index 20a9faa1784b7..22ef701d833b2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -53,7 +53,7 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana /** Create a new shuffle block handler. Factored out for subclasses to override. */ protected def newShuffleBlockHandler(conf: TransportConf): ExternalShuffleBlockHandler = { - new ExternalShuffleBlockHandler(conf) + new ExternalShuffleBlockHandler(conf, null) } /** Starts the external shuffle service if the user has configured us to. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala index 061857476a8a0..12337a940a414 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala @@ -34,7 +34,7 @@ import org.apache.spark.network.util.TransportConf * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. */ private[mesos] class MesosExternalShuffleBlockHandler(transportConf: TransportConf) - extends ExternalShuffleBlockHandler(transportConf) with Logging { + extends ExternalShuffleBlockHandler(transportConf, null) with Logging { // Stores a map of driver socket addresses to app ids private val connectedApps = new mutable.HashMap[SocketAddress, String] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index eedb27942e841..fefaef0ab82c8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -93,8 +93,17 @@ private[spark] class BlockManager( // Port used by the external shuffle service. In Yarn mode, this may be already be // set through the Hadoop configuration as the server is launched in the Yarn NM. - private val externalShuffleServicePort = - Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + private val externalShuffleServicePort = { + val tmpPort = Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + if (tmpPort == 0) { + // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds + // an open port. But we still need to tell our spark apps the right port to use. So + // only if the yarn config has the port set to 0, we prefer the value in the spark config + conf.get("spark.shuffle.service.port").toInt + } else { + tmpPort + } + } // Check that we're not using external shuffle service with consolidated shuffle files. if (externalShuffleServiceEnabled @@ -191,6 +200,7 @@ private[spark] class BlockManager( executorId, blockTransferService.hostName, blockTransferService.port) shuffleServerId = if (externalShuffleServiceEnabled) { + logInfo(s"external shuffle service port = $externalShuffleServicePort") BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) } else { blockManagerId diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index c38d70252add1..e846a72c888c6 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -36,7 +36,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { override def beforeAll() { val transportConf = SparkTransportConf.fromSparkConf(conf, numUsableCores = 2) - rpcHandler = new ExternalShuffleBlockHandler(transportConf) + rpcHandler = new ExternalShuffleBlockHandler(transportConf, null) val transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index 532463e96fbb7..3d2edf9d94515 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -43,6 +43,22 @@ ${project.version} + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.core + jackson-annotations + + org.slf4j diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index db9dc4f17cee9..0df1dd621f6e9 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -17,11 +17,12 @@ package org.apache.spark.network.shuffle; +import java.io.File; +import java.io.IOException; import java.util.List; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; -import org.apache.spark.network.util.TransportConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,10 +32,10 @@ import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; -import org.apache.spark.network.shuffle.protocol.OpenBlocks; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; -import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; +import org.apache.spark.network.shuffle.protocol.*; +import org.apache.spark.network.util.TransportConf; + /** * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. @@ -46,11 +47,13 @@ public class ExternalShuffleBlockHandler extends RpcHandler { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); - private final ExternalShuffleBlockResolver blockManager; + @VisibleForTesting + final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler(TransportConf conf) { - this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf)); + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { + this(new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } /** Enables mocking out the StreamManager and BlockManager. */ @@ -105,4 +108,22 @@ public StreamManager getStreamManager() { public void applicationRemoved(String appId, boolean cleanupLocalDirs) { blockManager.applicationRemoved(appId, cleanupLocalDirs); } + + /** + * Register an (application, executor) with the given shuffle info. + * + * The "re-" is meant to highlight the intended use of this method -- when this service is + * restarted, this is used to restore the state of executors from before the restart. Normal + * registration will happen via a message handled in receive() + * + * @param appExecId + * @param executorInfo + */ + public void reregisterExecutor(AppExecId appExecId, ExecutorShuffleInfo executorInfo) { + blockManager.registerExecutor(appExecId.appId, appExecId.execId, executorInfo); + } + + public void close() { + blockManager.close(); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 022ed88a16480..79beec4429a99 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -17,19 +17,24 @@ package org.apache.spark.network.shuffle; -import java.io.DataInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; +import java.io.*; +import java.util.*; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.Executors; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; import com.google.common.base.Objects; import com.google.common.collect.Maps; +import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.DBIterator; +import org.iq80.leveldb.Options; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,25 +57,87 @@ public class ExternalShuffleBlockResolver { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockResolver.class); + private static final ObjectMapper mapper = new ObjectMapper(); + /** + * This a common prefix to the key for each app registration we stick in leveldb, so they + * are easy to find, since leveldb lets you search based on prefix. + */ + private static final String APP_KEY_PREFIX = "AppExecShuffleInfo"; + private static final StoreVersion CURRENT_VERSION = new StoreVersion(1, 0); + // Map containing all registered executors' metadata. - private final ConcurrentMap executors; + @VisibleForTesting + final ConcurrentMap executors; // Single-threaded Java executor used to perform expensive recursive directory deletion. private final Executor directoryCleaner; private final TransportConf conf; - public ExternalShuffleBlockResolver(TransportConf conf) { - this(conf, Executors.newSingleThreadExecutor( + @VisibleForTesting + final File registeredExecutorFile; + @VisibleForTesting + final DB db; + + public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorFile) + throws IOException { + this(conf, registeredExecutorFile, Executors.newSingleThreadExecutor( // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-directory-cleaner"))); } // Allows tests to have more control over when directories are cleaned up. @VisibleForTesting - ExternalShuffleBlockResolver(TransportConf conf, Executor directoryCleaner) { + ExternalShuffleBlockResolver( + TransportConf conf, + File registeredExecutorFile, + Executor directoryCleaner) throws IOException { this.conf = conf; - this.executors = Maps.newConcurrentMap(); + this.registeredExecutorFile = registeredExecutorFile; + if (registeredExecutorFile != null) { + Options options = new Options(); + options.createIfMissing(false); + options.logger(new LevelDBLogger()); + DB tmpDb; + try { + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException e) { + if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { + logger.info("Creating state database at " + registeredExecutorFile); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } else { + // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new + // one, so we can keep processing new apps + logger.error("error opening leveldb file {}. Creating new file, will not be able to " + + "recover state for existing applications", registeredExecutorFile, e); + if (registeredExecutorFile.isDirectory()) { + for (File f : registeredExecutorFile.listFiles()) { + f.delete(); + } + } + registeredExecutorFile.delete(); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(registeredExecutorFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + + } + } + // if there is a version mismatch, we throw an exception, which means the service is unusable + checkVersion(tmpDb); + executors = reloadRegisteredExecutors(tmpDb); + db = tmpDb; + } else { + db = null; + executors = Maps.newConcurrentMap(); + } this.directoryCleaner = directoryCleaner; } @@ -81,6 +148,15 @@ public void registerExecutor( ExecutorShuffleInfo executorInfo) { AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); + try { + if (db != null) { + byte[] key = dbAppExecKey(fullId); + byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); + db.put(key, value); + } + } catch (Exception e) { + logger.error("Error saving registered executors", e); + } executors.put(fullId, executorInfo); } @@ -136,6 +212,13 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { // Only touch executors associated with the appId that was removed. if (appId.equals(fullId.appId)) { it.remove(); + if (db != null) { + try { + db.delete(dbAppExecKey(fullId)); + } catch (IOException e) { + logger.error("Error deleting {} from executor state db", appId, e); + } + } if (cleanupLocalDirs) { logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length); @@ -220,12 +303,23 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) return new File(new File(localDir, String.format("%02x", subDirId)), filename); } + void close() { + if (db != null) { + try { + db.close(); + } catch (IOException e) { + logger.error("Exception closing leveldb with registered executors", e); + } + } + } + /** Simply encodes an executor's full ID, which is appId + execId. */ - private static class AppExecId { - final String appId; - final String execId; + public static class AppExecId { + public final String appId; + public final String execId; - private AppExecId(String appId, String execId) { + @JsonCreator + public AppExecId(@JsonProperty("appId") String appId, @JsonProperty("execId") String execId) { this.appId = appId; this.execId = execId; } @@ -252,4 +346,105 @@ public String toString() { .toString(); } } + + private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { + // we stick a common prefix on all the keys so we can find them in the DB + String appExecJson = mapper.writeValueAsString(appExecId); + String key = (APP_KEY_PREFIX + ";" + appExecJson); + return key.getBytes(Charsets.UTF_8); + } + + private static AppExecId parseDbAppExecKey(String s) throws IOException { + if (!s.startsWith(APP_KEY_PREFIX)) { + throw new IllegalArgumentException("expected a string starting with " + APP_KEY_PREFIX); + } + String json = s.substring(APP_KEY_PREFIX.length() + 1); + AppExecId parsed = mapper.readValue(json, AppExecId.class); + return parsed; + } + + @VisibleForTesting + static ConcurrentMap reloadRegisteredExecutors(DB db) + throws IOException { + ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); + if (db != null) { + DBIterator itr = db.iterator(); + itr.seek(APP_KEY_PREFIX.getBytes(Charsets.UTF_8)); + while (itr.hasNext()) { + Map.Entry e = itr.next(); + String key = new String(e.getKey(), Charsets.UTF_8); + if (!key.startsWith(APP_KEY_PREFIX)) { + break; + } + AppExecId id = parseDbAppExecKey(key); + ExecutorShuffleInfo shuffleInfo = mapper.readValue(e.getValue(), ExecutorShuffleInfo.class); + registeredExecutors.put(id, shuffleInfo); + } + } + return registeredExecutors; + } + + private static class LevelDBLogger implements org.iq80.leveldb.Logger { + private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); + + @Override + public void log(String message) { + LOG.info(message); + } + } + + /** + * Simple major.minor versioning scheme. Any incompatible changes should be across major + * versions. Minor version differences are allowed -- meaning we should be able to read + * dbs that are either earlier *or* later on the minor version. + */ + private static void checkVersion(DB db) throws IOException { + byte[] bytes = db.get(StoreVersion.KEY); + if (bytes == null) { + storeVersion(db); + } else { + StoreVersion version = mapper.readValue(bytes, StoreVersion.class); + if (version.major != CURRENT_VERSION.major) { + throw new IOException("cannot read state DB with version " + version + ", incompatible " + + "with current version " + CURRENT_VERSION); + } + storeVersion(db); + } + } + + private static void storeVersion(DB db) throws IOException { + db.put(StoreVersion.KEY, mapper.writeValueAsBytes(CURRENT_VERSION)); + } + + + public static class StoreVersion { + + final static byte[] KEY = "StoreVersion".getBytes(Charsets.UTF_8); + + public final int major; + public final int minor; + + @JsonCreator public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { + this.major = major; + this.minor = minor; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StoreVersion that = (StoreVersion) o; + + return major == that.major && minor == that.minor; + } + + @Override + public int hashCode() { + int result = major; + result = 31 * result + minor; + return result; + } + } + } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index cadc8e8369c6a..102d4efb8bf3b 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -19,6 +19,8 @@ import java.util.Arrays; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; @@ -34,7 +36,11 @@ public class ExecutorShuffleInfo implements Encodable { /** Shuffle manager (SortShuffleManager or HashShuffleManager) that the executor is using. */ public final String shuffleManager; - public ExecutorShuffleInfo(String[] localDirs, int subDirsPerLocalDir, String shuffleManager) { + @JsonCreator + public ExecutorShuffleInfo( + @JsonProperty("localDirs") String[] localDirs, + @JsonProperty("subDirsPerLocalDir") int subDirsPerLocalDir, + @JsonProperty("shuffleManager") String shuffleManager) { this.localDirs = localDirs; this.subDirsPerLocalDir = subDirsPerLocalDir; this.shuffleManager = shuffleManager; diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index d02f4f0fdb682..3c6cb367dea46 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -21,9 +21,12 @@ import java.io.InputStream; import java.io.InputStreamReader; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CharStreams; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -59,8 +62,8 @@ public static void afterAll() { } @Test - public void testBadRequests() { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + public void testBadRequests() throws IOException { + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); // Unregistered executor try { resolver.getBlockData("app0", "exec1", "shuffle_1_1_0"); @@ -91,7 +94,7 @@ public void testBadRequests() { @Test public void testSortShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); @@ -110,7 +113,7 @@ public void testSortShuffleBlocks() throws IOException { @Test public void testHashShuffleBlocks() throws IOException { - ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf); + ExternalShuffleBlockResolver resolver = new ExternalShuffleBlockResolver(conf, null); resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); @@ -126,4 +129,28 @@ public void testHashShuffleBlocks() throws IOException { block1Stream.close(); assertEquals(hashBlock1, block1); } + + @Test + public void jsonSerializationOfExecutorRegistration() throws IOException { + ObjectMapper mapper = new ObjectMapper(); + AppExecId appId = new AppExecId("foo", "bar"); + String appIdJson = mapper.writeValueAsString(appId); + AppExecId parsedAppId = mapper.readValue(appIdJson, AppExecId.class); + assertEquals(parsedAppId, appId); + + ExecutorShuffleInfo shuffleInfo = + new ExecutorShuffleInfo(new String[]{"/bippy", "/flippy"}, 7, "hash"); + String shuffleJson = mapper.writeValueAsString(shuffleInfo); + ExecutorShuffleInfo parsedShuffleInfo = + mapper.readValue(shuffleJson, ExecutorShuffleInfo.class); + assertEquals(parsedShuffleInfo, shuffleInfo); + + // Intentionally keep these hard-coded strings in here, to check backwards-compatability. + // its not legacy yet, but keeping this here in case anybody changes it + String legacyAppIdJson = "{\"appId\":\"foo\", \"execId\":\"bar\"}"; + assertEquals(appId, mapper.readValue(legacyAppIdJson, AppExecId.class)); + String legacyShuffleJson = "{\"localDirs\": [\"/bippy\", \"/flippy\"], " + + "\"subDirsPerLocalDir\": 7, \"shuffleManager\": \"hash\"}"; + assertEquals(shuffleInfo, mapper.readValue(legacyShuffleJson, ExecutorShuffleInfo.class)); + } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index d9d9c1bf2f17a..2f4f1d0df478b 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -42,7 +42,7 @@ public void noCleanupAndCleanup() throws IOException { TestShuffleDataContext dataContext = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); resolver.applicationRemoved("app", false /* cleanup */); @@ -65,7 +65,8 @@ public void cleanupUsesExecutor() throws IOException { @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } }; - ExternalShuffleBlockResolver manager = new ExternalShuffleBlockResolver(conf, noThreadExecutor); + ExternalShuffleBlockResolver manager = + new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); manager.applicationRemoved("app", true); @@ -83,7 +84,7 @@ public void cleanupMultipleExecutors() throws IOException { TestShuffleDataContext dataContext1 = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); @@ -99,7 +100,7 @@ public void cleanupOnlyRemovedApp() throws IOException { TestShuffleDataContext dataContext1 = createSomeData(); ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, sameThreadExecutor); + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); resolver.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); resolver.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 39aa49911d9cb..a3f9a38b1aeb9 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -92,7 +92,7 @@ public static void beforeAll() throws IOException { dataContext1.insertHashShuffleData(1, 0, exec1Blocks); conf = new TransportConf(new SystemPropertyConfigProvider()); - handler = new ExternalShuffleBlockHandler(conf); + handler = new ExternalShuffleBlockHandler(conf, null); TransportContext transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index d4ec1956c1e29..aa99efda94948 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -43,8 +43,9 @@ public class ExternalShuffleSecuritySuite { TransportServer server; @Before - public void beforeEach() { - TransportContext context = new TransportContext(conf, new ExternalShuffleBlockHandler(conf)); + public void beforeEach() throws IOException { + TransportContext context = + new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, new TestSecretKeyHolder("my-app-id", "secret")); this.server = context.createServer(Arrays.asList(bootstrap)); diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 463f99ef3352d..11ea7f3fd3cfe 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -17,25 +17,21 @@ package org.apache.spark.network.yarn; +import java.io.File; import java.nio.ByteBuffer; import java.util.List; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.server.api.AuxiliaryService; -import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; -import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; -import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; -import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; +import org.apache.hadoop.yarn.server.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; import org.apache.spark.network.sasl.SaslServerBootstrap; import org.apache.spark.network.sasl.ShuffleSecretManager; -import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; @@ -79,11 +75,26 @@ public class YarnShuffleService extends AuxiliaryService { private TransportServer shuffleServer = null; // Handles registering executors and opening shuffle blocks - private ExternalShuffleBlockHandler blockHandler; + @VisibleForTesting + ExternalShuffleBlockHandler blockHandler; + + // Where to store & reload executor info for recovering state after an NM restart + @VisibleForTesting + File registeredExecutorFile; + + // just for testing when you want to find an open port + @VisibleForTesting + static int boundPort = -1; + + // just for integration tests that want to look at this file -- in general not sensible as + // a static + @VisibleForTesting + static YarnShuffleService instance; public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); + instance = this; } /** @@ -100,11 +111,24 @@ private boolean isAuthenticationEnabled() { */ @Override protected void serviceInit(Configuration conf) { + + // In case this NM was killed while there were running spark applications, we need to restore + // lost state for the existing executors. We look for an existing file in the NM's local dirs. + // If we don't find one, then we choose a file to use to save the state next time. Even if + // an application was stopped while the NM was down, we expect yarn to call stopApplication() + // when it comes back + registeredExecutorFile = + findRegisteredExecutorFile(conf.getStrings("yarn.nodemanager.local-dirs")); + TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); - blockHandler = new ExternalShuffleBlockHandler(transportConf); + try { + blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); + } catch (Exception e) { + logger.error("Failed to initialize external shuffle service", e); + } List bootstraps = Lists.newArrayList(); if (authEnabled) { @@ -116,9 +140,13 @@ protected void serviceInit(Configuration conf) { SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); TransportContext transportContext = new TransportContext(transportConf, blockHandler); shuffleServer = transportContext.createServer(port, bootstraps); + // the port should normally be fixed, but for tests its useful to find an open port + port = shuffleServer.getPort(); + boundPort = port; String authEnabledString = authEnabled ? "enabled" : "not enabled"; logger.info("Started YARN shuffle service for Spark on port {}. " + - "Authentication is {}.", port, authEnabledString); + "Authentication is {}. Registered executor file is {}", port, authEnabledString, + registeredExecutorFile); } @Override @@ -161,6 +189,16 @@ public void stopContainer(ContainerTerminationContext context) { logger.info("Stopping container {}", containerId); } + private File findRegisteredExecutorFile(String[] localDirs) { + for (String dir: localDirs) { + File f = new File(dir, "registeredExecutors.ldb"); + if (f.exists()) { + return f; + } + } + return new File(localDirs[0], "registeredExecutors.ldb"); + } + /** * Close the shuffle server to clean up any associated state. */ @@ -170,6 +208,9 @@ protected void serviceStop() { if (shuffleServer != null) { shuffleServer.close(); } + if (blockHandler != null) { + blockHandler.close(); + } } catch (Exception e) { logger.error("Exception when stopping service", e); } @@ -180,5 +221,4 @@ protected void serviceStop() { public ByteBuffer getMetaData() { return ByteBuffer.allocate(0); } - } diff --git a/pom.xml b/pom.xml index ccfa1ea19b21e..d5945f2546d38 100644 --- a/pom.xml +++ b/pom.xml @@ -655,6 +655,11 @@ jackson-databind ${fasterxml.jackson.version} + + com.fasterxml.jackson.core + jackson-annotations + ${fasterxml.jackson.version} + diff --git a/yarn/pom.xml b/yarn/pom.xml index 15db54e4e7909..f6737695307a2 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -38,6 +38,12 @@ spark-core_${scala.binary.version} ${project.version} + + org.apache.spark + spark-network-yarn_${scala.binary.version} + ${project.version} + test + org.apache.spark spark-core_${scala.binary.version} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala new file mode 100644 index 0000000000000..128e996b71fe5 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -0,0 +1,193 @@ +/* + * 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.deploy.yarn + +import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.util.Properties +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConversions._ + +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.Files +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster +import org.scalatest.{BeforeAndAfterAll, Matchers} + +import org.apache.spark._ +import org.apache.spark.util.Utils + +abstract class BaseYarnClusterSuite + extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { + + // log4j configuration for the YARN containers, so that their output is collected + // by YARN instead of trying to overwrite unit-tests.log. + protected val LOG4J_CONF = """ + |log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + """.stripMargin + + private var yarnCluster: MiniYARNCluster = _ + protected var tempDir: File = _ + private var fakeSparkJar: File = _ + private var hadoopConfDir: File = _ + private var logConfDir: File = _ + + + def yarnConfig: YarnConfiguration + + override def beforeAll() { + super.beforeAll() + + tempDir = Utils.createTempDir() + logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + System.setProperty("SPARK_YARN_MODE", "true") + + val logConfFile = new File(logConfDir, "log4j.properties") + Files.write(LOG4J_CONF, logConfFile, UTF_8) + + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + yarnCluster.init(yarnConfig) + yarnCluster.start() + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + + fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) + assert(hadoopConfDir.mkdir()) + File.createTempFile("token", ".txt", hadoopConfDir) + } + + override def afterAll() { + yarnCluster.stop() + System.clearProperty("SPARK_YARN_MODE") + super.afterAll() + } + + protected def runSpark( + clientMode: Boolean, + klass: String, + appArgs: Seq[String] = Nil, + sparkArgs: Seq[String] = Nil, + extraClassPath: Seq[String] = Nil, + extraJars: Seq[String] = Nil, + extraConf: Map[String, String] = Map()): Unit = { + val master = if (clientMode) "yarn-client" else "yarn-cluster" + val props = new Properties() + + props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) + + val childClasspath = logConfDir.getAbsolutePath() + + File.pathSeparator + + sys.props("java.class.path") + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator) + props.setProperty("spark.driver.extraClassPath", childClasspath) + props.setProperty("spark.executor.extraClassPath", childClasspath) + + // SPARK-4267: make sure java options are propagated correctly. + props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") + props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") + + yarnCluster.getConfig().foreach { e => + props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) + } + + sys.props.foreach { case (k, v) => + if (k.startsWith("spark.")) { + props.setProperty(k, v) + } + } + + extraConf.foreach { case (k, v) => props.setProperty(k, v) } + + val propsFile = File.createTempFile("spark", ".properties", tempDir) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + props.store(writer, "Spark properties.") + writer.close() + + val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil + val mainArgs = + if (klass.endsWith(".py")) { + Seq(klass) + } else { + Seq("--class", klass, fakeSparkJar.getAbsolutePath()) + } + val argv = + Seq( + new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), + "--master", master, + "--num-executors", "1", + "--properties-file", propsFile.getAbsolutePath()) ++ + extraJarArgs ++ + sparkArgs ++ + mainArgs ++ + appArgs + + Utils.executeAndGetOutput(argv, + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) + } + + /** + * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide + * any sort of error when the job process finishes successfully, but the job itself fails. So + * the tests enforce that something is written to a file after everything is ok to indicate + * that the job succeeded. + */ + protected def checkResult(result: File): Unit = { + checkResult(result, "success") + } + + protected def checkResult(result: File, expected: String): Unit = { + val resultString = Files.toString(result, UTF_8) + resultString should be (expected) + } + + protected def mainClassName(klass: Class[_]): String = { + klass.getName().stripSuffix("$") + } + +} diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index eb6e1fd370620..128350b648992 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -17,25 +17,20 @@ package org.apache.spark.deploy.yarn -import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.io.File import java.net.URL -import java.util.Properties -import java.util.concurrent.TimeUnit -import scala.collection.JavaConversions._ import scala.collection.mutable +import scala.collection.JavaConversions._ import com.google.common.base.Charsets.UTF_8 -import com.google.common.io.ByteStreams -import com.google.common.io.Files +import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.{BeforeAndAfterAll, Matchers} +import org.scalatest.Matchers import org.apache.spark._ +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** @@ -43,17 +38,9 @@ import org.apache.spark.util.Utils * applications, and require the Spark assembly to be built before they can be successfully * run. */ -class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { - - // log4j configuration for the YARN containers, so that their output is collected - // by YARN instead of trying to overwrite unit-tests.log. - private val LOG4J_CONF = """ - |log4j.rootCategory=DEBUG, console - |log4j.appender.console=org.apache.log4j.ConsoleAppender - |log4j.appender.console.target=System.err - |log4j.appender.console.layout=org.apache.log4j.PatternLayout - |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - """.stripMargin +class YarnClusterSuite extends BaseYarnClusterSuite { + + override def yarnConfig: YarnConfiguration = new YarnConfiguration() private val TEST_PYFILE = """ |import mod1, mod2 @@ -82,65 +69,6 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher | return 42 """.stripMargin - private var yarnCluster: MiniYARNCluster = _ - private var tempDir: File = _ - private var fakeSparkJar: File = _ - private var hadoopConfDir: File = _ - private var logConfDir: File = _ - - override def beforeAll() { - super.beforeAll() - - tempDir = Utils.createTempDir() - logConfDir = new File(tempDir, "log4j") - logConfDir.mkdir() - System.setProperty("SPARK_YARN_MODE", "true") - - val logConfFile = new File(logConfDir, "log4j.properties") - Files.write(LOG4J_CONF, logConfFile, UTF_8) - - yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) - yarnCluster.init(new YarnConfiguration()) - yarnCluster.start() - - // There's a race in MiniYARNCluster in which start() may return before the RM has updated - // its address in the configuration. You can see this in the logs by noticing that when - // MiniYARNCluster prints the address, it still has port "0" assigned, although later the - // test works sometimes: - // - // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 - // - // That log message prints the contents of the RM_ADDRESS config variable. If you check it - // later on, it looks something like this: - // - // INFO YarnClusterSuite: RM address in configuration is blah:42631 - // - // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't - // done so in a timely manner (defined to be 10 seconds). - val config = yarnCluster.getConfig() - val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) - while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { - if (System.currentTimeMillis() > deadline) { - throw new IllegalStateException("Timed out waiting for RM to come up.") - } - logDebug("RM address still not set in configuration, waiting...") - TimeUnit.MILLISECONDS.sleep(100) - } - - logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") - - fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) - hadoopConfDir = new File(tempDir, Client.LOCALIZED_CONF_DIR) - assert(hadoopConfDir.mkdir()) - File.createTempFile("token", ".txt", hadoopConfDir) - } - - override def afterAll() { - yarnCluster.stop() - System.clearProperty("SPARK_YARN_MODE") - super.afterAll() - } - test("run Spark in yarn-client mode") { testBasicYarnApp(true) } @@ -174,7 +102,7 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher } private def testBasicYarnApp(clientMode: Boolean): Unit = { - var result = File.createTempFile("result", null, tempDir) + val result = File.createTempFile("result", null, tempDir) runSpark(clientMode, mainClassName(YarnClusterDriver.getClass), appArgs = Seq(result.getAbsolutePath())) checkResult(result) @@ -224,89 +152,6 @@ class YarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matcher checkResult(executorResult, "OVERRIDDEN") } - private def runSpark( - clientMode: Boolean, - klass: String, - appArgs: Seq[String] = Nil, - sparkArgs: Seq[String] = Nil, - extraClassPath: Seq[String] = Nil, - extraJars: Seq[String] = Nil, - extraConf: Map[String, String] = Map()): Unit = { - val master = if (clientMode) "yarn-client" else "yarn-cluster" - val props = new Properties() - - props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) - - val childClasspath = logConfDir.getAbsolutePath() + - File.pathSeparator + - sys.props("java.class.path") + - File.pathSeparator + - extraClassPath.mkString(File.pathSeparator) - props.setProperty("spark.driver.extraClassPath", childClasspath) - props.setProperty("spark.executor.extraClassPath", childClasspath) - - // SPARK-4267: make sure java options are propagated correctly. - props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") - props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") - - yarnCluster.getConfig().foreach { e => - props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) - } - - sys.props.foreach { case (k, v) => - if (k.startsWith("spark.")) { - props.setProperty(k, v) - } - } - - extraConf.foreach { case (k, v) => props.setProperty(k, v) } - - val propsFile = File.createTempFile("spark", ".properties", tempDir) - val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) - props.store(writer, "Spark properties.") - writer.close() - - val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil - val mainArgs = - if (klass.endsWith(".py")) { - Seq(klass) - } else { - Seq("--class", klass, fakeSparkJar.getAbsolutePath()) - } - val argv = - Seq( - new File(sys.props("spark.test.home"), "bin/spark-submit").getAbsolutePath(), - "--master", master, - "--num-executors", "1", - "--properties-file", propsFile.getAbsolutePath()) ++ - extraJarArgs ++ - sparkArgs ++ - mainArgs ++ - appArgs - - Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) - } - - /** - * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide - * any sort of error when the job process finishes successfully, but the job itself fails. So - * the tests enforce that something is written to a file after everything is ok to indicate - * that the job succeeded. - */ - private def checkResult(result: File): Unit = { - checkResult(result, "success") - } - - private def checkResult(result: File, expected: String): Unit = { - var resultString = Files.toString(result, UTF_8) - resultString should be (expected) - } - - private def mainClassName(klass: Class[_]): String = { - klass.getName().stripSuffix("$") - } - } private[spark] class SaveExecutorInfo extends SparkListener { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala new file mode 100644 index 0000000000000..5e8238822b90a --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -0,0 +1,109 @@ +/* +* 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.deploy.yarn + +import java.io.File + +import com.google.common.base.Charsets.UTF_8 +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.scalatest.Matchers + +import org.apache.spark._ +import org.apache.spark.network.shuffle.ShuffleTestAccessor +import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} + +/** + * Integration test for the external shuffle service with a yarn mini-cluster + */ +class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { + + override def yarnConfig: YarnConfiguration = { + val yarnConfig = new YarnConfiguration() + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), + classOf[YarnShuffleService].getCanonicalName) + yarnConfig.set("spark.shuffle.service.port", "0") + yarnConfig + } + + test("external shuffle service") { + val shuffleServicePort = YarnTestAccessor.getShuffleServicePort + val shuffleService = YarnTestAccessor.getShuffleServiceInstance + + val registeredExecFile = YarnTestAccessor.getRegisteredExecutorFile(shuffleService) + + logInfo("Shuffle service port = " + shuffleServicePort) + val result = File.createTempFile("result", null, tempDir) + runSpark( + false, + mainClassName(YarnExternalShuffleDriver.getClass), + appArgs = Seq(result.getAbsolutePath(), registeredExecFile.getAbsolutePath), + extraConf = Map( + "spark.shuffle.service.enabled" -> "true", + "spark.shuffle.service.port" -> shuffleServicePort.toString + ) + ) + checkResult(result) + assert(YarnTestAccessor.getRegisteredExecutorFile(shuffleService).exists()) + } +} + +private object YarnExternalShuffleDriver extends Logging with Matchers { + + val WAIT_TIMEOUT_MILLIS = 10000 + + def main(args: Array[String]): Unit = { + if (args.length != 2) { + // scalastyle:off println + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: ExternalShuffleDriver [result file] [registed exec file] + """.stripMargin) + // scalastyle:on println + System.exit(1) + } + + val sc = new SparkContext(new SparkConf() + .setAppName("External Shuffle Test")) + val conf = sc.getConf + val status = new File(args(0)) + val registeredExecFile = new File(args(1)) + logInfo("shuffle service executor file = " + registeredExecFile) + var result = "failure" + val execStateCopy = new File(registeredExecFile.getAbsolutePath + "_dup") + try { + val data = sc.parallelize(0 until 100, 10).map { x => (x % 10) -> x }.reduceByKey{ _ + _ }. + collect().toSet + sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) + data should be ((0 until 10).map{x => x -> (x * 10 + 450)}.toSet) + result = "success" + // only one process can open a leveldb file at a time, so we copy the files + FileUtils.copyDirectory(registeredExecFile, execStateCopy) + assert(!ShuffleTestAccessor.reloadRegisteredExecutors(execStateCopy).isEmpty) + } finally { + sc.stop() + FileUtils.deleteDirectory(execStateCopy) + Files.write(result, status, UTF_8) + } + } + +} diff --git a/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala new file mode 100644 index 0000000000000..aa46ec5100f0e --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/shuffle/ShuffleTestAccessor.scala @@ -0,0 +1,71 @@ +/* + * 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.network.shuffle + +import java.io.{IOException, File} +import java.util.concurrent.ConcurrentMap + +import com.google.common.annotations.VisibleForTesting +import org.apache.hadoop.yarn.api.records.ApplicationId +import org.fusesource.leveldbjni.JniDBFactory +import org.iq80.leveldb.{DB, Options} + +import org.apache.spark.network.shuffle.ExternalShuffleBlockResolver.AppExecId +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo + +/** + * just a cheat to get package-visible members in tests + */ +object ShuffleTestAccessor { + + def getBlockResolver(handler: ExternalShuffleBlockHandler): ExternalShuffleBlockResolver = { + handler.blockManager + } + + def getExecutorInfo( + appId: ApplicationId, + execId: String, + resolver: ExternalShuffleBlockResolver + ): Option[ExecutorShuffleInfo] = { + val id = new AppExecId(appId.toString, execId) + Option(resolver.executors.get(id)) + } + + def registeredExecutorFile(resolver: ExternalShuffleBlockResolver): File = { + resolver.registeredExecutorFile + } + + def shuffleServiceLevelDB(resolver: ExternalShuffleBlockResolver): DB = { + resolver.db + } + + def reloadRegisteredExecutors( + file: File): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = { + val options: Options = new Options + options.createIfMissing(true) + val factory = new JniDBFactory + val db = factory.open(file, options) + val result = ExternalShuffleBlockResolver.reloadRegisteredExecutors(db) + db.close() + result + } + + def reloadRegisteredExecutors( + db: DB): ConcurrentMap[ExternalShuffleBlockResolver.AppExecId, ExecutorShuffleInfo] = { + ExternalShuffleBlockResolver.reloadRegisteredExecutors(db) + } +} diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala new file mode 100644 index 0000000000000..2f22cbdbeac37 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -0,0 +1,233 @@ +/* + * 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.network.yarn + +import java.io.{DataOutputStream, File, FileOutputStream} + +import scala.annotation.tailrec + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.yarn.api.records.ApplicationId +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.api.{ApplicationInitializationContext, ApplicationTerminationContext} +import org.scalatest.{BeforeAndAfterEach, Matchers} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.network.shuffle.ShuffleTestAccessor +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo + +class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { + private[yarn] var yarnConfig: YarnConfiguration = new YarnConfiguration + + override def beforeEach(): Unit = { + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") + yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), + classOf[YarnShuffleService].getCanonicalName) + + yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => + val d = new File(dir) + if (d.exists()) { + FileUtils.deleteDirectory(d) + } + FileUtils.forceMkdir(d) + logInfo(s"creating yarn.nodemanager.local-dirs: $d") + } + } + + var s1: YarnShuffleService = null + var s2: YarnShuffleService = null + var s3: YarnShuffleService = null + + override def afterEach(): Unit = { + if (s1 != null) { + s1.stop() + s1 = null + } + if (s2 != null) { + s2.stop() + s2 = null + } + if (s3 != null) { + s3.stop() + s3 = null + } + } + + test("executor state kept across NM restart") { + s1 = new YarnShuffleService + s1.init(yarnConfig) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + s1.initializeApplication(app1Data) + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + s1.initializeApplication(app2Data) + + val execStateFile = s1.registeredExecutorFile + execStateFile should not be (null) + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") + + val blockHandler = s1.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", blockResolver) should + be (Some(shuffleInfo1)) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", blockResolver) should + be (Some(shuffleInfo2)) + + if (!execStateFile.exists()) { + @tailrec def findExistingParent(file: File): File = { + if (file == null) file + else if (file.exists()) file + else findExistingParent(file.getParentFile()) + } + val existingParent = findExistingParent(execStateFile) + assert(false, s"$execStateFile does not exist -- closest existing parent is $existingParent") + } + assert(execStateFile.exists(), s"$execStateFile did not exist") + + // now we pretend the shuffle service goes down, and comes back up + s1.stop() + s2 = new YarnShuffleService + s2.init(yarnConfig) + s2.registeredExecutorFile should be (execStateFile) + + val handler2 = s2.blockHandler + val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) + + // now we reinitialize only one of the apps, and expect yarn to tell us that app2 was stopped + // during the restart + s2.initializeApplication(app1Data) + s2.stopApplication(new ApplicationTerminationContext(app2Id)) + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver2) should be (Some(shuffleInfo1)) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (None) + + // Act like the NM restarts one more time + s2.stop() + s3 = new YarnShuffleService + s3.init(yarnConfig) + s3.registeredExecutorFile should be (execStateFile) + + val handler3 = s3.blockHandler + val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) + + // app1 is still running + s3.initializeApplication(app1Data) + ShuffleTestAccessor.getExecutorInfo(app1Id, "exec-1", resolver3) should be (Some(shuffleInfo1)) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (None) + s3.stop() + } + + test("removed applications should not be in registered executor file") { + s1 = new YarnShuffleService + s1.init(yarnConfig) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + s1.initializeApplication(app1Data) + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + s1.initializeApplication(app2Data) + + val execStateFile = s1.registeredExecutorFile + execStateFile should not be (null) + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") + + val blockHandler = s1.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + blockResolver.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + + val db = ShuffleTestAccessor.shuffleServiceLevelDB(blockResolver) + ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty + + s1.stopApplication(new ApplicationTerminationContext(app1Id)) + ShuffleTestAccessor.reloadRegisteredExecutors(db) should not be empty + s1.stopApplication(new ApplicationTerminationContext(app2Id)) + ShuffleTestAccessor.reloadRegisteredExecutors(db) shouldBe empty + } + + test("shuffle service should be robust to corrupt registered executor file") { + s1 = new YarnShuffleService + s1.init(yarnConfig) + val app1Id = ApplicationId.newInstance(0, 1) + val app1Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app1Id, null) + s1.initializeApplication(app1Data) + + val execStateFile = s1.registeredExecutorFile + val shuffleInfo1 = new ExecutorShuffleInfo(Array("/foo", "/bar"), 3, "sort") + + val blockHandler = s1.blockHandler + val blockResolver = ShuffleTestAccessor.getBlockResolver(blockHandler) + ShuffleTestAccessor.registeredExecutorFile(blockResolver) should be (execStateFile) + + blockResolver.registerExecutor(app1Id.toString, "exec-1", shuffleInfo1) + + // now we pretend the shuffle service goes down, and comes back up. But we'll also + // make a corrupt registeredExecutor File + s1.stop() + + execStateFile.listFiles().foreach{_.delete()} + + val out = new DataOutputStream(new FileOutputStream(execStateFile + "/CURRENT")) + out.writeInt(42) + out.close() + + s2 = new YarnShuffleService + s2.init(yarnConfig) + s2.registeredExecutorFile should be (execStateFile) + + val handler2 = s2.blockHandler + val resolver2 = ShuffleTestAccessor.getBlockResolver(handler2) + + // we re-initialize app1, but since the file was corrupt there is nothing we can do about it ... + s2.initializeApplication(app1Data) + // however, when we initialize a totally new app2, everything is still happy + val app2Id = ApplicationId.newInstance(0, 2) + val app2Data: ApplicationInitializationContext = + new ApplicationInitializationContext("user", app2Id, null) + s2.initializeApplication(app2Data) + val shuffleInfo2 = new ExecutorShuffleInfo(Array("/bippy"), 5, "hash") + resolver2.registerExecutor(app2Id.toString, "exec-2", shuffleInfo2) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver2) should be (Some(shuffleInfo2)) + s2.stop() + + // another stop & restart should be fine though (eg., we recover from previous corruption) + s3 = new YarnShuffleService + s3.init(yarnConfig) + s3.registeredExecutorFile should be (execStateFile) + val handler3 = s3.blockHandler + val resolver3 = ShuffleTestAccessor.getBlockResolver(handler3) + + s3.initializeApplication(app2Data) + ShuffleTestAccessor.getExecutorInfo(app2Id, "exec-2", resolver3) should be (Some(shuffleInfo2)) + s3.stop() + + } + +} diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala new file mode 100644 index 0000000000000..db322cd18e150 --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnTestAccessor.scala @@ -0,0 +1,37 @@ +/* + * 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.network.yarn + +import java.io.File + +/** + * just a cheat to get package-visible members in tests + */ +object YarnTestAccessor { + def getShuffleServicePort: Int = { + YarnShuffleService.boundPort + } + + def getShuffleServiceInstance: YarnShuffleService = { + YarnShuffleService.instance + } + + def getRegisteredExecutorFile(service: YarnShuffleService): File = { + service.registeredExecutorFile + } + +} From 3c462f5d87a9654c5a68fd658a40f5062029fd9a Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 21 Aug 2015 12:21:51 -0700 Subject: [PATCH 042/232] [SPARK-10130] [SQL] type coercion for IF should have children resolved first Type coercion for IF should have children resolved first, or we could meet unresolved exception. Author: Daoyuan Wang Closes #8331 from adrian-wang/spark10130. --- .../spark/sql/catalyst/analysis/HiveTypeCoercion.scala | 1 + .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index f2f2ba2f96552..2cb067f4aac91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -639,6 +639,7 @@ object HiveTypeCoercion { */ object IfCoercion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { + case e if !e.childrenResolved => e // Find tightest common type for If, if the true value and false value have different types. case i @ If(pred, left, right) if left.dataType != right.dataType => findTightestCommonTypeToString(left.dataType, right.dataType).map { widestType => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index da50aec17c89e..dcb4e83710982 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1679,4 +1679,11 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sqlContext.table("`db.t`"), df) } } + + test("SPARK-10130 type coercion for IF should have children resolved first") { + val df = Seq((1, 1), (-1, 1)).toDF("key", "value") + df.registerTempTable("src") + checkAnswer( + sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) + } } From d89cc38b33815e7b99fb3389b5038a543527065d Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 21 Aug 2015 13:10:11 -0700 Subject: [PATCH 043/232] [SPARK-10122] [PYSPARK] [STREAMING] Fix getOffsetRanges bug in PySpark-Streaming transform function Details of the bug and explanations can be seen in [SPARK-10122](https://issues.apache.org/jira/browse/SPARK-10122). tdas , please help to review. Author: jerryshao Closes #8347 from jerryshao/SPARK-10122 and squashes the following commits: 4039b16 [jerryshao] Fix getOffsetRanges in transform() bug --- python/pyspark/streaming/dstream.py | 5 ++++- python/pyspark/streaming/tests.py | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 8dcb9645cdc6b..698336cfce18d 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -610,7 +610,10 @@ def __init__(self, prev, func): self.is_checkpointed = False self._jdstream_val = None - if (isinstance(prev, TransformedDStream) and + # Using type() to avoid folding the functions and compacting the DStreams which is not + # not strictly a object of TransformedDStream. + # Changed here is to avoid bug in KafkaTransformedDStream when calling offsetRanges(). + if (type(prev) is TransformedDStream and not prev.is_cached and not prev.is_checkpointed): prev_func = prev.func self.func = lambda t, rdd: func(t, prev_func(t, rdd)) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 6108c845c1efe..214d5be439003 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -850,7 +850,9 @@ def transformWithOffsetRanges(rdd): offsetRanges.append(o) return rdd - stream.transform(transformWithOffsetRanges).foreachRDD(lambda rdd: rdd.count()) + # Test whether it is ok mixing KafkaTransformedDStream and TransformedDStream together, + # only the TransformedDstreams can be folded together. + stream.transform(transformWithOffsetRanges).map(lambda kv: kv[1]).count().pprint() self.ssc.start() self.wait_for(offsetRanges, 1) From f5b028ed2f1ad6de43c8b50ebf480e1b6c047035 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Fri, 21 Aug 2015 14:19:24 -0700 Subject: [PATCH 044/232] [SPARK-9864] [DOC] [MLlib] [SQL] Replace since in scaladoc to Since annotation Author: MechCoder Closes #8352 from MechCoder/since. --- .../classification/ClassificationModel.scala | 8 +- .../classification/LogisticRegression.scala | 30 ++--- .../mllib/classification/NaiveBayes.scala | 7 +- .../spark/mllib/classification/SVM.scala | 28 ++--- .../mllib/clustering/GaussianMixture.scala | 28 ++--- .../clustering/GaussianMixtureModel.scala | 28 ++--- .../spark/mllib/clustering/KMeans.scala | 50 ++++----- .../spark/mllib/clustering/KMeansModel.scala | 27 ++--- .../apache/spark/mllib/clustering/LDA.scala | 56 ++++----- .../spark/mllib/clustering/LDAModel.scala | 69 +++++------- .../spark/mllib/clustering/LDAOptimizer.scala | 24 ++-- .../clustering/PowerIterationClustering.scala | 38 +++---- .../mllib/clustering/StreamingKMeans.scala | 35 +++--- .../BinaryClassificationMetrics.scala | 26 ++--- .../mllib/evaluation/MulticlassMetrics.scala | 20 ++-- .../mllib/evaluation/MultilabelMetrics.scala | 9 +- .../mllib/evaluation/RankingMetrics.scala | 10 +- .../mllib/evaluation/RegressionMetrics.scala | 14 +-- .../spark/mllib/fpm/AssociationRules.scala | 20 ++-- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 22 ++-- .../apache/spark/mllib/linalg/Matrices.scala | 106 ++++++++---------- .../linalg/SingularValueDecomposition.scala | 4 +- .../apache/spark/mllib/linalg/Vectors.scala | 90 +++++---------- .../linalg/distributed/BlockMatrix.scala | 88 ++++++--------- .../linalg/distributed/CoordinateMatrix.scala | 40 +++---- .../distributed/DistributedMatrix.scala | 4 +- .../linalg/distributed/IndexedRowMatrix.scala | 38 +++---- .../mllib/linalg/distributed/RowMatrix.scala | 39 +++---- .../spark/mllib/recommendation/ALS.scala | 22 ++-- .../MatrixFactorizationModel.scala | 28 +++-- .../GeneralizedLinearAlgorithm.scala | 24 ++-- .../mllib/regression/IsotonicRegression.scala | 22 ++-- .../spark/mllib/regression/LabeledPoint.scala | 7 +- .../apache/spark/mllib/regression/Lasso.scala | 25 ++--- .../mllib/regression/LinearRegression.scala | 25 ++--- .../mllib/regression/RegressionModel.scala | 12 +- .../mllib/regression/RidgeRegression.scala | 25 ++--- .../regression/StreamingLinearAlgorithm.scala | 18 +-- .../spark/mllib/stat/KernelDensity.scala | 12 +- .../stat/MultivariateOnlineSummarizer.scala | 24 ++-- .../stat/MultivariateStatisticalSummary.scala | 19 ++-- .../apache/spark/mllib/stat/Statistics.scala | 30 ++--- .../distribution/MultivariateGaussian.scala | 8 +- .../spark/mllib/tree/DecisionTree.scala | 28 +++-- .../mllib/tree/GradientBoostedTrees.scala | 20 ++-- .../spark/mllib/tree/RandomForest.scala | 20 ++-- .../spark/mllib/tree/configuration/Algo.scala | 4 +- .../tree/configuration/BoostingStrategy.scala | 12 +- .../tree/configuration/FeatureType.scala | 4 +- .../tree/configuration/QuantileStrategy.scala | 4 +- .../mllib/tree/configuration/Strategy.scala | 24 ++-- .../spark/mllib/tree/impurity/Entropy.scala | 10 +- .../spark/mllib/tree/impurity/Gini.scala | 10 +- .../spark/mllib/tree/impurity/Impurity.scala | 8 +- .../spark/mllib/tree/impurity/Variance.scala | 10 +- .../spark/mllib/tree/loss/AbsoluteError.scala | 6 +- .../spark/mllib/tree/loss/LogLoss.scala | 6 +- .../apache/spark/mllib/tree/loss/Loss.scala | 8 +- .../apache/spark/mllib/tree/loss/Losses.scala | 10 +- .../spark/mllib/tree/loss/SquaredError.scala | 6 +- .../mllib/tree/model/DecisionTreeModel.scala | 22 ++-- .../tree/model/InformationGainStats.scala | 4 +- .../apache/spark/mllib/tree/model/Node.scala | 8 +- .../spark/mllib/tree/model/Predict.scala | 4 +- .../apache/spark/mllib/tree/model/Split.scala | 4 +- .../mllib/tree/model/treeEnsembleModels.scala | 26 ++--- .../org/apache/spark/mllib/tree/package.scala | 1 - .../org/apache/spark/mllib/util/MLUtils.scala | 36 +++--- 68 files changed, 692 insertions(+), 862 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index ba73024e3c04d..a29b425a71fd6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.classification import org.json4s.{DefaultFormats, JValue} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD @@ -36,8 +36,8 @@ trait ClassificationModel extends Serializable { * * @param testData RDD representing data points to be predicted * @return an RDD[Double] where each entry contains the corresponding prediction - * @since 0.8.0 */ + @Since("0.8.0") def predict(testData: RDD[Vector]): RDD[Double] /** @@ -45,16 +45,16 @@ trait ClassificationModel extends Serializable { * * @param testData array representing a single data point * @return predicted category from the trained model - * @since 0.8.0 */ + @Since("0.8.0") def predict(testData: Vector): Double /** * Predict values for examples stored in a JavaRDD. * @param testData JavaRDD representing data points to be predicted * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction - * @since 0.8.0 */ + @Since("0.8.0") def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } 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 268642ac6a2f6..e03e662227d14 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 @@ -18,7 +18,7 @@ package org.apache.spark.mllib.classification import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.linalg.{DenseVector, Vector} @@ -85,8 +85,8 @@ class LogisticRegressionModel ( * in Binary Logistic Regression. An example with prediction score greater than or equal to * this threshold is identified as an positive, and negative otherwise. The default value is 0.5. * It is only used for binary classification. - * @since 1.0.0 */ + @Since("1.0.0") @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) @@ -97,8 +97,8 @@ class LogisticRegressionModel ( * :: Experimental :: * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. * It is only used for binary classification. - * @since 1.3.0 */ + @Since("1.3.0") @Experimental def getThreshold: Option[Double] = threshold @@ -106,8 +106,8 @@ class LogisticRegressionModel ( * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. * It is only used for binary classification. - * @since 1.0.0 */ + @Since("1.0.0") @Experimental def clearThreshold(): this.type = { threshold = None @@ -158,9 +158,7 @@ class LogisticRegressionModel ( } } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { GLMClassificationModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, numFeatures, numClasses, weights, intercept, threshold) @@ -168,9 +166,7 @@ class LogisticRegressionModel ( override protected def formatVersion: String = "1.0" - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def toString: String = { s"${super.toString}, numClasses = ${numClasses}, threshold = ${threshold.getOrElse("None")}" } @@ -178,9 +174,7 @@ class LogisticRegressionModel ( object LogisticRegressionModel extends Loader[LogisticRegressionModel] { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): LogisticRegressionModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) // Hard-code class name string in case it changes in the future @@ -261,8 +255,8 @@ object LogisticRegressionWithSGD { * @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 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -284,8 +278,8 @@ object LogisticRegressionWithSGD { * @param stepSize Step size to be used for each iteration of gradient descent. * @param miniBatchFraction Fraction of data to be used per iteration. - * @since 1.0.0 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -306,8 +300,8 @@ object LogisticRegressionWithSGD { * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. - * @since 1.0.0 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -324,8 +318,8 @@ object LogisticRegressionWithSGD { * @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. - * @since 1.0.0 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int): LogisticRegressionModel = { @@ -361,8 +355,8 @@ class LogisticRegressionWithLBFGS * Set the number of possible outcomes for k classes classification problem in * Multinomial Logistic Regression. * By default, it is binary logistic regression so k will be set to 2. - * @since 1.3.0 */ + @Since("1.3.0") @Experimental def setNumClasses(numClasses: Int): this.type = { require(numClasses > 1) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 2df91c09421e9..dab369207cc9a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -25,6 +25,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, SparseVector, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{Loader, Saveable} @@ -444,8 +445,8 @@ object NaiveBayes { * * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency * vector or a count vector. - * @since 0.9.0 */ + @Since("0.9.0") def train(input: RDD[LabeledPoint]): NaiveBayesModel = { new NaiveBayes().run(input) } @@ -460,8 +461,8 @@ object NaiveBayes { * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency * vector or a count vector. * @param lambda The smoothing parameter - * @since 0.9.0 */ + @Since("0.9.0") def train(input: RDD[LabeledPoint], lambda: Double): NaiveBayesModel = { new NaiveBayes(lambda, Multinomial).run(input) } @@ -483,8 +484,8 @@ object NaiveBayes { * * @param modelType The type of NB model to fit from the enumeration NaiveBayesModels, can be * multinomial or bernoulli - * @since 0.9.0 */ + @Since("0.9.0") def train(input: RDD[LabeledPoint], lambda: Double, modelType: String): NaiveBayesModel = { require(supportedModelTypes.contains(modelType), s"NaiveBayes was created with an unknown modelType: $modelType.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 5b54feeb10467..5f87269863572 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.classification import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ @@ -46,8 +46,8 @@ class SVMModel ( * Sets the threshold that separates positive predictions from negative predictions. An example * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.0. - * @since 1.3.0 */ + @Since("1.3.0") @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) @@ -57,16 +57,16 @@ class SVMModel ( /** * :: Experimental :: * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. - * @since 1.3.0 */ + @Since("1.3.0") @Experimental def getThreshold: Option[Double] = threshold /** * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. - * @since 1.0.0 */ + @Since("1.0.0") @Experimental def clearThreshold(): this.type = { threshold = None @@ -84,9 +84,7 @@ class SVMModel ( } } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { GLMClassificationModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, numFeatures = weights.size, numClasses = 2, weights, intercept, threshold) @@ -94,9 +92,7 @@ class SVMModel ( override protected def formatVersion: String = "1.0" - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def toString: String = { s"${super.toString}, numClasses = 2, threshold = ${threshold.getOrElse("None")}" } @@ -104,9 +100,7 @@ class SVMModel ( object SVMModel extends Loader[SVMModel] { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): SVMModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) // Hard-code class name string in case it changes in the future @@ -185,8 +179,8 @@ object SVMWithSGD { * @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 0.8.0 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -209,8 +203,8 @@ object SVMWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -231,8 +225,8 @@ object SVMWithSGD { * @param regParam Regularization parameter. * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. - * @since 0.8.0 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -250,8 +244,8 @@ object SVMWithSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. - * @since 0.8.0 */ + @Since("0.8.0") def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 0.01, 1.0) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index bc27b1fe7390b..fcc9dfecac54f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.IndexedSeq import breeze.linalg.{diag, DenseMatrix => BreezeMatrix, DenseVector => BDV, Vector => BV} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, Matrices, Vector, Vectors} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian @@ -62,8 +62,8 @@ class GaussianMixture private ( /** * Constructs a default instance. The default parameters are {k: 2, convergenceTol: 0.01, * maxIterations: 100, seed: random}. - * @since 1.3.0 */ + @Since("1.3.0") def this() = this(2, 0.01, 100, Utils.random.nextLong()) // number of samples per cluster to use when initializing Gaussians @@ -77,8 +77,8 @@ class GaussianMixture private ( * Set the initial GMM starting point, bypassing the random initialization. * You must call setK() prior to calling this method, and the condition * (model.k == this.k) must be met; failure will result in an IllegalArgumentException - * @since 1.3.0 */ + @Since("1.3.0") def setInitialModel(model: GaussianMixtureModel): this.type = { if (model.k == k) { initialModel = Some(model) @@ -90,14 +90,14 @@ class GaussianMixture private ( /** * Return the user supplied initial GMM, if supplied - * @since 1.3.0 */ + @Since("1.3.0") def getInitialModel: Option[GaussianMixtureModel] = initialModel /** * Set the number of Gaussians in the mixture model. Default: 2 - * @since 1.3.0 */ + @Since("1.3.0") def setK(k: Int): this.type = { this.k = k this @@ -105,14 +105,14 @@ class GaussianMixture private ( /** * Return the number of Gaussians in the mixture model - * @since 1.3.0 */ + @Since("1.3.0") def getK: Int = k /** * Set the maximum number of iterations to run. Default: 100 - * @since 1.3.0 */ + @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this @@ -120,15 +120,15 @@ class GaussianMixture private ( /** * Return the maximum number of iterations to run - * @since 1.3.0 */ + @Since("1.3.0") def getMaxIterations: Int = maxIterations /** * Set the largest change in log-likelihood at which convergence is * considered to have occurred. - * @since 1.3.0 */ + @Since("1.3.0") def setConvergenceTol(convergenceTol: Double): this.type = { this.convergenceTol = convergenceTol this @@ -137,14 +137,14 @@ class GaussianMixture private ( /** * Return the largest change in log-likelihood at which convergence is * considered to have occurred. - * @since 1.3.0 */ + @Since("1.3.0") def getConvergenceTol: Double = convergenceTol /** * Set the random seed - * @since 1.3.0 */ + @Since("1.3.0") def setSeed(seed: Long): this.type = { this.seed = seed this @@ -152,14 +152,14 @@ class GaussianMixture private ( /** * Return the random seed - * @since 1.3.0 */ + @Since("1.3.0") def getSeed: Long = seed /** * Perform expectation maximization - * @since 1.3.0 */ + @Since("1.3.0") def run(data: RDD[Vector]): GaussianMixtureModel = { val sc = data.sparkContext @@ -235,8 +235,8 @@ class GaussianMixture private ( /** * Java-friendly version of [[run()]] - * @since 1.3.0 */ + @Since("1.3.0") def run(data: JavaRDD[Vector]): GaussianMixtureModel = run(data.rdd) private def updateWeightsAndGaussians( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 2fa0473737aae..1a10a8b624218 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -24,7 +24,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Matrices, Matrix} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian @@ -43,8 +43,8 @@ import org.apache.spark.sql.{SQLContext, Row} * the weight for Gaussian i, and weights.sum == 1 * @param gaussians Array of MultivariateGaussian where gaussians(i) represents * the Multivariate Gaussian (Normal) Distribution for Gaussian i - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class GaussianMixtureModel( val weights: Array[Double], @@ -54,23 +54,21 @@ class GaussianMixtureModel( override protected def formatVersion = "1.0" - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { GaussianMixtureModel.SaveLoadV1_0.save(sc, path, weights, gaussians) } /** * Number of gaussians in mixture - * @since 1.3.0 */ + @Since("1.3.0") def k: Int = weights.length /** * Maps given points to their cluster indices. - * @since 1.3.0 */ + @Since("1.3.0") def predict(points: RDD[Vector]): RDD[Int] = { val responsibilityMatrix = predictSoft(points) responsibilityMatrix.map(r => r.indexOf(r.max)) @@ -78,8 +76,8 @@ class GaussianMixtureModel( /** * Maps given point to its cluster index. - * @since 1.5.0 */ + @Since("1.5.0") def predict(point: Vector): Int = { val r = computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) r.indexOf(r.max) @@ -87,16 +85,16 @@ class GaussianMixtureModel( /** * Java-friendly version of [[predict()]] - * @since 1.4.0 */ + @Since("1.4.0") def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] /** * Given the input vectors, return the membership value of each vector * to all mixture components. - * @since 1.3.0 */ + @Since("1.3.0") def predictSoft(points: RDD[Vector]): RDD[Array[Double]] = { val sc = points.sparkContext val bcDists = sc.broadcast(gaussians) @@ -108,8 +106,8 @@ class GaussianMixtureModel( /** * Given the input vector, return the membership values to all mixture components. - * @since 1.4.0 */ + @Since("1.4.0") def predictSoft(point: Vector): Array[Double] = { computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) } @@ -133,9 +131,7 @@ class GaussianMixtureModel( } } -/** - * @since 1.4.0 - */ +@Since("1.4.0") @Experimental object GaussianMixtureModel extends Loader[GaussianMixtureModel] { @@ -186,9 +182,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { } } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def load(sc: SparkContext, path: String) : GaussianMixtureModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats 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 9ef6834e5ea8d..3e9545a74bef3 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 @@ -20,7 +20,7 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.util.MLUtils @@ -49,20 +49,20 @@ class KMeans private ( /** * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4, seed: random}. - * @since 0.8.0 */ + @Since("0.8.0") def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong()) /** * Number of clusters to create (k). - * @since 1.4.0 */ + @Since("1.4.0") def getK: Int = k /** * Set the number of clusters to create (k). Default: 2. - * @since 0.8.0 */ + @Since("0.8.0") def setK(k: Int): this.type = { this.k = k this @@ -70,14 +70,14 @@ class KMeans private ( /** * Maximum number of iterations to run. - * @since 1.4.0 */ + @Since("1.4.0") def getMaxIterations: Int = maxIterations /** * Set maximum number of iterations to run. Default: 20. - * @since 0.8.0 */ + @Since("0.8.0") def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this @@ -85,16 +85,16 @@ class KMeans private ( /** * The initialization algorithm. This can be either "random" or "k-means||". - * @since 1.4.0 */ + @Since("1.4.0") def getInitializationMode: String = initializationMode /** * Set the initialization algorithm. This can be either "random" to choose random points as * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - * @since 0.8.0 */ + @Since("0.8.0") def setInitializationMode(initializationMode: String): this.type = { KMeans.validateInitMode(initializationMode) this.initializationMode = initializationMode @@ -104,8 +104,8 @@ class KMeans private ( /** * :: Experimental :: * Number of runs of the algorithm to execute in parallel. - * @since 1.4.0 */ + @Since("1.4.0") @Experimental def getRuns: Int = runs @@ -114,8 +114,8 @@ class KMeans private ( * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. - * @since 0.8.0 */ + @Since("0.8.0") @Experimental def setRuns(runs: Int): this.type = { if (runs <= 0) { @@ -127,15 +127,15 @@ class KMeans private ( /** * Number of steps for the k-means|| initialization mode - * @since 1.4.0 */ + @Since("1.4.0") def getInitializationSteps: Int = initializationSteps /** * Set the number of steps for the k-means|| initialization mode. This is an advanced * setting -- the default of 5 is almost always enough. Default: 5. - * @since 0.8.0 */ + @Since("0.8.0") def setInitializationSteps(initializationSteps: Int): this.type = { if (initializationSteps <= 0) { throw new IllegalArgumentException("Number of initialization steps must be positive") @@ -146,15 +146,15 @@ class KMeans private ( /** * The distance threshold within which we've consider centers to have converged. - * @since 1.4.0 */ + @Since("1.4.0") def getEpsilon: Double = epsilon /** * Set the distance threshold within which we've consider centers to have converged. * If all centers move less than this Euclidean distance, we stop iterating one run. - * @since 0.8.0 */ + @Since("0.8.0") def setEpsilon(epsilon: Double): this.type = { this.epsilon = epsilon this @@ -162,14 +162,14 @@ class KMeans private ( /** * The random seed for cluster initialization. - * @since 1.4.0 */ + @Since("1.4.0") def getSeed: Long = seed /** * Set the random seed for cluster initialization. - * @since 1.4.0 */ + @Since("1.4.0") def setSeed(seed: Long): this.type = { this.seed = seed this @@ -183,8 +183,8 @@ class KMeans private ( * Set the initial starting point, bypassing the random initialization or k-means|| * The condition model.k == this.k must be met, failure results * in an IllegalArgumentException. - * @since 1.4.0 */ + @Since("1.4.0") def setInitialModel(model: KMeansModel): this.type = { require(model.k == k, "mismatched cluster count") initialModel = Some(model) @@ -194,8 +194,8 @@ class KMeans private ( /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. - * @since 0.8.0 */ + @Since("0.8.0") def run(data: RDD[Vector]): KMeansModel = { if (data.getStorageLevel == StorageLevel.NONE) { @@ -453,14 +453,14 @@ class KMeans private ( /** * Top-level methods for calling K-means clustering. - * @since 0.8.0 */ +@Since("0.8.0") object KMeans { // Initialization mode names - /** @since 0.8.0 */ + @Since("0.8.0") val RANDOM = "random" - /** @since 0.8.0 */ + @Since("0.8.0") val K_MEANS_PARALLEL = "k-means||" /** @@ -472,8 +472,8 @@ object KMeans { * @param runs number of parallel runs, defaults to 1. The best model is returned. * @param initializationMode initialization model, either "random" or "k-means||" (default). * @param seed random seed value for cluster initialization - * @since 1.3.0 */ + @Since("1.3.0") def train( data: RDD[Vector], k: Int, @@ -497,8 +497,8 @@ object KMeans { * @param maxIterations max number of iterations * @param runs number of parallel runs, defaults to 1. The best model is returned. * @param initializationMode initialization model, either "random" or "k-means||" (default). - * @since 0.8.0 */ + @Since("0.8.0") def train( data: RDD[Vector], k: Int, @@ -514,8 +514,8 @@ object KMeans { /** * Trains a k-means model using specified parameters and the default values for unspecified. - * @since 0.8.0 */ + @Since("0.8.0") def train( data: RDD[Vector], k: Int, @@ -525,8 +525,8 @@ object KMeans { /** * Trains a k-means model using specified parameters and the default values for unspecified. - * @since 0.8.0 */ + @Since("0.8.0") def train( data: RDD[Vector], k: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 8de2087ceb4df..e425ecdd481c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -23,6 +23,7 @@ import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.pmml.PMMLExportable @@ -34,35 +35,35 @@ import org.apache.spark.sql.Row /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. - * @since 0.8.0 */ +@Since("0.8.0") class KMeansModel ( val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { /** * A Java-friendly constructor that takes an Iterable of Vectors. - * @since 1.4.0 */ + @Since("1.4.0") def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) /** * Total number of clusters. - * @since 0.8.0 */ + @Since("0.8.0") def k: Int = clusterCenters.length /** * Returns the cluster index that a given point belongs to. - * @since 0.8.0 */ + @Since("0.8.0") def predict(point: Vector): Int = { KMeans.findClosest(clusterCentersWithNorm, new VectorWithNorm(point))._1 } /** * Maps given points to their cluster indices. - * @since 1.0.0 */ + @Since("1.0.0") def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm val bcCentersWithNorm = points.context.broadcast(centersWithNorm) @@ -71,16 +72,16 @@ class KMeansModel ( /** * Maps given points to their cluster indices. - * @since 1.0.0 */ + @Since("1.0.0") def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] = predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]] /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. - * @since 0.8.0 */ + @Since("0.8.0") def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm val bcCentersWithNorm = data.context.broadcast(centersWithNorm) @@ -90,9 +91,7 @@ class KMeansModel ( private def clusterCentersWithNorm: Iterable[VectorWithNorm] = clusterCenters.map(new VectorWithNorm(_)) - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { KMeansModel.SaveLoadV1_0.save(sc, this, path) } @@ -100,14 +99,10 @@ class KMeansModel ( override protected def formatVersion: String = "1.0" } -/** - * @since 1.4.0 - */ +@Since("1.4.0") object KMeansModel extends Loader[KMeansModel] { - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def load(sc: SparkContext, path: String): KMeansModel = { KMeansModel.SaveLoadV1_0.load(sc, path) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala index 2a8c6acbaec61..92a321afb0ca3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BDV} import org.apache.spark.Logging -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.JavaPairRDD import org.apache.spark.graphx._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -43,8 +43,8 @@ import org.apache.spark.util.Utils * * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation * (Wikipedia)]] - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class LDA private ( private var k: Int, @@ -57,8 +57,8 @@ class LDA private ( /** * Constructs a LDA instance with default parameters. - * @since 1.3.0 */ + @Since("1.3.0") def this() = this(k = 10, maxIterations = 20, docConcentration = Vectors.dense(-1), topicConcentration = -1, seed = Utils.random.nextLong(), checkpointInterval = 10, ldaOptimizer = new EMLDAOptimizer) @@ -66,15 +66,15 @@ class LDA private ( /** * Number of topics to infer. I.e., the number of soft cluster centers. * - * @since 1.3.0 */ + @Since("1.3.0") def getK: Int = k /** * Number of topics to infer. I.e., the number of soft cluster centers. * (default = 10) - * @since 1.3.0 */ + @Since("1.3.0") def setK(k: Int): this.type = { require(k > 0, s"LDA k (number of clusters) must be > 0, but was set to $k") this.k = k @@ -86,8 +86,8 @@ class LDA private ( * distributions over topics ("theta"). * * This is the parameter to a Dirichlet distribution. - * @since 1.5.0 */ + @Since("1.5.0") def getAsymmetricDocConcentration: Vector = this.docConcentration /** @@ -96,8 +96,8 @@ class LDA private ( * * This method assumes the Dirichlet distribution is symmetric and can be described by a single * [[Double]] parameter. It should fail if docConcentration is asymmetric. - * @since 1.3.0 */ + @Since("1.3.0") def getDocConcentration: Double = { val parameter = docConcentration(0) if (docConcentration.size == 1) { @@ -131,8 +131,8 @@ class LDA private ( * - Values should be >= 0 * - default = uniformly (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. - * @since 1.5.0 */ + @Since("1.5.0") def setDocConcentration(docConcentration: Vector): this.type = { require(docConcentration.size > 0, "docConcentration must have > 0 elements") this.docConcentration = docConcentration @@ -141,8 +141,8 @@ class LDA private ( /** * Replicates a [[Double]] docConcentration to create a symmetric prior. - * @since 1.3.0 */ + @Since("1.3.0") def setDocConcentration(docConcentration: Double): this.type = { this.docConcentration = Vectors.dense(docConcentration) this @@ -150,26 +150,26 @@ class LDA private ( /** * Alias for [[getAsymmetricDocConcentration]] - * @since 1.5.0 */ + @Since("1.5.0") def getAsymmetricAlpha: Vector = getAsymmetricDocConcentration /** * Alias for [[getDocConcentration]] - * @since 1.3.0 */ + @Since("1.3.0") def getAlpha: Double = getDocConcentration /** * Alias for [[setDocConcentration()]] - * @since 1.5.0 */ + @Since("1.5.0") def setAlpha(alpha: Vector): this.type = setDocConcentration(alpha) /** * Alias for [[setDocConcentration()]] - * @since 1.3.0 */ + @Since("1.3.0") def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) /** @@ -180,8 +180,8 @@ class LDA private ( * * Note: The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. - * @since 1.3.0 */ + @Since("1.3.0") def getTopicConcentration: Double = this.topicConcentration /** @@ -205,8 +205,8 @@ class LDA private ( * - Value should be >= 0 * - default = (1.0 / k), following the implementation from * [[https://github.com/Blei-Lab/onlineldavb]]. - * @since 1.3.0 */ + @Since("1.3.0") def setTopicConcentration(topicConcentration: Double): this.type = { this.topicConcentration = topicConcentration this @@ -214,27 +214,27 @@ class LDA private ( /** * Alias for [[getTopicConcentration]] - * @since 1.3.0 */ + @Since("1.3.0") def getBeta: Double = getTopicConcentration /** * Alias for [[setTopicConcentration()]] - * @since 1.3.0 */ + @Since("1.3.0") def setBeta(beta: Double): this.type = setTopicConcentration(beta) /** * Maximum number of iterations for learning. - * @since 1.3.0 */ + @Since("1.3.0") def getMaxIterations: Int = maxIterations /** * Maximum number of iterations for learning. * (default = 20) - * @since 1.3.0 */ + @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this @@ -242,14 +242,14 @@ class LDA private ( /** * Random seed - * @since 1.3.0 */ + @Since("1.3.0") def getSeed: Long = seed /** * Random seed - * @since 1.3.0 */ + @Since("1.3.0") def setSeed(seed: Long): this.type = { this.seed = seed this @@ -257,8 +257,8 @@ class LDA private ( /** * Period (in iterations) between checkpoints. - * @since 1.3.0 */ + @Since("1.3.0") def getCheckpointInterval: Int = checkpointInterval /** @@ -268,8 +268,8 @@ class LDA private ( * [[org.apache.spark.SparkContext]], this setting is ignored. * * @see [[org.apache.spark.SparkContext#setCheckpointDir]] - * @since 1.3.0 */ + @Since("1.3.0") def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval this @@ -280,8 +280,8 @@ class LDA private ( * :: DeveloperApi :: * * LDAOptimizer used to perform the actual calculation - * @since 1.4.0 */ + @Since("1.4.0") @DeveloperApi def getOptimizer: LDAOptimizer = ldaOptimizer @@ -289,8 +289,8 @@ class LDA private ( * :: DeveloperApi :: * * LDAOptimizer used to perform the actual calculation (default = EMLDAOptimizer) - * @since 1.4.0 */ + @Since("1.4.0") @DeveloperApi def setOptimizer(optimizer: LDAOptimizer): this.type = { this.ldaOptimizer = optimizer @@ -300,8 +300,8 @@ class LDA private ( /** * Set the LDAOptimizer used to perform the actual calculation by algorithm name. * Currently "em", "online" are supported. - * @since 1.4.0 */ + @Since("1.4.0") def setOptimizer(optimizerName: String): this.type = { this.ldaOptimizer = optimizerName.toLowerCase match { @@ -321,8 +321,8 @@ class LDA private ( * (where the vocabulary size is the length of the vector). * Document IDs must be unique and >= 0. * @return Inferred LDA model - * @since 1.3.0 */ + @Since("1.3.0") def run(documents: RDD[(Long, Vector)]): LDAModel = { val state = ldaOptimizer.initialize(documents, this) var iter = 0 @@ -339,8 +339,8 @@ class LDA private ( /** * Java-friendly version of [[run()]] - * @since 1.3.0 */ + @Since("1.3.0") def run(documents: JavaPairRDD[java.lang.Long, Vector]): LDAModel = { run(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 6bc68a4c18b99..667374a2bc418 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.graphx.{Edge, EdgeContext, Graph, VertexId} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} @@ -192,24 +192,16 @@ class LocalLDAModel private[clustering] ( override protected[clustering] val gammaShape: Double = 100) extends LDAModel with Serializable { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def k: Int = topics.numCols - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def vocabSize: Int = topics.numRows - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def topicsMatrix: Matrix = topics - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val brzTopics = topics.toBreeze.toDenseMatrix Range(0, k).map { topicIndex => @@ -222,9 +214,7 @@ class LocalLDAModel private[clustering] ( override protected def formatVersion = "1.0" - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def save(sc: SparkContext, path: String): Unit = { LocalLDAModel.SaveLoadV1_0.save(sc, path, topicsMatrix, docConcentration, topicConcentration, gammaShape) @@ -238,16 +228,16 @@ class LocalLDAModel private[clustering] ( * * @param documents test corpus to use for calculating log likelihood * @return variational lower bound on the log likelihood of the entire corpus - * @since 1.5.0 */ + @Since("1.5.0") def logLikelihood(documents: RDD[(Long, Vector)]): Double = logLikelihoodBound(documents, docConcentration, topicConcentration, topicsMatrix.toBreeze.toDenseMatrix, gammaShape, k, vocabSize) /** * Java-friendly version of [[logLikelihood]] - * @since 1.5.0 */ + @Since("1.5.0") def logLikelihood(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logLikelihood(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } @@ -258,8 +248,8 @@ class LocalLDAModel private[clustering] ( * * @param documents test corpus to use for calculating perplexity * @return Variational upper bound on log perplexity per token. - * @since 1.5.0 */ + @Since("1.5.0") def logPerplexity(documents: RDD[(Long, Vector)]): Double = { val corpusTokenCount = documents .map { case (_, termCounts) => termCounts.toArray.sum } @@ -267,9 +257,8 @@ class LocalLDAModel private[clustering] ( -logLikelihood(documents) / corpusTokenCount } - /** Java-friendly version of [[logPerplexity]] - * @since 1.5.0 - */ + /** Java-friendly version of [[logPerplexity]] */ + @Since("1.5.0") def logPerplexity(documents: JavaPairRDD[java.lang.Long, Vector]): Double = { logPerplexity(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) } @@ -347,8 +336,8 @@ class LocalLDAModel private[clustering] ( * for each document. * @param documents documents to predict topic mixture distributions for * @return An RDD of (document ID, topic mixture distribution for document) - * @since 1.3.0 */ + @Since("1.3.0") // TODO: declare in LDAModel and override once implemented in DistributedLDAModel def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = { // Double transpose because dirichletExpectation normalizes by row and we need to normalize @@ -376,8 +365,8 @@ class LocalLDAModel private[clustering] ( /** * Java-friendly version of [[topicDistributions]] - * @since 1.4.1 */ + @Since("1.4.1") def topicDistributions( documents: JavaPairRDD[java.lang.Long, Vector]): JavaPairRDD[java.lang.Long, Vector] = { val distributions = topicDistributions(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) @@ -451,9 +440,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { } } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def load(sc: SparkContext, path: String): LocalLDAModel = { val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats @@ -510,8 +497,8 @@ class DistributedLDAModel private[clustering] ( * Convert model to a local model. * The local model stores the inferred topics but not the topic distributions for training * documents. - * @since 1.3.0 */ + @Since("1.3.0") def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix, docConcentration, topicConcentration, gammaShape) @@ -521,8 +508,8 @@ class DistributedLDAModel private[clustering] ( * No guarantees are given about the ordering of the topics. * * WARNING: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. - * @since 1.3.0 */ + @Since("1.3.0") override lazy val topicsMatrix: Matrix = { // Collect row-major topics val termTopicCounts: Array[(Int, TopicCounts)] = @@ -541,9 +528,7 @@ class DistributedLDAModel private[clustering] ( Matrices.fromBreeze(brzTopics) } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { val numTopics = k // Note: N_k is not needed to find the top terms, but it is needed to normalize weights @@ -582,8 +567,8 @@ class DistributedLDAModel private[clustering] ( * @return Array over topics. Each element represent as a pair of matching arrays: * (IDs for the documents, weights of the topic in these documents). * For each topic, documents are sorted in order of decreasing topic weights. - * @since 1.5.0 */ + @Since("1.5.0") def topDocumentsPerTopic(maxDocumentsPerTopic: Int): Array[(Array[Long], Array[Double])] = { val numTopics = k val topicsInQueues: Array[BoundedPriorityQueue[(Double, Long)]] = @@ -666,8 +651,8 @@ class DistributedLDAModel private[clustering] ( * - This excludes the prior; for that, use [[logPrior]]. * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the * hyperparameters. - * @since 1.3.0 */ + @Since("1.3.0") lazy val logLikelihood: Double = { // TODO: generalize this for asymmetric (non-scalar) alpha val alpha = this.docConcentration(0) // To avoid closure capture of enclosing object @@ -693,8 +678,8 @@ class DistributedLDAModel private[clustering] ( /** * Log probability of the current parameter estimate: * log P(topics, topic distributions for docs | alpha, eta) - * @since 1.3.0 */ + @Since("1.3.0") lazy val logPrior: Double = { // TODO: generalize this for asymmetric (non-scalar) alpha val alpha = this.docConcentration(0) // To avoid closure capture of enclosing object @@ -725,8 +710,8 @@ class DistributedLDAModel private[clustering] ( * ("theta_doc"). * * @return RDD of (document ID, topic distribution) pairs - * @since 1.3.0 */ + @Since("1.3.0") def topicDistributions: RDD[(Long, Vector)] = { graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => (docID.toLong, Vectors.fromBreeze(normalize(topicCounts, 1.0))) @@ -735,8 +720,8 @@ class DistributedLDAModel private[clustering] ( /** * Java-friendly version of [[topicDistributions]] - * @since 1.4.1 */ + @Since("1.4.1") def javaTopicDistributions: JavaPairRDD[java.lang.Long, Vector] = { JavaPairRDD.fromRDD(topicDistributions.asInstanceOf[RDD[(java.lang.Long, Vector)]]) } @@ -744,8 +729,8 @@ class DistributedLDAModel private[clustering] ( /** * For each document, return the top k weighted topics for that document and their weights. * @return RDD of (doc ID, topic indices, topic weights) - * @since 1.5.0 */ + @Since("1.5.0") def topTopicsPerDocument(k: Int): RDD[(Long, Array[Int], Array[Double])] = { graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => val topIndices = argtopk(topicCounts, k) @@ -761,8 +746,8 @@ class DistributedLDAModel private[clustering] ( /** * Java-friendly version of [[topTopicsPerDocument]] - * @since 1.5.0 */ + @Since("1.5.0") def javaTopTopicsPerDocument(k: Int): JavaRDD[(java.lang.Long, Array[Int], Array[Double])] = { val topics = topTopicsPerDocument(k) topics.asInstanceOf[RDD[(java.lang.Long, Array[Int], Array[Double])]].toJavaRDD() @@ -775,8 +760,8 @@ class DistributedLDAModel private[clustering] ( /** * Java-friendly version of [[topicDistributions]] - * @since 1.5.0 */ + @Since("1.5.0") override def save(sc: SparkContext, path: String): Unit = { DistributedLDAModel.SaveLoadV1_0.save( sc, path, graph, globalTopicTotals, k, vocabSize, docConcentration, topicConcentration, @@ -877,9 +862,7 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def load(sc: SparkContext, path: String): DistributedLDAModel = { val (loadedClassName, loadedVersion, metadata) = Loader.loadMetadata(sc, path) implicit val formats = DefaultFormats diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index cb517f9689ade..5c2aae6403bea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -23,7 +23,7 @@ import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, all, normalize, su import breeze.numerics.{trigamma, abs, exp} import breeze.stats.distributions.{Gamma, RandBasis} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer @@ -35,8 +35,8 @@ import org.apache.spark.rdd.RDD * * An LDAOptimizer specifies which optimization/learning/inference algorithm to use, and it can * hold optimizer-specific parameters for users to set. - * @since 1.4.0 */ +@Since("1.4.0") @DeveloperApi sealed trait LDAOptimizer { @@ -74,8 +74,8 @@ sealed trait LDAOptimizer { * - Paper which clearly explains several algorithms, including EM: * Asuncion, Welling, Smyth, and Teh. * "On Smoothing and Inference for Topic Models." UAI, 2009. - * @since 1.4.0 */ +@Since("1.4.0") @DeveloperApi final class EMLDAOptimizer extends LDAOptimizer { @@ -226,8 +226,8 @@ final class EMLDAOptimizer extends LDAOptimizer { * * Original Online LDA paper: * Hoffman, Blei and Bach, "Online Learning for Latent Dirichlet Allocation." NIPS, 2010. - * @since 1.4.0 */ +@Since("1.4.0") @DeveloperApi final class OnlineLDAOptimizer extends LDAOptimizer { @@ -276,16 +276,16 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. - * @since 1.4.0 */ + @Since("1.4.0") def getTau0: Double = this.tau0 /** * A (positive) learning parameter that downweights early iterations. Larger values make early * iterations count less. * Default: 1024, following the original Online LDA paper. - * @since 1.4.0 */ + @Since("1.4.0") def setTau0(tau0: Double): this.type = { require(tau0 > 0, s"LDA tau0 must be positive, but was set to $tau0") this.tau0 = tau0 @@ -294,16 +294,16 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Learning rate: exponential decay rate - * @since 1.4.0 */ + @Since("1.4.0") def getKappa: Double = this.kappa /** * Learning rate: exponential decay rate---should be between * (0.5, 1.0] to guarantee asymptotic convergence. * Default: 0.51, based on the original Online LDA paper. - * @since 1.4.0 */ + @Since("1.4.0") def setKappa(kappa: Double): this.type = { require(kappa >= 0, s"Online LDA kappa must be nonnegative, but was set to $kappa") this.kappa = kappa @@ -312,8 +312,8 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Mini-batch fraction, which sets the fraction of document sampled and used in each iteration - * @since 1.4.0 */ + @Since("1.4.0") def getMiniBatchFraction: Double = this.miniBatchFraction /** @@ -325,8 +325,8 @@ final class OnlineLDAOptimizer extends LDAOptimizer { * maxIterations * miniBatchFraction >= 1. * * Default: 0.05, i.e., 5% of total documents. - * @since 1.4.0 */ + @Since("1.4.0") def setMiniBatchFraction(miniBatchFraction: Double): this.type = { require(miniBatchFraction > 0.0 && miniBatchFraction <= 1.0, s"Online LDA miniBatchFraction must be in range (0,1], but was set to $miniBatchFraction") @@ -337,16 +337,16 @@ final class OnlineLDAOptimizer extends LDAOptimizer { /** * Optimize alpha, indicates whether alpha (Dirichlet parameter for document-topic distribution) * will be optimized during training. - * @since 1.5.0 */ + @Since("1.5.0") def getOptimzeAlpha: Boolean = this.optimizeAlpha /** * Sets whether to optimize alpha parameter during training. * * Default: false - * @since 1.5.0 */ + @Since("1.5.0") def setOptimzeAlpha(optimizeAlpha: Boolean): this.type = { this.optimizeAlpha = optimizeAlpha this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index b4733ca975152..396b36f2f6454 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -21,7 +21,7 @@ import org.json4s.JsonDSL._ import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl @@ -39,16 +39,14 @@ import org.apache.spark.{Logging, SparkContext, SparkException} * * @param k number of clusters * @param assignments an RDD of clustering [[PowerIterationClustering#Assignment]]s - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class PowerIterationClusteringModel( val k: Int, val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) } @@ -56,9 +54,7 @@ class PowerIterationClusteringModel( override protected def formatVersion: String = "1.0" } -/** - * @since 1.4.0 - */ +@Since("1.4.0") object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) @@ -73,8 +69,8 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" /** - * @since 1.4.0 */ + @Since("1.4.0") def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { val sqlContext = new SQLContext(sc) import sqlContext.implicits._ @@ -87,9 +83,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode dataRDD.write.parquet(Loader.dataPath(path)) } - /** - * @since 1.4.0 - */ + @Since("1.4.0") def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { implicit val formats = DefaultFormats val sqlContext = new SQLContext(sc) @@ -136,14 +130,14 @@ class PowerIterationClustering private[clustering] ( /** * Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100, * initMode: "random"}. - * @since 1.3.0 */ + @Since("1.3.0") def this() = this(k = 2, maxIterations = 100, initMode = "random") /** * Set the number of clusters. - * @since 1.3.0 */ + @Since("1.3.0") def setK(k: Int): this.type = { this.k = k this @@ -151,8 +145,8 @@ class PowerIterationClustering private[clustering] ( /** * Set maximum number of iterations of the power iteration loop - * @since 1.3.0 */ + @Since("1.3.0") def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this @@ -161,8 +155,8 @@ class PowerIterationClustering private[clustering] ( /** * Set the initialization mode. This can be either "random" to use a random vector * as vertex properties, or "degree" to use normalized sum similarities. Default: random. - * @since 1.3.0 */ + @Since("1.3.0") def setInitializationMode(mode: String): this.type = { this.initMode = mode match { case "random" | "degree" => mode @@ -182,8 +176,8 @@ class PowerIterationClustering private[clustering] ( * assume s,,ij,, = 0.0. * * @return a [[PowerIterationClusteringModel]] that contains the clustering result - * @since 1.5.0 */ + @Since("1.5.0") def run(graph: Graph[Double, Double]): PowerIterationClusteringModel = { val w = normalize(graph) val w0 = initMode match { @@ -204,8 +198,8 @@ class PowerIterationClustering private[clustering] ( * assume s,,ij,, = 0.0. * * @return a [[PowerIterationClusteringModel]] that contains the clustering result - * @since 1.3.0 */ + @Since("1.3.0") def run(similarities: RDD[(Long, Long, Double)]): PowerIterationClusteringModel = { val w = normalize(similarities) val w0 = initMode match { @@ -217,8 +211,8 @@ class PowerIterationClustering private[clustering] ( /** * A Java-friendly version of [[PowerIterationClustering.run]]. - * @since 1.3.0 */ + @Since("1.3.0") def run(similarities: JavaRDD[(java.lang.Long, java.lang.Long, java.lang.Double)]) : PowerIterationClusteringModel = { run(similarities.rdd.asInstanceOf[RDD[(Long, Long, Double)]]) @@ -242,9 +236,7 @@ class PowerIterationClustering private[clustering] ( } } -/** - * @since 1.3.0 - */ +@Since("1.3.0") @Experimental object PowerIterationClustering extends Logging { @@ -253,8 +245,8 @@ object PowerIterationClustering extends Logging { * Cluster assignment. * @param id node id * @param cluster assigned cluster id - * @since 1.3.0 */ + @Since("1.3.0") @Experimental case class Assignment(id: Long, cluster: Int) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index a915804b02c90..41f2668ec6a7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.clustering import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaSparkContext._ import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD @@ -63,9 +63,8 @@ import org.apache.spark.util.random.XORShiftRandom * such that at time t + h the discount applied to the data from t is 0.5. * The definition remains the same whether the time unit is given * as batches or points. - * @since 1.2.0 - * */ +@Since("1.2.0") @Experimental class StreamingKMeansModel( override val clusterCenters: Array[Vector], @@ -73,8 +72,8 @@ class StreamingKMeansModel( /** * Perform a k-means update on a batch of data. - * @since 1.2.0 */ + @Since("1.2.0") def update(data: RDD[Vector], decayFactor: Double, timeUnit: String): StreamingKMeansModel = { // find nearest cluster to each point @@ -166,23 +165,23 @@ class StreamingKMeansModel( * .setRandomCenters(5, 100.0) * .trainOn(DStream) * }}} - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class StreamingKMeans( var k: Int, var decayFactor: Double, var timeUnit: String) extends Logging with Serializable { - /** @since 1.2.0 */ + @Since("1.2.0") def this() = this(2, 1.0, StreamingKMeans.BATCHES) protected var model: StreamingKMeansModel = new StreamingKMeansModel(null, null) /** * Set the number of clusters. - * @since 1.2.0 */ + @Since("1.2.0") def setK(k: Int): this.type = { this.k = k this @@ -190,8 +189,8 @@ class StreamingKMeans( /** * Set the decay factor directly (for forgetful algorithms). - * @since 1.2.0 */ + @Since("1.2.0") def setDecayFactor(a: Double): this.type = { this.decayFactor = a this @@ -199,8 +198,8 @@ class StreamingKMeans( /** * Set the half life and time unit ("batches" or "points") for forgetful algorithms. - * @since 1.2.0 */ + @Since("1.2.0") def setHalfLife(halfLife: Double, timeUnit: String): this.type = { if (timeUnit != StreamingKMeans.BATCHES && timeUnit != StreamingKMeans.POINTS) { throw new IllegalArgumentException("Invalid time unit for decay: " + timeUnit) @@ -213,8 +212,8 @@ class StreamingKMeans( /** * Specify initial centers directly. - * @since 1.2.0 */ + @Since("1.2.0") def setInitialCenters(centers: Array[Vector], weights: Array[Double]): this.type = { model = new StreamingKMeansModel(centers, weights) this @@ -226,8 +225,8 @@ class StreamingKMeans( * @param dim Number of dimensions * @param weight Weight for each center * @param seed Random seed - * @since 1.2.0 */ + @Since("1.2.0") def setRandomCenters(dim: Int, weight: Double, seed: Long = Utils.random.nextLong): this.type = { val random = new XORShiftRandom(seed) val centers = Array.fill(k)(Vectors.dense(Array.fill(dim)(random.nextGaussian()))) @@ -238,8 +237,8 @@ class StreamingKMeans( /** * Return the latest model. - * @since 1.2.0 */ + @Since("1.2.0") def latestModel(): StreamingKMeansModel = { model } @@ -251,8 +250,8 @@ class StreamingKMeans( * and updates the model using each batch of data from the stream. * * @param data DStream containing vector data - * @since 1.2.0 */ + @Since("1.2.0") def trainOn(data: DStream[Vector]) { assertInitialized() data.foreachRDD { (rdd, time) => @@ -262,8 +261,8 @@ class StreamingKMeans( /** * Java-friendly version of `trainOn`. - * @since 1.4.0 */ + @Since("1.4.0") def trainOn(data: JavaDStream[Vector]): Unit = trainOn(data.dstream) /** @@ -271,8 +270,8 @@ class StreamingKMeans( * * @param data DStream containing vector data * @return DStream containing predictions - * @since 1.2.0 */ + @Since("1.2.0") def predictOn(data: DStream[Vector]): DStream[Int] = { assertInitialized() data.map(model.predict) @@ -280,8 +279,8 @@ class StreamingKMeans( /** * Java-friendly version of `predictOn`. - * @since 1.4.0 */ + @Since("1.4.0") def predictOn(data: JavaDStream[Vector]): JavaDStream[java.lang.Integer] = { JavaDStream.fromDStream(predictOn(data.dstream).asInstanceOf[DStream[java.lang.Integer]]) } @@ -292,8 +291,8 @@ class StreamingKMeans( * @param data DStream containing (key, feature vector) pairs * @tparam K key type * @return DStream containing the input keys and the predictions as values - * @since 1.2.0 */ + @Since("1.2.0") def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Int)] = { assertInitialized() data.mapValues(model.predict) @@ -301,8 +300,8 @@ class StreamingKMeans( /** * Java-friendly version of `predictOnValues`. - * @since 1.4.0 */ + @Since("1.4.0") def predictOnValues[K]( data: JavaPairDStream[K, Vector]): JavaPairDStream[K, java.lang.Integer] = { implicit val tag = fakeClassTag[K] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 486741edd6f5a..76ae847921f44 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.evaluation -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.evaluation.binary._ @@ -41,8 +41,8 @@ import org.apache.spark.sql.DataFrame * of bins may not exactly equal numBins. The last bin in each partition may * be smaller as a result, meaning there may be an extra sample at * partition boundaries. - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class BinaryClassificationMetrics( val scoreAndLabels: RDD[(Double, Double)], @@ -52,8 +52,8 @@ class BinaryClassificationMetrics( /** * Defaults `numBins` to 0. - * @since 1.0.0 */ + @Since("1.0.0") def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0) /** @@ -65,16 +65,16 @@ class BinaryClassificationMetrics( /** * Unpersist intermediate RDDs used in the computation. - * @since 1.0.0 */ + @Since("1.0.0") def unpersist() { cumulativeCounts.unpersist() } /** * Returns thresholds in descending order. - * @since 1.0.0 */ + @Since("1.0.0") def thresholds(): RDD[Double] = cumulativeCounts.map(_._1) /** @@ -82,8 +82,8 @@ class BinaryClassificationMetrics( * which is an RDD of (false positive rate, true positive rate) * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic - * @since 1.0.0 */ + @Since("1.0.0") def roc(): RDD[(Double, Double)] = { val rocCurve = createCurve(FalsePositiveRate, Recall) val sc = confusions.context @@ -94,16 +94,16 @@ class BinaryClassificationMetrics( /** * Computes the area under the receiver operating characteristic (ROC) curve. - * @since 1.0.0 */ + @Since("1.0.0") def areaUnderROC(): Double = AreaUnderCurve.of(roc()) /** * Returns the precision-recall curve, which is an RDD of (recall, precision), * NOT (precision, recall), with (0.0, 1.0) prepended to it. * @see http://en.wikipedia.org/wiki/Precision_and_recall - * @since 1.0.0 */ + @Since("1.0.0") def pr(): RDD[(Double, Double)] = { val prCurve = createCurve(Recall, Precision) val sc = confusions.context @@ -113,8 +113,8 @@ class BinaryClassificationMetrics( /** * Computes the area under the precision-recall curve. - * @since 1.0.0 */ + @Since("1.0.0") def areaUnderPR(): Double = AreaUnderCurve.of(pr()) /** @@ -122,26 +122,26 @@ class BinaryClassificationMetrics( * @param beta the beta factor in F-Measure computation. * @return an RDD of (threshold, F-Measure) pairs. * @see http://en.wikipedia.org/wiki/F1_score - * @since 1.0.0 */ + @Since("1.0.0") def fMeasureByThreshold(beta: Double): RDD[(Double, Double)] = createCurve(FMeasure(beta)) /** * Returns the (threshold, F-Measure) curve with beta = 1.0. - * @since 1.0.0 */ + @Since("1.0.0") def fMeasureByThreshold(): RDD[(Double, Double)] = fMeasureByThreshold(1.0) /** * Returns the (threshold, precision) curve. - * @since 1.0.0 */ + @Since("1.0.0") def precisionByThreshold(): RDD[(Double, Double)] = createCurve(Precision) /** * Returns the (threshold, recall) curve. - * @since 1.0.0 */ + @Since("1.0.0") def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) private lazy val ( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index dddfa3ea5b800..02e89d921033c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.evaluation import scala.collection.Map import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.{Matrices, Matrix} import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -30,8 +30,8 @@ import org.apache.spark.sql.DataFrame * Evaluator for multiclass classification. * * @param predictionAndLabels an RDD of (prediction, label) pairs. - * @since 1.1.0 */ +@Since("1.1.0") @Experimental class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { @@ -65,8 +65,8 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { * predicted classes are in columns, * they are ordered by class label ascending, * as in "labels" - * @since 1.1.0 */ + @Since("1.1.0") def confusionMatrix: Matrix = { val n = labels.size val values = Array.ofDim[Double](n * n) @@ -85,15 +85,15 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns true positive rate for a given label (category) * @param label the label. - * @since 1.1.0 */ + @Since("1.1.0") def truePositiveRate(label: Double): Double = recall(label) /** * Returns false positive rate for a given label (category) * @param label the label. - * @since 1.1.0 */ + @Since("1.1.0") def falsePositiveRate(label: Double): Double = { val fp = fpByClass.getOrElse(label, 0) fp.toDouble / (labelCount - labelCountByClass(label)) @@ -102,8 +102,8 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns precision for a given label (category) * @param label the label. - * @since 1.1.0 */ + @Since("1.1.0") def precision(label: Double): Double = { val tp = tpByClass(label) val fp = fpByClass.getOrElse(label, 0) @@ -113,16 +113,16 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns recall for a given label (category) * @param label the label. - * @since 1.1.0 */ + @Since("1.1.0") def recall(label: Double): Double = tpByClass(label).toDouble / labelCountByClass(label) /** * Returns f-measure for a given label (category) * @param label the label. * @param beta the beta parameter. - * @since 1.1.0 */ + @Since("1.1.0") def fMeasure(label: Double, beta: Double): Double = { val p = precision(label) val r = recall(label) @@ -133,8 +133,8 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns f1-measure for a given label (category) * @param label the label. - * @since 1.1.0 */ + @Since("1.1.0") def fMeasure(label: Double): Double = fMeasure(label, 1.0) /** @@ -187,8 +187,8 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns weighted averaged f-measure * @param beta the beta parameter. - * @since 1.1.0 */ + @Since("1.1.0") def weightedFMeasure(beta: Double): Double = labelCountByClass.map { case (category, count) => fMeasure(category, beta) * count.toDouble / labelCount }.sum diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala index 77cb1e09bdbb5..a0a8d9c56847b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.evaluation +import org.apache.spark.annotation.Since import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.sql.DataFrame @@ -25,8 +26,8 @@ import org.apache.spark.sql.DataFrame * Evaluator for multilabel classification. * @param predictionAndLabels an RDD of (predictions, labels) pairs, * both are non-null Arrays, each with unique elements. - * @since 1.2.0 */ +@Since("1.2.0") class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])]) { /** @@ -104,8 +105,8 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns precision for a given label (category) * @param label the label. - * @since 1.2.0 */ + @Since("1.2.0") def precision(label: Double): Double = { val tp = tpPerClass(label) val fp = fpPerClass.getOrElse(label, 0L) @@ -115,8 +116,8 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns recall for a given label (category) * @param label the label. - * @since 1.2.0 */ + @Since("1.2.0") def recall(label: Double): Double = { val tp = tpPerClass(label) val fn = fnPerClass.getOrElse(label, 0L) @@ -126,8 +127,8 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns f1-measure for a given label (category) * @param label the label. - * @since 1.2.0 */ + @Since("1.2.0") def f1Measure(label: Double): Double = { val p = precision(label) val r = recall(label) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index 063fbed8cdeea..a7f43f0b110f5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} import org.apache.spark.rdd.RDD @@ -34,8 +34,8 @@ import org.apache.spark.rdd.RDD * Java users should use [[RankingMetrics$.of]] to create a [[RankingMetrics]] instance. * * @param predictionAndLabels an RDD of (predicted ranking, ground truth set) pairs. - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])]) extends Logging with Serializable { @@ -56,8 +56,8 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] * * @param k the position to compute the truncated precision, must be positive * @return the average precision at the first k ranking positions - * @since 1.2.0 */ + @Since("1.2.0") def precisionAt(k: Int): Double = { require(k > 0, "ranking position k should be positive") predictionAndLabels.map { case (pred, lab) => @@ -126,8 +126,8 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] * * @param k the position to compute the truncated ndcg, must be positive * @return the average ndcg at the first k ranking positions - * @since 1.2.0 */ + @Since("1.2.0") def ndcgAt(k: Int): Double = { require(k > 0, "ranking position k should be positive") predictionAndLabels.map { case (pred, lab) => @@ -165,8 +165,8 @@ object RankingMetrics { /** * Creates a [[RankingMetrics]] instance (for Java users). * @param predictionAndLabels a JavaRDD of (predicted ranking, ground truth set) pairs - * @since 1.4.0 */ + @Since("1.4.0") def of[E, T <: jl.Iterable[E]](predictionAndLabels: JavaRDD[(T, T)]): RankingMetrics[E] = { implicit val tag = JavaSparkContext.fakeClassTag[E] val rdd = predictionAndLabels.rdd.map { case (predictions, labels) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 54dfd8c099494..36a6c357c3897 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.evaluation -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.rdd.RDD import org.apache.spark.Logging import org.apache.spark.mllib.linalg.Vectors @@ -29,8 +29,8 @@ import org.apache.spark.sql.DataFrame * Evaluator for regression. * * @param predictionAndObservations an RDD of (prediction, observation) pairs. - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extends Logging { @@ -67,8 +67,8 @@ class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extend * Returns the variance explained by regression. * explainedVariance = \sum_i (\hat{y_i} - \bar{y})^2 / n * @see [[https://en.wikipedia.org/wiki/Fraction_of_variance_unexplained]] - * @since 1.2.0 */ + @Since("1.2.0") def explainedVariance: Double = { SSreg / summary.count } @@ -76,8 +76,8 @@ class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extend /** * Returns the mean absolute error, which is a risk function corresponding to the * expected value of the absolute error loss or l1-norm loss. - * @since 1.2.0 */ + @Since("1.2.0") def meanAbsoluteError: Double = { summary.normL1(1) / summary.count } @@ -85,8 +85,8 @@ class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extend /** * Returns the mean squared error, which is a risk function corresponding to the * expected value of the squared error loss or quadratic loss. - * @since 1.2.0 */ + @Since("1.2.0") def meanSquaredError: Double = { SSerr / summary.count } @@ -94,8 +94,8 @@ class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extend /** * Returns the root mean squared error, which is defined as the square root of * the mean squared error. - * @since 1.2.0 */ + @Since("1.2.0") def rootMeanSquaredError: Double = { math.sqrt(this.meanSquaredError) } @@ -103,8 +103,8 @@ class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extend /** * Returns R^2^, the unadjusted coefficient of determination. * @see [[http://en.wikipedia.org/wiki/Coefficient_of_determination]] - * @since 1.2.0 */ + @Since("1.2.0") def r2: Double = { 1 - SSerr / SStot } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 7f4de77044994..ba3b447a83398 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -20,7 +20,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.fpm.AssociationRules.Rule @@ -33,24 +33,22 @@ import org.apache.spark.rdd.RDD * Generates association rules from a [[RDD[FreqItemset[Item]]]. This method only generates * association rules which have a single item as the consequent. * - * @since 1.5.0 */ +@Since("1.5.0") @Experimental class AssociationRules private[fpm] ( private var minConfidence: Double) extends Logging with Serializable { /** * Constructs a default instance with default parameters {minConfidence = 0.8}. - * - * @since 1.5.0 */ + @Since("1.5.0") def this() = this(0.8) /** * Sets the minimal confidence (default: `0.8`). - * - * @since 1.5.0 */ + @Since("1.5.0") def setMinConfidence(minConfidence: Double): this.type = { require(minConfidence >= 0.0 && minConfidence <= 1.0) this.minConfidence = minConfidence @@ -62,8 +60,8 @@ class AssociationRules private[fpm] ( * @param freqItemsets frequent itemset model obtained from [[FPGrowth]] * @return a [[Set[Rule[Item]]] containing the assocation rules. * - * @since 1.5.0 */ + @Since("1.5.0") def run[Item: ClassTag](freqItemsets: RDD[FreqItemset[Item]]): RDD[Rule[Item]] = { // For candidate rule X => Y, generate (X, (Y, freq(X union Y))) val candidates = freqItemsets.flatMap { itemset => @@ -102,8 +100,8 @@ object AssociationRules { * instead. * @tparam Item item type * - * @since 1.5.0 */ + @Since("1.5.0") @Experimental class Rule[Item] private[fpm] ( val antecedent: Array[Item], @@ -114,8 +112,8 @@ object AssociationRules { /** * Returns the confidence of the rule. * - * @since 1.5.0 */ + @Since("1.5.0") def confidence: Double = freqUnion.toDouble / freqAntecedent require(antecedent.toSet.intersect(consequent.toSet).isEmpty, { @@ -127,8 +125,8 @@ object AssociationRules { /** * Returns antecedent in a Java List. * - * @since 1.5.0 */ + @Since("1.5.0") def javaAntecedent: java.util.List[Item] = { antecedent.toList.asJava } @@ -136,8 +134,8 @@ object AssociationRules { /** * Returns consequent in a Java List. * - * @since 1.5.0 */ + @Since("1.5.0") def javaConsequent: java.util.List[Item] = { consequent.toList.asJava } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index e2370a52f4930..e37f806271680 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.fpm.FPGrowth._ @@ -39,15 +39,15 @@ import org.apache.spark.storage.StorageLevel * @param freqItemsets frequent itemset, which is an RDD of [[FreqItemset]] * @tparam Item item type * - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable { /** * Generates association rules for the [[Item]]s in [[freqItemsets]]. * @param confidence minimal confidence of the rules produced - * @since 1.5.0 */ + @Since("1.5.0") def generateAssociationRules(confidence: Double): RDD[AssociationRules.Rule[Item]] = { val associationRules = new AssociationRules(confidence) associationRules.run(freqItemsets) @@ -71,8 +71,8 @@ class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) ex * @see [[http://en.wikipedia.org/wiki/Association_rule_learning Association rule learning * (Wikipedia)]] * - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class FPGrowth private ( private var minSupport: Double, @@ -82,15 +82,15 @@ class FPGrowth private ( * Constructs a default instance with default parameters {minSupport: `0.3`, numPartitions: same * as the input data}. * - * @since 1.3.0 */ + @Since("1.3.0") def this() = this(0.3, -1) /** * Sets the minimal support level (default: `0.3`). * - * @since 1.3.0 */ + @Since("1.3.0") def setMinSupport(minSupport: Double): this.type = { this.minSupport = minSupport this @@ -99,8 +99,8 @@ class FPGrowth private ( /** * Sets the number of partitions used by parallel FP-growth (default: same as input data). * - * @since 1.3.0 */ + @Since("1.3.0") def setNumPartitions(numPartitions: Int): this.type = { this.numPartitions = numPartitions this @@ -111,8 +111,8 @@ class FPGrowth private ( * @param data input data set, each element contains a transaction * @return an [[FPGrowthModel]] * - * @since 1.3.0 */ + @Since("1.3.0") def run[Item: ClassTag](data: RDD[Array[Item]]): FPGrowthModel[Item] = { if (data.getStorageLevel == StorageLevel.NONE) { logWarning("Input data is not cached.") @@ -213,8 +213,8 @@ class FPGrowth private ( /** * :: Experimental :: * - * @since 1.3.0 */ +@Since("1.3.0") @Experimental object FPGrowth { @@ -224,15 +224,15 @@ object FPGrowth { * @param freq frequency * @tparam Item item type * - * @since 1.3.0 */ + @Since("1.3.0") class FreqItemset[Item](val items: Array[Item], val freq: Long) extends Serializable { /** * Returns items in a Java List. * - * @since 1.3.0 */ + @Since("1.3.0") def javaItems: java.util.List[Item] = { items.toList.asJava } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index dfa8910fcbc8c..28b5b4637bf17 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHash import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ @@ -227,8 +227,8 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { * @param values matrix entries in column major if not transposed or in row major otherwise * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. - * @since 1.0.0 */ +@Since("1.0.0") @SQLUserDefinedType(udt = classOf[MatrixUDT]) class DenseMatrix( val numRows: Int, @@ -253,8 +253,8 @@ class DenseMatrix( * @param numRows number of rows * @param numCols number of columns * @param values matrix entries in column major - * @since 1.3.0 */ + @Since("1.3.0") def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) @@ -278,9 +278,7 @@ class DenseMatrix( private[mllib] def apply(i: Int): Double = values(i) - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def apply(i: Int, j: Int): Double = values(index(i, j)) private[mllib] def index(i: Int, j: Int): Int = { @@ -291,9 +289,7 @@ class DenseMatrix( values(index(i, j)) = v } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def copy: DenseMatrix = new DenseMatrix(numRows, numCols, values.clone()) private[spark] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f), @@ -309,9 +305,7 @@ class DenseMatrix( this } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def transpose: DenseMatrix = new DenseMatrix(numCols, numRows, values, !isTransposed) private[spark] override def foreachActive(f: (Int, Int, Double) => Unit): Unit = { @@ -342,21 +336,17 @@ class DenseMatrix( } } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def numNonzeros: Int = values.count(_ != 0) - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def numActives: Int = values.length /** * Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed * set to false. - * @since 1.3.0 */ + @Since("1.3.0") def toSparse: SparseMatrix = { val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble val colPtrs: Array[Int] = new Array[Int](numCols + 1) @@ -383,8 +373,8 @@ class DenseMatrix( /** * Factory methods for [[org.apache.spark.mllib.linalg.DenseMatrix]]. - * @since 1.3.0 */ +@Since("1.3.0") object DenseMatrix { /** @@ -392,8 +382,8 @@ object DenseMatrix { * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros - * @since 1.3.0 */ + @Since("1.3.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -405,8 +395,8 @@ object DenseMatrix { * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones - * @since 1.3.0 */ + @Since("1.3.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -417,8 +407,8 @@ object DenseMatrix { * Generate an Identity Matrix in `DenseMatrix` format. * @param n number of rows and columns of the matrix * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal - * @since 1.3.0 */ + @Since("1.3.0") def eye(n: Int): DenseMatrix = { val identity = DenseMatrix.zeros(n, n) var i = 0 @@ -435,8 +425,8 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -449,8 +439,8 @@ object DenseMatrix { * @param numCols number of columns of the matrix * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { require(numRows.toLong * numCols <= Int.MaxValue, s"$numRows x $numCols dense matrix is too large to allocate") @@ -462,8 +452,8 @@ object DenseMatrix { * @param vector a `Vector` that will form the values on the diagonal of the matrix * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` * on the diagonal - * @since 1.3.0 */ + @Since("1.3.0") def diag(vector: Vector): DenseMatrix = { val n = vector.size val matrix = DenseMatrix.zeros(n, n) @@ -498,8 +488,8 @@ object DenseMatrix { * @param isTransposed whether the matrix is transposed. If true, the matrix can be considered * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. - * @since 1.2.0 */ +@Since("1.2.0") @SQLUserDefinedType(udt = classOf[MatrixUDT]) class SparseMatrix( val numRows: Int, @@ -536,8 +526,8 @@ class SparseMatrix( * @param rowIndices the row index of the entry. They must be in strictly increasing * order for each column * @param values non-zero matrix entries in column major - * @since 1.3.0 */ + @Since("1.3.0") def this( numRows: Int, numCols: Int, @@ -560,8 +550,8 @@ class SparseMatrix( } /** - * @since 1.3.0 */ + @Since("1.3.0") override def apply(i: Int, j: Int): Double = { val ind = index(i, j) if (ind < 0) 0.0 else values(ind) @@ -585,9 +575,7 @@ class SparseMatrix( } } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def copy: SparseMatrix = { new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) } @@ -605,9 +593,7 @@ class SparseMatrix( this } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def transpose: SparseMatrix = new SparseMatrix(numCols, numRows, colPtrs, rowIndices, values, !isTransposed) @@ -641,28 +627,24 @@ class SparseMatrix( /** * Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed * set to false. - * @since 1.3.0 */ + @Since("1.3.0") def toDense: DenseMatrix = { new DenseMatrix(numRows, numCols, toArray) } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def numNonzeros: Int = values.count(_ != 0) - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def numActives: Int = values.length } /** * Factory methods for [[org.apache.spark.mllib.linalg.SparseMatrix]]. - * @since 1.3.0 */ +@Since("1.3.0") object SparseMatrix { /** @@ -673,8 +655,8 @@ object SparseMatrix { * @param numCols number of columns of the matrix * @param entries Array of (i, j, value) tuples * @return The corresponding `SparseMatrix` - * @since 1.3.0 */ + @Since("1.3.0") def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = { val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1)) val numEntries = sortedEntries.size @@ -722,8 +704,8 @@ object SparseMatrix { * Generate an Identity Matrix in `SparseMatrix` format. * @param n number of rows and columns of the matrix * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal - * @since 1.3.0 */ + @Since("1.3.0") def speye(n: Int): SparseMatrix = { new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) } @@ -792,8 +774,8 @@ object SparseMatrix { * @param density the desired density for the matrix * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextDouble()) @@ -806,8 +788,8 @@ object SparseMatrix { * @param density the desired density for the matrix * @param rng a random number generator * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { val mat = genRandMatrix(numRows, numCols, density, rng) mat.update(i => rng.nextGaussian()) @@ -818,8 +800,8 @@ object SparseMatrix { * @param vector a `Vector` that will form the values on the diagonal of the matrix * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero * `values` on the diagonal - * @since 1.3.0 */ + @Since("1.3.0") def spdiag(vector: Vector): SparseMatrix = { val n = vector.size vector match { @@ -835,8 +817,8 @@ object SparseMatrix { /** * Factory methods for [[org.apache.spark.mllib.linalg.Matrix]]. - * @since 1.0.0 */ +@Since("1.0.0") object Matrices { /** @@ -845,8 +827,8 @@ object Matrices { * @param numRows number of rows * @param numCols number of columns * @param values matrix entries in column major - * @since 1.0.0 */ + @Since("1.0.0") def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { new DenseMatrix(numRows, numCols, values) } @@ -859,8 +841,8 @@ object Matrices { * @param colPtrs the index corresponding to the start of a new column * @param rowIndices the row index of the entry * @param values non-zero matrix entries in column major - * @since 1.2.0 */ + @Since("1.2.0") def sparse( numRows: Int, numCols: Int, @@ -893,8 +875,8 @@ object Matrices { * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of zeros - * @since 1.2.0 */ + @Since("1.2.0") def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** @@ -902,24 +884,24 @@ object Matrices { * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @return `Matrix` with size `numRows` x `numCols` and values of ones - * @since 1.2.0 */ + @Since("1.2.0") def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** * Generate a dense Identity Matrix in `Matrix` format. * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal - * @since 1.2.0 */ + @Since("1.2.0") def eye(n: Int): Matrix = DenseMatrix.eye(n) /** * Generate a sparse Identity Matrix in `Matrix` format. * @param n number of rows and columns of the matrix * @return `Matrix` with size `n` x `n` and values of ones on the diagonal - * @since 1.3.0 */ + @Since("1.3.0") def speye(n: Int): Matrix = SparseMatrix.speye(n) /** @@ -928,8 +910,8 @@ object Matrices { * @param numCols number of columns of the matrix * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) - * @since 1.2.0 */ + @Since("1.2.0") def rand(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.rand(numRows, numCols, rng) @@ -940,8 +922,8 @@ object Matrices { * @param density the desired density for the matrix * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprand(numRows, numCols, density, rng) @@ -951,8 +933,8 @@ object Matrices { * @param numCols number of columns of the matrix * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) - * @since 1.2.0 */ + @Since("1.2.0") def randn(numRows: Int, numCols: Int, rng: Random): Matrix = DenseMatrix.randn(numRows, numCols, rng) @@ -963,8 +945,8 @@ object Matrices { * @param density the desired density for the matrix * @param rng a random number generator * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) - * @since 1.3.0 */ + @Since("1.3.0") def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = SparseMatrix.sprandn(numRows, numCols, density, rng) @@ -973,8 +955,8 @@ object Matrices { * @param vector a `Vector` that will form the values on the diagonal of the matrix * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal - * @since 1.2.0 */ + @Since("1.2.0") def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) /** @@ -983,8 +965,8 @@ object Matrices { * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were horizontally concatenated - * @since 1.3.0 */ + @Since("1.3.0") def horzcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) @@ -1042,8 +1024,8 @@ object Matrices { * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned. * @param matrices array of matrices * @return a single `Matrix` composed of the matrices that were vertically concatenated - * @since 1.3.0 */ + @Since("1.3.0") def vertcat(matrices: Array[Matrix]): Matrix = { if (matrices.isEmpty) { return new DenseMatrix(0, 0, Array[Double]()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 8f504f6984cb0..a37aca99d5e72 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} /** * :: Experimental :: * Represents singular value decomposition (SVD) factors. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 52ef7be3b38be..3d577edbe23e1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.apache.spark.SparkException -import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.annotation.{AlphaComponent, Since} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow @@ -240,14 +240,14 @@ class VectorUDT extends UserDefinedType[Vector] { * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. * We don't use the name `Vector` because Scala imports * [[scala.collection.immutable.Vector]] by default. - * @since 1.0.0 */ +@Since("1.0.0") object Vectors { /** * Creates a dense vector from its values. - * @since 1.0.0 */ + @Since("1.0.0") @varargs def dense(firstValue: Double, otherValues: Double*): Vector = new DenseVector((firstValue +: otherValues).toArray) @@ -255,8 +255,8 @@ object Vectors { // A dummy implicit is used to avoid signature collision with the one generated by @varargs. /** * Creates a dense vector from a double array. - * @since 1.0.0 */ + @Since("1.0.0") def dense(values: Array[Double]): Vector = new DenseVector(values) /** @@ -265,8 +265,8 @@ object Vectors { * @param size vector size. * @param indices index array, must be strictly increasing. * @param values value array, must have the same length as indices. - * @since 1.0.0 */ + @Since("1.0.0") def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = new SparseVector(size, indices, values) @@ -275,8 +275,8 @@ object Vectors { * * @param size vector size. * @param elements vector elements in (index, value) pairs. - * @since 1.0.0 */ + @Since("1.0.0") def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { require(size > 0, "The size of the requested sparse vector must be greater than 0.") @@ -297,8 +297,8 @@ object Vectors { * * @param size vector size. * @param elements vector elements in (index, value) pairs. - * @since 1.0.0 */ + @Since("1.0.0") def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { sparse(size, elements.asScala.map { case (i, x) => (i.intValue(), x.doubleValue()) @@ -310,16 +310,16 @@ object Vectors { * * @param size vector size * @return a zero vector - * @since 1.1.0 */ + @Since("1.1.0") def zeros(size: Int): Vector = { new DenseVector(new Array[Double](size)) } /** * Parses a string resulted from [[Vector.toString]] into a [[Vector]]. - * @since 1.1.0 */ + @Since("1.1.0") def parse(s: String): Vector = { parseNumeric(NumericParser.parse(s)) } @@ -362,8 +362,8 @@ object Vectors { * @param vector input vector. * @param p norm. * @return norm in L^p^ space. - * @since 1.3.0 */ + @Since("1.3.0") def norm(vector: Vector, p: Double): Double = { require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + s"You specified p=$p.") @@ -415,8 +415,8 @@ object Vectors { * @param v1 first Vector. * @param v2 second Vector. * @return squared distance between two Vectors. - * @since 1.3.0 */ + @Since("1.3.0") def sqdist(v1: Vector, v2: Vector): Double = { require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + s"=${v2.size}.") @@ -529,33 +529,25 @@ object Vectors { /** * A dense vector represented by a value array. - * @since 1.0.0 */ +@Since("1.0.0") @SQLUserDefinedType(udt = classOf[VectorUDT]) class DenseVector(val values: Array[Double]) extends Vector { - /** - * @since 1.0.0 - */ + @Since("1.0.0") override def size: Int = values.length override def toString: String = values.mkString("[", ",", "]") - /** - * @since 1.0.0 - */ + @Since("1.0.0") override def toArray: Array[Double] = values private[spark] override def toBreeze: BV[Double] = new BDV[Double](values) - /** - * @since 1.0.0 - */ + @Since("1.0.0") override def apply(i: Int): Double = values(i) - /** - * @since 1.1.0 - */ + @Since("1.1.0") override def copy: DenseVector = { new DenseVector(values.clone()) } @@ -587,14 +579,10 @@ class DenseVector(val values: Array[Double]) extends Vector { result } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def numActives: Int = size - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def numNonzeros: Int = { // same as values.count(_ != 0.0) but faster var nnz = 0 @@ -606,9 +594,7 @@ class DenseVector(val values: Array[Double]) extends Vector { nnz } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def toSparse: SparseVector = { val nnz = numNonzeros val ii = new Array[Int](nnz) @@ -624,9 +610,7 @@ class DenseVector(val values: Array[Double]) extends Vector { new SparseVector(size, ii, vv) } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def argmax: Int = { if (size == 0) { -1 @@ -646,9 +630,7 @@ class DenseVector(val values: Array[Double]) extends Vector { } } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object DenseVector { /** Extracts the value array from a dense vector. */ def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) @@ -660,8 +642,8 @@ object DenseVector { * @param size size of the vector. * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. - * @since 1.0.0 */ +@Since("1.0.0") @SQLUserDefinedType(udt = classOf[VectorUDT]) class SparseVector( override val size: Int, @@ -677,9 +659,7 @@ class SparseVector( override def toString: String = s"($size,${indices.mkString("[", ",", "]")},${values.mkString("[", ",", "]")})" - /** - * @since 1.0.0 - */ + @Since("1.0.0") override def toArray: Array[Double] = { val data = new Array[Double](size) var i = 0 @@ -691,9 +671,7 @@ class SparseVector( data } - /** - * @since 1.1.0 - */ + @Since("1.1.0") override def copy: SparseVector = { new SparseVector(size, indices.clone(), values.clone()) } @@ -734,14 +712,10 @@ class SparseVector( result } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def numActives: Int = values.length - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def numNonzeros: Int = { var nnz = 0 values.foreach { v => @@ -752,9 +726,7 @@ class SparseVector( nnz } - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def toSparse: SparseVector = { val nnz = numNonzeros if (nnz == numActives) { @@ -774,9 +746,7 @@ class SparseVector( } } - /** - * @since 1.5.0 - */ + @Since("1.5.0") override def argmax: Int = { if (size == 0) { -1 @@ -847,9 +817,7 @@ class SparseVector( } } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object SparseVector { def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index cfb6680a18b34..94376c24a7ac6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseMatrix => BDM} import org.apache.spark.{Logging, Partitioner, SparkException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -128,9 +128,8 @@ private[mllib] object GridPartitioner { * the number of rows will be calculated when `numRows` is invoked. * @param nCols Number of columns of this matrix. If the supplied value is less than or equal to * zero, the number of columns will be calculated when `numCols` is invoked. - * @since 1.3.0 - * */ +@Since("1.3.0") @Experimental class BlockMatrix( val blocks: RDD[((Int, Int), Matrix)], @@ -151,10 +150,8 @@ class BlockMatrix( * rows are not required to have the given number of rows * @param colsPerBlock Number of columns that make up each block. The blocks forming the final * columns are not required to have the given number of columns - * - * @since 1.3.0 - * */ + @Since("1.3.0") def this( blocks: RDD[((Int, Int), Matrix)], rowsPerBlock: Int, @@ -162,20 +159,13 @@ class BlockMatrix( this(blocks, rowsPerBlock, colsPerBlock, 0L, 0L) } - /** - * @since 1.3.0 - * */ - + @Since("1.3.0") override def numRows(): Long = { if (nRows <= 0L) estimateDim() nRows } - /** - * - * @since 1.3.0 - */ - + @Since("1.3.0") override def numCols(): Long = { if (nCols <= 0L) estimateDim() nCols @@ -206,8 +196,8 @@ class BlockMatrix( /** * Validates the block matrix info against the matrix data (`blocks`) and throws an exception if * any error is found. - * @since 1.3.0 */ + @Since("1.3.0") def validate(): Unit = { logDebug("Validating BlockMatrix...") // check if the matrix is larger than the claimed dimensions @@ -243,25 +233,22 @@ class BlockMatrix( logDebug("BlockMatrix is valid!") } - /** Caches the underlying RDD. - * @since 1.3.0 - * */ + /** Caches the underlying RDD. */ + @Since("1.3.0") def cache(): this.type = { blocks.cache() this } - /** Persists the underlying RDD with the specified storage level. - * @since 1.3.0 - * */ + /** Persists the underlying RDD with the specified storage level. */ + @Since("1.3.0") def persist(storageLevel: StorageLevel): this.type = { blocks.persist(storageLevel) this } - /** Converts to CoordinateMatrix. - * @since 1.3.0 - * */ + /** Converts to CoordinateMatrix. */ + @Since("1.3.0") def toCoordinateMatrix(): CoordinateMatrix = { val entryRDD = blocks.flatMap { case ((blockRowIndex, blockColIndex), mat) => val rowStart = blockRowIndex.toLong * rowsPerBlock @@ -275,9 +262,8 @@ class BlockMatrix( new CoordinateMatrix(entryRDD, numRows(), numCols()) } - /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. - * @since 1.3.0 - * */ + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ + @Since("1.3.0") def toIndexedRowMatrix(): IndexedRowMatrix = { require(numCols() < Int.MaxValue, "The number of columns must be within the integer range. " + s"numCols: ${numCols()}") @@ -285,9 +271,8 @@ class BlockMatrix( toCoordinateMatrix().toIndexedRowMatrix() } - /** Collect the distributed matrix on the driver as a `DenseMatrix`. - * @since 1.3.0 - * */ + /** Collect the distributed matrix on the driver as a `DenseMatrix`. */ + @Since("1.3.0") def toLocalMatrix(): Matrix = { require(numRows() < Int.MaxValue, "The number of rows of this matrix should be less than " + s"Int.MaxValue. Currently numRows: ${numRows()}") @@ -312,11 +297,11 @@ class BlockMatrix( new DenseMatrix(m, n, values) } - /** Transpose this `BlockMatrix`. Returns a new `BlockMatrix` instance sharing the - * same underlying data. Is a lazy operation. - * @since 1.3.0 - * - * */ + /** + * Transpose this `BlockMatrix`. Returns a new `BlockMatrix` instance sharing the + * same underlying data. Is a lazy operation. + */ + @Since("1.3.0") def transpose: BlockMatrix = { val transposedBlocks = blocks.map { case ((blockRowIndex, blockColIndex), mat) => ((blockColIndex, blockRowIndex), mat.transpose) @@ -330,13 +315,14 @@ class BlockMatrix( new BDM[Double](localMat.numRows, localMat.numCols, localMat.toArray) } - /** Adds two block matrices together. The matrices must have the same size and matching - * `rowsPerBlock` and `colsPerBlock` values. If one of the blocks that are being added are - * instances of [[SparseMatrix]], the resulting sub matrix will also be a [[SparseMatrix]], even - * if it is being added to a [[DenseMatrix]]. If two dense matrices are added, the output will - * also be a [[DenseMatrix]]. - * @since 1.3.0 - */ + /** + * Adds two block matrices together. The matrices must have the same size and matching + * `rowsPerBlock` and `colsPerBlock` values. If one of the blocks that are being added are + * instances of [[SparseMatrix]], the resulting sub matrix will also be a [[SparseMatrix]], even + * if it is being added to a [[DenseMatrix]]. If two dense matrices are added, the output will + * also be a [[DenseMatrix]]. + */ + @Since("1.3.0") def add(other: BlockMatrix): BlockMatrix = { require(numRows() == other.numRows(), "Both matrices must have the same number of rows. " + s"A.numRows: ${numRows()}, B.numRows: ${other.numRows()}") @@ -364,14 +350,14 @@ class BlockMatrix( } } - /** Left multiplies this [[BlockMatrix]] to `other`, another [[BlockMatrix]]. The `colsPerBlock` - * of this matrix must equal the `rowsPerBlock` of `other`. If `other` contains - * [[SparseMatrix]], they will have to be converted to a [[DenseMatrix]]. The output - * [[BlockMatrix]] will only consist of blocks of [[DenseMatrix]]. This may cause - * some performance issues until support for multiplying two sparse matrices is added. - * - * @since 1.3.0 - */ + /** + * Left multiplies this [[BlockMatrix]] to `other`, another [[BlockMatrix]]. The `colsPerBlock` + * of this matrix must equal the `rowsPerBlock` of `other`. If `other` contains + * [[SparseMatrix]], they will have to be converted to a [[DenseMatrix]]. The output + * [[BlockMatrix]] will only consist of blocks of [[DenseMatrix]]. This may cause + * some performance issues until support for multiplying two sparse matrices is added. + */ + @Since("1.3.0") def multiply(other: BlockMatrix): BlockMatrix = { require(numCols() == other.numRows(), "The number of columns of A and the number of rows " + s"of B must be equal. A.numCols: ${numCols()}, B.numRows: ${other.numRows()}. If you " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 2b751e45dd76c..4bb27ec840902 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Matrix, SparseMatrix, Vectors} @@ -29,8 +29,8 @@ import org.apache.spark.mllib.linalg.{Matrix, SparseMatrix, Vectors} * @param i row index * @param j column index * @param value value of the entry - * @since 1.0.0 */ +@Since("1.0.0") @Experimental case class MatrixEntry(i: Long, j: Long, value: Double) @@ -43,22 +43,20 @@ case class MatrixEntry(i: Long, j: Long, value: Double) * be determined by the max row index plus one. * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the max column index plus one. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class CoordinateMatrix( val entries: RDD[MatrixEntry], private var nRows: Long, private var nCols: Long) extends DistributedMatrix { - /** Alternative constructor leaving matrix dimensions to be determined automatically. - * @since 1.0.0 - * */ + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + @Since("1.0.0") def this(entries: RDD[MatrixEntry]) = this(entries, 0L, 0L) - /** Gets or computes the number of columns. - * @since 1.0.0 - * */ + /** Gets or computes the number of columns. */ + @Since("1.0.0") override def numCols(): Long = { if (nCols <= 0L) { computeSize() @@ -66,9 +64,8 @@ class CoordinateMatrix( nCols } - /** Gets or computes the number of rows. - * @since 1.0.0 - * */ + /** Gets or computes the number of rows. */ + @Since("1.0.0") override def numRows(): Long = { if (nRows <= 0L) { computeSize() @@ -76,16 +73,14 @@ class CoordinateMatrix( nRows } - /** Transposes this CoordinateMatrix. - * @since 1.3.0 - * */ + /** Transposes this CoordinateMatrix. */ + @Since("1.3.0") def transpose(): CoordinateMatrix = { new CoordinateMatrix(entries.map(x => MatrixEntry(x.j, x.i, x.value)), numCols(), numRows()) } - /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. - * @since 1.0.0 - * */ + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ + @Since("1.0.0") def toIndexedRowMatrix(): IndexedRowMatrix = { val nl = numCols() if (nl > Int.MaxValue) { @@ -104,15 +99,14 @@ class CoordinateMatrix( /** * Converts to RowMatrix, dropping row indices after grouping by row index. * The number of columns must be within the integer range. - * @since 1.0.0 */ + @Since("1.0.0") def toRowMatrix(): RowMatrix = { toIndexedRowMatrix().toRowMatrix() } - /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. - * @since 1.3.0 - * */ + /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. */ + @Since("1.3.0") def toBlockMatrix(): BlockMatrix = { toBlockMatrix(1024, 1024) } @@ -124,8 +118,8 @@ class CoordinateMatrix( * @param colsPerBlock The number of columns of each block. The blocks at the right edge may have * a smaller value. Must be an integer value greater than 0. * @return a [[BlockMatrix]] - * @since 1.3.0 */ + @Since("1.3.0") def toBlockMatrix(rowsPerBlock: Int, colsPerBlock: Int): BlockMatrix = { require(rowsPerBlock > 0, s"rowsPerBlock needs to be greater than 0. rowsPerBlock: $rowsPerBlock") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 98e90af84abac..e51327ebb7b58 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -19,10 +19,12 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Since + /** * Represents a distributively stored matrix backed by one or more RDDs. - * @since 1.0.0 */ +@Since("1.0.0") trait DistributedMatrix extends Serializable { /** Gets or computes the number of rows. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index a09f88ce28e58..6d2c05a47d049 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.SingularValueDecomposition @@ -27,8 +27,8 @@ import org.apache.spark.mllib.linalg.SingularValueDecomposition /** * :: Experimental :: * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental case class IndexedRow(index: Long, vector: Vector) @@ -42,23 +42,19 @@ case class IndexedRow(index: Long, vector: Vector) * be determined by the max row index plus one. * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class IndexedRowMatrix( val rows: RDD[IndexedRow], private var nRows: Long, private var nCols: Int) extends DistributedMatrix { - /** Alternative constructor leaving matrix dimensions to be determined automatically. - * @since 1.0.0 - * */ + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + @Since("1.0.0") def this(rows: RDD[IndexedRow]) = this(rows, 0L, 0) - /** - * - * @since 1.0.0 - */ + @Since("1.0.0") override def numCols(): Long = { if (nCols <= 0) { // Calling `first` will throw an exception if `rows` is empty. @@ -67,10 +63,7 @@ class IndexedRowMatrix( nCols } - /** - * - * @since 1.0.0 - */ + @Since("1.0.0") override def numRows(): Long = { if (nRows <= 0L) { // Reduce will throw an exception if `rows` is empty. @@ -82,15 +75,14 @@ class IndexedRowMatrix( /** * Drops row indices and converts this matrix to a * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]]. - * @since 1.0.0 */ + @Since("1.0.0") def toRowMatrix(): RowMatrix = { new RowMatrix(rows.map(_.vector), 0L, nCols) } - /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. - * @since 1.3.0 - * */ + /** Converts to BlockMatrix. Creates blocks of [[SparseMatrix]] with size 1024 x 1024. */ + @Since("1.3.0") def toBlockMatrix(): BlockMatrix = { toBlockMatrix(1024, 1024) } @@ -102,8 +94,8 @@ class IndexedRowMatrix( * @param colsPerBlock The number of columns of each block. The blocks at the right edge may have * a smaller value. Must be an integer value greater than 0. * @return a [[BlockMatrix]] - * @since 1.3.0 */ + @Since("1.3.0") def toBlockMatrix(rowsPerBlock: Int, colsPerBlock: Int): BlockMatrix = { // TODO: This implementation may be optimized toCoordinateMatrix().toBlockMatrix(rowsPerBlock, colsPerBlock) @@ -112,8 +104,8 @@ class IndexedRowMatrix( /** * Converts this matrix to a * [[org.apache.spark.mllib.linalg.distributed.CoordinateMatrix]]. - * @since 1.3.0 */ + @Since("1.3.0") def toCoordinateMatrix(): CoordinateMatrix = { val entries = rows.flatMap { row => val rowIndex = row.index @@ -149,8 +141,8 @@ class IndexedRowMatrix( * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) * are treated as zero, where sigma(0) is the largest singular value. * @return SingularValueDecomposition(U, s, V) - * @since 1.0.0 */ + @Since("1.0.0") def computeSVD( k: Int, computeU: Boolean = false, @@ -176,8 +168,8 @@ class IndexedRowMatrix( * * @param B a local matrix whose number of rows must match the number of columns of this matrix * @return an IndexedRowMatrix representing the product, which preserves partitioning - * @since 1.0.0 */ + @Since("1.0.0") def multiply(B: Matrix): IndexedRowMatrix = { val mat = toRowMatrix().multiply(B) val indexedRows = rows.map(_.index).zip(mat.rows).map { case (i, v) => @@ -188,8 +180,8 @@ class IndexedRowMatrix( /** * Computes the Gramian matrix `A^T A`. - * @since 1.0.0 */ + @Since("1.0.0") def computeGramianMatrix(): Matrix = { toRowMatrix().computeGramianMatrix() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index b2e94f2dd6201..78036eba5c3e6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -28,7 +28,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.rdd.RDD @@ -44,22 +44,20 @@ import org.apache.spark.storage.StorageLevel * be determined by the number of records in the RDD `rows`. * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class RowMatrix( val rows: RDD[Vector], private var nRows: Long, private var nCols: Int) extends DistributedMatrix with Logging { - /** Alternative constructor leaving matrix dimensions to be determined automatically. - * @since 1.0.0 - * */ + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + @Since("1.0.0") def this(rows: RDD[Vector]) = this(rows, 0L, 0) - /** Gets or computes the number of columns. - * @since 1.0.0 - * */ + /** Gets or computes the number of columns. */ + @Since("1.0.0") override def numCols(): Long = { if (nCols <= 0) { try { @@ -74,9 +72,8 @@ class RowMatrix( nCols } - /** Gets or computes the number of rows. - * @since 1.0.0 - * */ + /** Gets or computes the number of rows. */ + @Since("1.0.0") override def numRows(): Long = { if (nRows <= 0L) { nRows = rows.count() @@ -114,8 +111,8 @@ class RowMatrix( /** * Computes the Gramian matrix `A^T A`. - * @since 1.0.0 */ + @Since("1.0.0") def computeGramianMatrix(): Matrix = { val n = numCols().toInt checkNumColumns(n) @@ -185,8 +182,8 @@ class RowMatrix( * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) * are treated as zero, where sigma(0) is the largest singular value. * @return SingularValueDecomposition(U, s, V). U = null if computeU = false. - * @since 1.0.0 */ + @Since("1.0.0") def computeSVD( k: Int, computeU: Boolean = false, @@ -326,8 +323,8 @@ class RowMatrix( /** * Computes the covariance matrix, treating each row as an observation. * @return a local dense matrix of size n x n - * @since 1.0.0 */ + @Since("1.0.0") def computeCovariance(): Matrix = { val n = numCols().toInt checkNumColumns(n) @@ -380,8 +377,8 @@ class RowMatrix( * * @param k number of top principal components. * @return a matrix of size n-by-k, whose columns are principal components - * @since 1.0.0 */ + @Since("1.0.0") def computePrincipalComponents(k: Int): Matrix = { val n = numCols().toInt require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") @@ -399,8 +396,8 @@ class RowMatrix( /** * Computes column-wise summary statistics. - * @since 1.0.0 */ + @Since("1.0.0") def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { val summary = rows.treeAggregate(new MultivariateOnlineSummarizer)( (aggregator, data) => aggregator.add(data), @@ -415,8 +412,8 @@ class RowMatrix( * @param B a local matrix whose number of rows must match the number of columns of this matrix * @return a [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] representing the product, * which preserves partitioning - * @since 1.0.0 */ + @Since("1.0.0") def multiply(B: Matrix): RowMatrix = { val n = numCols().toInt val k = B.numCols @@ -448,8 +445,8 @@ class RowMatrix( * * @return An n x n sparse upper-triangular matrix of cosine similarities between * columns of this matrix. - * @since 1.2.0 */ + @Since("1.2.0") def columnSimilarities(): CoordinateMatrix = { columnSimilarities(0.0) } @@ -492,8 +489,8 @@ class RowMatrix( * with the cost vs estimate quality trade-off described above. * @return An n x n sparse upper-triangular matrix of cosine similarities * between columns of this matrix. - * @since 1.2.0 */ + @Since("1.2.0") def columnSimilarities(threshold: Double): CoordinateMatrix = { require(threshold >= 0, s"Threshold cannot be negative: $threshold") @@ -671,9 +668,7 @@ class RowMatrix( } } -/** - * @since 1.0.0 - */ +@Since("1.0.0") @Experimental object RowMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 56c549ef99cb7..b27ef1b949e2e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.recommendation import org.apache.spark.Logging -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.recommendation.{ALS => NewALS} import org.apache.spark.rdd.RDD @@ -26,8 +26,8 @@ import org.apache.spark.storage.StorageLevel /** * A more compact class to represent a rating than Tuple3[Int, Int, Double]. - * @since 0.8.0 */ +@Since("0.8.0") case class Rating(user: Int, product: Int, rating: Double) /** @@ -255,8 +255,8 @@ class ALS private ( /** * Top-level methods for calling Alternating Least Squares (ALS) matrix factorization. - * @since 0.8.0 */ +@Since("0.8.0") object ALS { /** * Train a matrix factorization model given an RDD of ratings given by users to some products, @@ -271,8 +271,8 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into * @param seed random seed - * @since 0.9.1 */ + @Since("0.9.1") def train( ratings: RDD[Rating], rank: Int, @@ -296,8 +296,8 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into - * @since 0.8.0 */ + @Since("0.8.0") def train( ratings: RDD[Rating], rank: Int, @@ -319,8 +319,8 @@ object ALS { * @param rank number of features to use * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) - * @since 0.8.0 */ + @Since("0.8.0") def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) : MatrixFactorizationModel = { train(ratings, rank, iterations, lambda, -1) @@ -336,8 +336,8 @@ object ALS { * @param ratings RDD of (userID, productID, rating) pairs * @param rank number of features to use * @param iterations number of iterations of ALS (recommended: 10-20) - * @since 0.8.0 */ + @Since("0.8.0") def train(ratings: RDD[Rating], rank: Int, iterations: Int) : MatrixFactorizationModel = { train(ratings, rank, iterations, 0.01, -1) @@ -357,8 +357,8 @@ object ALS { * @param blocks level of parallelism to split computation into * @param alpha confidence parameter * @param seed random seed - * @since 0.8.1 */ + @Since("0.8.1") def trainImplicit( ratings: RDD[Rating], rank: Int, @@ -384,8 +384,8 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into * @param alpha confidence parameter - * @since 0.8.1 */ + @Since("0.8.1") def trainImplicit( ratings: RDD[Rating], rank: Int, @@ -409,8 +409,8 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param alpha confidence parameter - * @since 0.8.1 */ + @Since("0.8.1") def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double) : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, lambda, -1, alpha) @@ -427,8 +427,8 @@ object ALS { * @param ratings RDD of (userID, productID, rating) pairs * @param rank number of features to use * @param iterations number of iterations of ALS (recommended: 10-20) - * @since 0.8.1 */ + @Since("0.8.1") def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int) : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 261ca9cef0c5b..ba4cfdcd9f1dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -30,6 +30,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.annotation.Since import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.rdd.MLPairRDDFunctions._ @@ -49,8 +50,8 @@ import org.apache.spark.storage.StorageLevel * the features computed for this user. * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. - * @since 0.8.0 */ +@Since("0.8.0") class MatrixFactorizationModel( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], @@ -74,9 +75,8 @@ class MatrixFactorizationModel( } } - /** Predict the rating of one user for one product. - * @since 0.8.0 - */ + /** Predict the rating of one user for one product. */ + @Since("0.8.0") def predict(user: Int, product: Int): Double = { val userVector = userFeatures.lookup(user).head val productVector = productFeatures.lookup(product).head @@ -114,8 +114,8 @@ class MatrixFactorizationModel( * * @param usersProducts RDD of (user, product) pairs. * @return RDD of Ratings. - * @since 0.9.0 */ + @Since("0.9.0") def predict(usersProducts: RDD[(Int, Int)]): RDD[Rating] = { // Previously the partitions of ratings are only based on the given products. // So if the usersProducts given for prediction contains only few products or @@ -146,8 +146,8 @@ class MatrixFactorizationModel( /** * Java-friendly version of [[MatrixFactorizationModel.predict]]. - * @since 1.2.0 */ + @Since("1.2.0") def predict(usersProducts: JavaPairRDD[JavaInteger, JavaInteger]): JavaRDD[Rating] = { predict(usersProducts.rdd.asInstanceOf[RDD[(Int, Int)]]).toJavaRDD() } @@ -162,8 +162,8 @@ class MatrixFactorizationModel( * by score, decreasing. The first returned is the one predicted to be most strongly * recommended to the user. The score is an opaque value that indicates how strongly * recommended the product is. - * @since 1.1.0 */ + @Since("1.1.0") def recommendProducts(user: Int, num: Int): Array[Rating] = MatrixFactorizationModel.recommend(userFeatures.lookup(user).head, productFeatures, num) .map(t => Rating(user, t._1, t._2)) @@ -179,8 +179,8 @@ class MatrixFactorizationModel( * by score, decreasing. The first returned is the one predicted to be most strongly * recommended to the product. The score is an opaque value that indicates how strongly * recommended the user is. - * @since 1.1.0 */ + @Since("1.1.0") def recommendUsers(product: Int, num: Int): Array[Rating] = MatrixFactorizationModel.recommend(productFeatures.lookup(product).head, userFeatures, num) .map(t => Rating(t._1, product, t._2)) @@ -199,8 +199,8 @@ class MatrixFactorizationModel( * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. * If the directory already exists, this method throws an exception. - * @since 1.3.0 */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { MatrixFactorizationModel.SaveLoadV1_0.save(this, path) } @@ -212,8 +212,8 @@ class MatrixFactorizationModel( * @return [(Int, Array[Rating])] objects, where every tuple contains a userID and an array of * rating objects which contains the same userId, recommended productID and a "score" in the * rating field. Semantics of score is same as recommendProducts API - * @since 1.4.0 */ + @Since("1.4.0") def recommendProductsForUsers(num: Int): RDD[(Int, Array[Rating])] = { MatrixFactorizationModel.recommendForAll(rank, userFeatures, productFeatures, num).map { case (user, top) => @@ -230,8 +230,8 @@ class MatrixFactorizationModel( * @return [(Int, Array[Rating])] objects, where every tuple contains a productID and an array * of rating objects which contains the recommended userId, same productID and a "score" in the * rating field. Semantics of score is same as recommendUsers API - * @since 1.4.0 */ + @Since("1.4.0") def recommendUsersForProducts(num: Int): RDD[(Int, Array[Rating])] = { MatrixFactorizationModel.recommendForAll(rank, productFeatures, userFeatures, num).map { case (product, top) => @@ -241,9 +241,7 @@ class MatrixFactorizationModel( } } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { import org.apache.spark.mllib.util.Loader._ @@ -326,8 +324,8 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. * @return Model instance - * @since 1.3.0 */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): MatrixFactorizationModel = { val (loadedClassName, formatVersion, _) = loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 2980b94de35b0..509f6a2d169c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.feature.StandardScaler import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD @@ -35,8 +35,8 @@ import org.apache.spark.storage.StorageLevel * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. * - * @since 0.8.0 */ +@Since("0.8.0") @DeveloperApi abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) extends Serializable { @@ -56,8 +56,8 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction * - * @since 1.0.0 */ + @Since("1.0.0") def predict(testData: RDD[Vector]): RDD[Double] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. @@ -76,8 +76,8 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double * @param testData array representing a single data point * @return Double prediction from the trained model * - * @since 1.0.0 */ + @Since("1.0.0") def predict(testData: Vector): Double = { predictPoint(testData, weights, intercept) } @@ -95,8 +95,8 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. * - * @since 0.8.0 */ +@Since("0.8.0") @DeveloperApi abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { @@ -106,8 +106,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * The optimizer to solve the problem. * - * @since 1.0.0 */ + @Since("1.0.0") def optimizer: Optimizer /** Whether to add intercept (default: false). */ @@ -143,8 +143,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * The dimension of training features. * - * @since 1.4.0 */ + @Since("1.4.0") def getNumFeatures: Int = this.numFeatures /** @@ -168,16 +168,16 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * Get if the algorithm uses addIntercept * - * @since 1.4.0 */ + @Since("1.4.0") def isAddIntercept: Boolean = this.addIntercept /** * Set if the algorithm should add an intercept. Default false. * We set the default to false because adding the intercept will cause memory allocation. * - * @since 0.8.0 */ + @Since("0.8.0") def setIntercept(addIntercept: Boolean): this.type = { this.addIntercept = addIntercept this @@ -186,8 +186,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * Set if the algorithm should validate data before training. Default true. * - * @since 0.8.0 */ + @Since("0.8.0") def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this @@ -197,8 +197,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. * - * @since 0.8.0 */ + @Since("0.8.0") def run(input: RDD[LabeledPoint]): M = { if (numFeatures < 0) { numFeatures = input.map(_.features.size).first() @@ -231,8 +231,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Run the algorithm with the configured parameters on an input RDD * of LabeledPoint entries starting from the initial weights provided. * - * @since 1.0.0 */ + @Since("1.0.0") def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { if (numFeatures < 0) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 8995591d9e8ce..31ca7c2f207d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -29,7 +29,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} @@ -47,8 +47,8 @@ import org.apache.spark.sql.SQLContext * Results of isotonic regression and therefore monotone. * @param isotonic indicates whether this is isotonic or antitonic. * - * @since 1.3.0 */ +@Since("1.3.0") @Experimental class IsotonicRegressionModel ( val boundaries: Array[Double], @@ -64,8 +64,8 @@ class IsotonicRegressionModel ( /** * A Java-friendly constructor that takes two Iterable parameters and one Boolean parameter. * - * @since 1.4.0 */ + @Since("1.4.0") def this(boundaries: java.lang.Iterable[Double], predictions: java.lang.Iterable[Double], isotonic: java.lang.Boolean) = { @@ -90,8 +90,8 @@ class IsotonicRegressionModel ( * @param testData Features to be labeled. * @return Predicted labels. * - * @since 1.3.0 */ + @Since("1.3.0") def predict(testData: RDD[Double]): RDD[Double] = { testData.map(predict) } @@ -103,8 +103,8 @@ class IsotonicRegressionModel ( * @param testData Features to be labeled. * @return Predicted labels. * - * @since 1.3.0 */ + @Since("1.3.0") def predict(testData: JavaDoubleRDD): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(predict(testData.rdd.retag.asInstanceOf[RDD[Double]])) } @@ -125,8 +125,8 @@ class IsotonicRegressionModel ( * as piecewise linear function and interpolated value is returned. In case there are * multiple values with the same boundary then the same rules as in 2) are used. * - * @since 1.3.0 */ + @Since("1.3.0") def predict(testData: Double): Double = { def linearInterpolation(x1: Double, y1: Double, x2: Double, y2: Double, x: Double): Double = { @@ -160,9 +160,7 @@ class IsotonicRegressionModel ( /** A convenient method for boundaries called by the Python API. */ private[mllib] def predictionVector: Vector = Vectors.dense(predictions) - /** - * @since 1.4.0 - */ + @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) } @@ -170,9 +168,7 @@ class IsotonicRegressionModel ( override protected def formatVersion: String = "1.0" } -/** - * @since 1.4.0 - */ +@Since("1.4.0") object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { import org.apache.spark.mllib.util.Loader._ @@ -219,8 +215,8 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { } /** - * @since 1.4.0 */ + @Since("1.4.0") override def load(sc: SparkContext, path: String): IsotonicRegressionModel = { implicit val formats = DefaultFormats val (loadedClassName, version, metadata) = loadMetadata(sc, path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 8b51011eeb297..f7fe1b7b21fca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import scala.beans.BeanInfo +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Vectors, Vector} import org.apache.spark.mllib.util.NumericParser import org.apache.spark.SparkException @@ -29,8 +30,8 @@ import org.apache.spark.SparkException * @param label Label for this data point. * @param features List of features for this data point. * - * @since 0.8.0 */ +@Since("0.8.0") @BeanInfo case class LabeledPoint(label: Double, features: Vector) { override def toString: String = { @@ -41,15 +42,15 @@ case class LabeledPoint(label: Double, features: Vector) { /** * Parser for [[org.apache.spark.mllib.regression.LabeledPoint]]. * - * @since 1.1.0 */ +@Since("1.1.0") object LabeledPoint { /** * Parses a string resulted from `LabeledPoint#toString` into * an [[org.apache.spark.mllib.regression.LabeledPoint]]. * - * @since 1.1.0 */ + @Since("1.1.0") def parse(s: String): LabeledPoint = { if (s.startsWith("(")) { NumericParser.parse(s) match { 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 03eb589b05a0e..556411a366bd2 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable @@ -31,8 +32,8 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. * - * @since 0.8.0 */ +@Since("0.8.0") class LassoModel ( override val weights: Vector, override val intercept: Double) @@ -46,9 +47,7 @@ class LassoModel ( weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } @@ -56,14 +55,10 @@ class LassoModel ( override protected def formatVersion: String = "1.0" } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object LassoModel extends Loader[LassoModel] { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): LassoModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) // Hard-code class name string in case it changes in the future @@ -118,8 +113,8 @@ class LassoWithSGD private ( /** * Top-level methods for calling Lasso. * - * @since 0.8.0 */ +@Since("0.8.0") object LassoWithSGD { /** @@ -137,8 +132,8 @@ object LassoWithSGD { * @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 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -162,8 +157,8 @@ object LassoWithSGD { * @param regParam Regularization parameter. * @param miniBatchFraction Fraction of data to be used per iteration. * - * @since 0.8.0 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -185,8 +180,8 @@ object LassoWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -205,8 +200,8 @@ object LassoWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int): LassoModel = { 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 fb5c220daaedb..00ab06e3ba738 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable @@ -31,8 +32,8 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. * - * @since 0.8.0 */ +@Since("0.8.0") class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) @@ -46,9 +47,7 @@ class LinearRegressionModel ( weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } @@ -56,14 +55,10 @@ class LinearRegressionModel ( override protected def formatVersion: String = "1.0" } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object LinearRegressionModel extends Loader[LinearRegressionModel] { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): LinearRegressionModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) // Hard-code class name string in case it changes in the future @@ -117,8 +112,8 @@ class LinearRegressionWithSGD private[mllib] ( /** * Top-level methods for calling LinearRegression. * - * @since 0.8.0 */ +@Since("0.8.0") object LinearRegressionWithSGD { /** @@ -135,8 +130,8 @@ object LinearRegressionWithSGD { * @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 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -158,8 +153,8 @@ object LinearRegressionWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -179,8 +174,8 @@ object LinearRegressionWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -198,8 +193,8 @@ object LinearRegressionWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int): LinearRegressionModel = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index b097fd38fdd82..0e72d6591ce83 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -19,14 +19,12 @@ package org.apache.spark.mllib.regression import org.json4s.{DefaultFormats, JValue} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD -/** - * @since 0.8.0 - */ +@Since("0.8.0") @Experimental trait RegressionModel extends Serializable { /** @@ -35,8 +33,8 @@ trait RegressionModel extends Serializable { * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction * - * @since 1.0.0 */ + @Since("1.0.0") def predict(testData: RDD[Vector]): RDD[Double] /** @@ -45,8 +43,8 @@ trait RegressionModel extends Serializable { * @param testData array representing a single data point * @return Double prediction from the trained model * - * @since 1.0.0 */ + @Since("1.0.0") def predict(testData: Vector): Double /** @@ -54,8 +52,8 @@ trait RegressionModel extends Serializable { * @param testData JavaRDD representing data points to be predicted * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction * - * @since 1.0.0 */ + @Since("1.0.0") def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } 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 5bced6b4b7b53..21a791d98b2cb 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.pmml.PMMLExportable @@ -32,8 +33,8 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. * - * @since 0.8.0 */ +@Since("0.8.0") class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) @@ -47,9 +48,7 @@ class RidgeRegressionModel ( weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } @@ -57,14 +56,10 @@ class RidgeRegressionModel ( override protected def formatVersion: String = "1.0" } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object RidgeRegressionModel extends Loader[RidgeRegressionModel] { - /** - * @since 1.3.0 - */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): RidgeRegressionModel = { val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) // Hard-code class name string in case it changes in the future @@ -120,8 +115,8 @@ class RidgeRegressionWithSGD private ( /** * Top-level methods for calling RidgeRegression. * - * @since 0.8.0 */ +@Since("0.8.0") object RidgeRegressionWithSGD { /** @@ -138,8 +133,8 @@ object RidgeRegressionWithSGD { * @param initialWeights Initial set of weights to be used. Array should be equal in size to * the number of features in the data. * - * @since 0.8.0 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -162,8 +157,8 @@ object RidgeRegressionWithSGD { * @param regParam Regularization parameter. * @param miniBatchFraction Fraction of data to be used per iteration. * - * @since 0.8.0 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -184,8 +179,8 @@ object RidgeRegressionWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int, @@ -203,8 +198,8 @@ object RidgeRegressionWithSGD { * @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 */ + @Since("0.8.0") def train( input: RDD[LabeledPoint], numIterations: Int): RidgeRegressionModel = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index a2ab95c474765..cd3ed8a1549db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.regression import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} @@ -54,8 +54,8 @@ import org.apache.spark.streaming.dstream.DStream * the model using each of the different sources, in sequence. * * - * @since 1.1.0 */ +@Since("1.1.0") @DeveloperApi abstract class StreamingLinearAlgorithm[ M <: GeneralizedLinearModel, @@ -70,8 +70,8 @@ abstract class StreamingLinearAlgorithm[ /** * Return the latest model. * - * @since 1.1.0 */ + @Since("1.1.0") def latestModel(): M = { model.get } @@ -84,8 +84,8 @@ abstract class StreamingLinearAlgorithm[ * * @param data DStream containing labeled data * - * @since 1.3.0 */ + @Since("1.3.0") def trainOn(data: DStream[LabeledPoint]): Unit = { if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting training.") @@ -106,8 +106,8 @@ abstract class StreamingLinearAlgorithm[ /** * Java-friendly version of `trainOn`. * - * @since 1.3.0 */ + @Since("1.3.0") def trainOn(data: JavaDStream[LabeledPoint]): Unit = trainOn(data.dstream) /** @@ -116,8 +116,8 @@ abstract class StreamingLinearAlgorithm[ * @param data DStream containing feature vectors * @return DStream containing predictions * - * @since 1.1.0 */ + @Since("1.1.0") def predictOn(data: DStream[Vector]): DStream[Double] = { if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction.") @@ -128,8 +128,8 @@ abstract class StreamingLinearAlgorithm[ /** * Java-friendly version of `predictOn`. * - * @since 1.1.0 */ + @Since("1.1.0") def predictOn(data: JavaDStream[Vector]): JavaDStream[java.lang.Double] = { JavaDStream.fromDStream(predictOn(data.dstream).asInstanceOf[DStream[java.lang.Double]]) } @@ -140,8 +140,8 @@ abstract class StreamingLinearAlgorithm[ * @tparam K key type * @return DStream containing the input keys and the predictions as values * - * @since 1.1.0 */ + @Since("1.1.0") def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Double)] = { if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting prediction") @@ -153,8 +153,8 @@ abstract class StreamingLinearAlgorithm[ /** * Java-friendly version of `predictOnValues`. * - * @since 1.3.0 */ + @Since("1.3.0") def predictOnValues[K](data: JavaPairDStream[K, Vector]): JavaPairDStream[K, java.lang.Double] = { implicit val tag = fakeClassTag[K] JavaPairDStream.fromPairDStream( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala index 93a6753efd4d9..4a856f7f3434a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/KernelDensity.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD @@ -37,8 +37,8 @@ import org.apache.spark.rdd.RDD * .setBandwidth(3.0) * val densities = kd.estimate(Array(-1.0, 2.0, 5.0)) * }}} - * @since 1.4.0 */ +@Since("1.4.0") @Experimental class KernelDensity extends Serializable { @@ -52,8 +52,8 @@ class KernelDensity extends Serializable { /** * Sets the bandwidth (standard deviation) of the Gaussian kernel (default: `1.0`). - * @since 1.4.0 */ + @Since("1.4.0") def setBandwidth(bandwidth: Double): this.type = { require(bandwidth > 0, s"Bandwidth must be positive, but got $bandwidth.") this.bandwidth = bandwidth @@ -62,8 +62,8 @@ class KernelDensity extends Serializable { /** * Sets the sample to use for density estimation. - * @since 1.4.0 */ + @Since("1.4.0") def setSample(sample: RDD[Double]): this.type = { this.sample = sample this @@ -71,8 +71,8 @@ class KernelDensity extends Serializable { /** * Sets the sample to use for density estimation (for Java users). - * @since 1.4.0 */ + @Since("1.4.0") def setSample(sample: JavaRDD[java.lang.Double]): this.type = { this.sample = sample.rdd.asInstanceOf[RDD[Double]] this @@ -80,8 +80,8 @@ class KernelDensity extends Serializable { /** * Estimates probability density function at the given array of points. - * @since 1.4.0 */ + @Since("1.4.0") def estimate(points: Array[Double]): Array[Double] = { val sample = this.sample val bandwidth = this.bandwidth diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 64e4be0ebb97e..51b713e263e0c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.stat -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.{Vectors, Vector} /** @@ -33,8 +33,8 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * Reference: [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]] * Zero elements (including explicit zero values) are skipped when calling add(), * to have time complexity O(nnz) instead of O(n) for each column. - * @since 1.1.0 */ +@Since("1.1.0") @DeveloperApi class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with Serializable { @@ -53,8 +53,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S * * @param sample The sample in dense/sparse vector format to be added into this summarizer. * @return This MultivariateOnlineSummarizer object. - * @since 1.1.0 */ + @Since("1.1.0") def add(sample: Vector): this.type = { if (n == 0) { require(sample.size > 0, s"Vector should have dimension larger than zero.") @@ -109,8 +109,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S * * @param other The other MultivariateOnlineSummarizer to be merged. * @return This MultivariateOnlineSummarizer object. - * @since 1.1.0 */ + @Since("1.1.0") def merge(other: MultivariateOnlineSummarizer): this.type = { if (this.totalCnt != 0 && other.totalCnt != 0) { require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + @@ -155,8 +155,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * Sample mean of each dimension. * - * @since 1.1.0 */ + @Since("1.1.0") override def mean: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -172,8 +172,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * Sample variance of each dimension. * - * @since 1.1.0 */ + @Since("1.1.0") override def variance: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -199,15 +199,15 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * Sample size. * - * @since 1.1.0 */ + @Since("1.1.0") override def count: Long = totalCnt /** * Number of nonzero elements in each dimension. * - * @since 1.1.0 */ + @Since("1.1.0") override def numNonzeros: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -217,8 +217,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * Maximum value of each dimension. * - * @since 1.1.0 */ + @Since("1.1.0") override def max: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -233,8 +233,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * Minimum value of each dimension. * - * @since 1.1.0 */ + @Since("1.1.0") override def min: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -249,8 +249,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * L2 (Euclidian) norm of each dimension. * - * @since 1.2.0 */ + @Since("1.2.0") override def normL2: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") @@ -268,8 +268,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S /** * L1 norm of each dimension. * - * @since 1.2.0 */ + @Since("1.2.0") override def normL1: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala index 3bb49f12289e1..39a16fb743d64 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -17,59 +17,60 @@ package org.apache.spark.mllib.stat +import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.Vector /** * Trait for multivariate statistical summary of a data matrix. - * @since 1.0.0 */ +@Since("1.0.0") trait MultivariateStatisticalSummary { /** * Sample mean vector. - * @since 1.0.0 */ + @Since("1.0.0") def mean: Vector /** * Sample variance vector. Should return a zero vector if the sample size is 1. - * @since 1.0.0 */ + @Since("1.0.0") def variance: Vector /** * Sample size. - * @since 1.0.0 */ + @Since("1.0.0") def count: Long /** * Number of nonzero elements (including explicitly presented zero values) in each column. - * @since 1.0.0 */ + @Since("1.0.0") def numNonzeros: Vector /** * Maximum value of each column. - * @since 1.0.0 */ + @Since("1.0.0") def max: Vector /** * Minimum value of each column. - * @since 1.0.0 */ + @Since("1.0.0") def min: Vector /** * Euclidean magnitude of each column - * @since 1.2.0 */ + @Since("1.2.0") def normL2: Vector /** * L1 norm of each column - * @since 1.2.0 */ + @Since("1.2.0") def normL1: Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index ef8d78607048f..84d64a5bfb38e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat import scala.annotation.varargs -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.{JavaRDD, JavaDoubleRDD} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.{Matrix, Vector} @@ -32,8 +32,8 @@ import org.apache.spark.rdd.RDD /** * :: Experimental :: * API for statistical functions in MLlib. - * @since 1.1.0 */ +@Since("1.1.0") @Experimental object Statistics { @@ -42,8 +42,8 @@ object Statistics { * * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. - * @since 1.1.0 */ + @Since("1.1.0") def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { new RowMatrix(X).computeColumnSummaryStatistics() } @@ -54,8 +54,8 @@ object Statistics { * * @param X an RDD[Vector] for which the correlation matrix is to be computed. * @return Pearson correlation matrix comparing columns in X. - * @since 1.1.0 */ + @Since("1.1.0") def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) /** @@ -71,8 +71,8 @@ object Statistics { * @param method String specifying the method to use for computing correlation. * Supported: `pearson` (default), `spearman` * @return Correlation matrix comparing columns in X. - * @since 1.1.0 */ + @Since("1.1.0") def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) /** @@ -85,14 +85,14 @@ object Statistics { * @param x RDD[Double] of the same cardinality as y. * @param y RDD[Double] of the same cardinality as x. * @return A Double containing the Pearson correlation between the two input RDD[Double]s - * @since 1.1.0 */ + @Since("1.1.0") def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) /** * Java-friendly version of [[corr()]] - * @since 1.4.1 */ + @Since("1.4.1") def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double]): Double = corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]]) @@ -109,14 +109,14 @@ object Statistics { * Supported: `pearson` (default), `spearman` * @return A Double containing the correlation between the two input RDD[Double]s using the * specified method. - * @since 1.1.0 */ + @Since("1.1.0") def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) /** * Java-friendly version of [[corr()]] - * @since 1.4.1 */ + @Since("1.4.1") def corr(x: JavaRDD[java.lang.Double], y: JavaRDD[java.lang.Double], method: String): Double = corr(x.rdd.asInstanceOf[RDD[Double]], y.rdd.asInstanceOf[RDD[Double]], method) @@ -133,8 +133,8 @@ object Statistics { * `expected` is rescaled if the `expected` sum differs from the `observed` sum. * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. - * @since 1.1.0 */ + @Since("1.1.0") def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { ChiSqTest.chiSquared(observed, expected) } @@ -148,8 +148,8 @@ object Statistics { * @param observed Vector containing the observed categorical counts/relative frequencies. * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. - * @since 1.1.0 */ + @Since("1.1.0") def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) /** @@ -159,8 +159,8 @@ object Statistics { * @param observed The contingency matrix (containing either counts or relative frequencies). * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. - * @since 1.1.0 */ + @Since("1.1.0") def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) /** @@ -172,13 +172,14 @@ object Statistics { * Real-valued features will be treated as categorical for each distinct value. * @return an array containing the ChiSquaredTestResult for every feature against the label. * The order of the elements in the returned array reflects the order of input features. - * @since 1.1.0 */ + @Since("1.1.0") def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { ChiSqTest.chiSquaredFeatures(data) } /** Java-friendly version of [[chiSqTest()]] */ + @Since("1.5.0") def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = chiSqTest(data.rdd) /** @@ -194,6 +195,7 @@ object Statistics { * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test * statistic, p-value, and null hypothesis. */ + @Since("1.5.0") def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double) : KolmogorovSmirnovTestResult = { KolmogorovSmirnovTest.testOneSample(data, cdf) @@ -210,6 +212,7 @@ object Statistics { * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test * statistic, p-value, and null hypothesis. */ + @Since("1.5.0") @varargs def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*) : KolmogorovSmirnovTestResult = { @@ -217,6 +220,7 @@ object Statistics { } /** Java-friendly version of [[kolmogorovSmirnovTest()]] */ + @Since("1.5.0") @varargs def kolmogorovSmirnovTest( data: JavaDoubleRDD, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index 9aa7763d7890d..bd4d81390bfae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.stat.distribution import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym, Vector => BV} -import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} import org.apache.spark.mllib.util.MLUtils @@ -32,8 +32,8 @@ import org.apache.spark.mllib.util.MLUtils * * @param mu The mean vector of the distribution * @param sigma The covariance matrix of the distribution - * @since 1.3.0 */ +@Since("1.3.0") @DeveloperApi class MultivariateGaussian ( val mu: Vector, @@ -62,15 +62,15 @@ class MultivariateGaussian ( private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants /** Returns density of this multivariate Gaussian at given point, x - * @since 1.3.0 */ + @Since("1.3.0") def pdf(x: Vector): Double = { pdf(x.toBreeze) } /** Returns the log-density of this multivariate Gaussian at given point, x - * @since 1.3.0 */ + @Since("1.3.0") def logpdf(x: Vector): Double = { logpdf(x.toBreeze) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index e5200b86fddd4..972841015d4f0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuilder import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo @@ -43,8 +43,8 @@ import org.apache.spark.util.random.XORShiftRandom * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { @@ -54,8 +54,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * Method to train a decision tree model over an RDD * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] * @return DecisionTreeModel that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def run(input: RDD[LabeledPoint]): DecisionTreeModel = { // Note: random seed will not be used since numTrees = 1. val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = 0) @@ -64,9 +64,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo } } -/** - * @since 1.0.0 - */ +@Since("1.0.0") object DecisionTree extends Serializable with Logging { /** @@ -84,8 +82,8 @@ object DecisionTree extends Serializable with Logging { * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. * @return DecisionTreeModel that can be used for prediction - * @since 1.0.0 - */ + */ + @Since("1.0.0") def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { new DecisionTree(strategy).run(input) } @@ -106,8 +104,8 @@ object DecisionTree extends Serializable with Logging { * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * @return DecisionTreeModel that can be used for prediction - * @since 1.0.0 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], algo: Algo, @@ -134,8 +132,8 @@ object DecisionTree extends Serializable with Logging { * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * @param numClasses number of classes for classification. Default value of 2. * @return DecisionTreeModel that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def train( input: RDD[LabeledPoint], algo: Algo, @@ -168,8 +166,8 @@ object DecisionTree extends Serializable with Logging { * E.g., an entry (n -> k) indicates that feature n is categorical * with k categories indexed from 0: {0, 1, ..., k-1}. * @return DecisionTreeModel that can be used for prediction - * @since 1.0.0 */ + @Since("1.0.0") def train( input: RDD[LabeledPoint], algo: Algo, @@ -201,8 +199,8 @@ object DecisionTree extends Serializable with Logging { * @param maxBins maximum number of bins used for splitting features * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction - * @since 1.1.0 */ + @Since("1.1.0") def trainClassifier( input: RDD[LabeledPoint], numClasses: Int, @@ -217,8 +215,8 @@ object DecisionTree extends Serializable with Logging { /** * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] - * @since 1.1.0 */ + @Since("1.1.0") def trainClassifier( input: JavaRDD[LabeledPoint], numClasses: Int, @@ -247,8 +245,8 @@ object DecisionTree extends Serializable with Logging { * @param maxBins maximum number of bins used for splitting features * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction - * @since 1.1.0 */ + @Since("1.1.0") def trainRegressor( input: RDD[LabeledPoint], categoricalFeaturesInfo: Map[Int, Int], @@ -261,8 +259,8 @@ object DecisionTree extends Serializable with Logging { /** * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] - * @since 1.1.0 */ + @Since("1.1.0") def trainRegressor( input: JavaRDD[LabeledPoint], categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 143617098637a..e750408600c33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.tree import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer import org.apache.spark.mllib.regression.LabeledPoint @@ -48,8 +48,8 @@ import org.apache.spark.storage.StorageLevel * for other loss functions. * * @param boostingStrategy Parameters for the gradient boosting algorithm. - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) extends Serializable with Logging { @@ -58,8 +58,8 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) * Method to train a gradient boosting model * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * @return a gradient boosted trees model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def run(input: RDD[LabeledPoint]): GradientBoostedTreesModel = { val algo = boostingStrategy.treeStrategy.algo algo match { @@ -76,8 +76,8 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) /** * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees!#run]]. - * @since 1.2.0 */ + @Since("1.2.0") def run(input: JavaRDD[LabeledPoint]): GradientBoostedTreesModel = { run(input.rdd) } @@ -91,8 +91,8 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) * E.g., these two datasets could be created from an original dataset * by using [[org.apache.spark.rdd.RDD.randomSplit()]] * @return a gradient boosted trees model that can be used for prediction - * @since 1.4.0 */ + @Since("1.4.0") def runWithValidation( input: RDD[LabeledPoint], validationInput: RDD[LabeledPoint]): GradientBoostedTreesModel = { @@ -115,8 +115,8 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) /** * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees!#runWithValidation]]. - * @since 1.4.0 */ + @Since("1.4.0") def runWithValidation( input: JavaRDD[LabeledPoint], validationInput: JavaRDD[LabeledPoint]): GradientBoostedTreesModel = { @@ -124,9 +124,7 @@ class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) } } -/** - * @since 1.2.0 - */ +@Since("1.2.0") object GradientBoostedTrees extends Logging { /** @@ -137,8 +135,8 @@ object GradientBoostedTrees extends Logging { * For regression, labels are real numbers. * @param boostingStrategy Configuration options for the boosting algorithm. * @return a gradient boosted trees model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def train( input: RDD[LabeledPoint], boostingStrategy: BoostingStrategy): GradientBoostedTreesModel = { @@ -147,8 +145,8 @@ object GradientBoostedTrees extends Logging { /** * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoostedTrees$#train]] - * @since 1.2.0 */ + @Since("1.2.0") def train( input: JavaRDD[LabeledPoint], boostingStrategy: BoostingStrategy): GradientBoostedTreesModel = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 9f3230656acc5..63a902f3eb51e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.collection.JavaConverters._ import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy @@ -260,9 +260,7 @@ private class RandomForest ( } -/** - * @since 1.2.0 - */ +@Since("1.2.0") object RandomForest extends Serializable with Logging { /** @@ -279,8 +277,8 @@ object RandomForest extends Serializable with Logging { * if numTrees > 1 (forest) set to "sqrt". * @param seed Random seed for bootstrapping and choosing feature subsets. * @return a random forest model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def trainClassifier( input: RDD[LabeledPoint], strategy: Strategy, @@ -317,8 +315,8 @@ object RandomForest extends Serializable with Logging { * (suggested value: 100) * @param seed Random seed for bootstrapping and choosing feature subsets. * @return a random forest model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def trainClassifier( input: RDD[LabeledPoint], numClasses: Int, @@ -337,8 +335,8 @@ object RandomForest extends Serializable with Logging { /** * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]] - * @since 1.2.0 */ + @Since("1.2.0") def trainClassifier( input: JavaRDD[LabeledPoint], numClasses: Int, @@ -368,8 +366,8 @@ object RandomForest extends Serializable with Logging { * if numTrees > 1 (forest) set to "onethird". * @param seed Random seed for bootstrapping and choosing feature subsets. * @return a random forest model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def trainRegressor( input: RDD[LabeledPoint], strategy: Strategy, @@ -405,8 +403,8 @@ object RandomForest extends Serializable with Logging { * (suggested value: 100) * @param seed Random seed for bootstrapping and choosing feature subsets. * @return a random forest model that can be used for prediction - * @since 1.2.0 */ + @Since("1.2.0") def trainRegressor( input: RDD[LabeledPoint], categoricalFeaturesInfo: Map[Int, Int], @@ -424,8 +422,8 @@ object RandomForest extends Serializable with Logging { /** * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]] - * @since 1.2.0 */ + @Since("1.2.0") def trainRegressor( input: JavaRDD[LabeledPoint], categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], @@ -442,8 +440,8 @@ object RandomForest extends Serializable with Logging { /** * List of supported feature subset sampling strategies. - * @since 1.2.0 */ + @Since("1.2.0") val supportedFeatureSubsetStrategies: Array[String] = Array("auto", "all", "sqrt", "log2", "onethird") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index d9a49aa71fcfb..8301ad160836b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.tree.configuration -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} /** * :: Experimental :: * Enum to select the algorithm for the decision tree - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object Algo extends Enumeration { type Algo = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 88e5f57e9ab32..7c569981977b4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.tree.configuration import scala.beans.BeanProperty -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} @@ -38,8 +38,8 @@ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} * validation input between two iterations is less than the validationTol * then stop. Ignored when * [[org.apache.spark.mllib.tree.GradientBoostedTrees.run()]] is used. - * @since 1.2.0 */ +@Since("1.2.0") @Experimental case class BoostingStrategy( // Required boosting parameters @@ -71,9 +71,7 @@ case class BoostingStrategy( } } -/** - * @since 1.2.0 - */ +@Since("1.2.0") @Experimental object BoostingStrategy { @@ -81,8 +79,8 @@ object BoostingStrategy { * Returns default configuration for the boosting algorithm * @param algo Learning goal. Supported: "Classification" or "Regression" * @return Configuration for boosting algorithm - * @since 1.2.0 */ + @Since("1.2.0") def defaultParams(algo: String): BoostingStrategy = { defaultParams(Algo.fromString(algo)) } @@ -93,8 +91,8 @@ object BoostingStrategy { * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] * @return Configuration for boosting algorithm - * @since 1.3.0 */ + @Since("1.3.0") def defaultParams(algo: Algo): BoostingStrategy = { val treeStrategy = Strategy.defaultStrategy(algo) treeStrategy.maxDepth = 3 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index 0684cafa486bd..bb7c7ee4f964f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.tree.configuration -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} /** * :: Experimental :: * Enum to describe whether a feature is "continuous" or "categorical" - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object FeatureType extends Enumeration { type FeatureType = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index 2daa63c4d2771..904e42deebb5f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.tree.configuration -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} /** * :: Experimental :: * Enum for selecting the quantile calculation strategy - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object QuantileStrategy extends Enumeration { type QuantileStrategy = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 7ae25a88bf500..a58f01ba8544e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.configuration import scala.beans.BeanProperty import scala.collection.JavaConverters._ -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.tree.impurity.{Variance, Entropy, Gini, Impurity} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ @@ -66,8 +66,8 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * E.g. 10 means that the cache will get checkpointed every 10 updates. If * the checkpoint directory is not set in * [[org.apache.spark.SparkContext]], this setting is ignored. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class Strategy ( @BeanProperty var algo: Algo, @@ -85,23 +85,23 @@ class Strategy ( @BeanProperty var checkpointInterval: Int = 10) extends Serializable { /** - * @since 1.2.0 */ + @Since("1.2.0") def isMulticlassClassification: Boolean = { algo == Classification && numClasses > 2 } /** - * @since 1.2.0 */ + @Since("1.2.0") def isMulticlassWithCategoricalFeatures: Boolean = { isMulticlassClassification && (categoricalFeaturesInfo.size > 0) } /** * Java-friendly constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] - * @since 1.1.0 */ + @Since("1.1.0") def this( algo: Algo, impurity: Impurity, @@ -115,8 +115,8 @@ class Strategy ( /** * Sets Algorithm using a String. - * @since 1.2.0 */ + @Since("1.2.0") def setAlgo(algo: String): Unit = algo match { case "Classification" => setAlgo(Classification) case "Regression" => setAlgo(Regression) @@ -124,8 +124,8 @@ class Strategy ( /** * Sets categoricalFeaturesInfo using a Java Map. - * @since 1.2.0 */ + @Since("1.2.0") def setCategoricalFeaturesInfo( categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]): Unit = { this.categoricalFeaturesInfo = @@ -174,8 +174,8 @@ class Strategy ( /** * Returns a shallow copy of this instance. - * @since 1.2.0 */ + @Since("1.2.0") def copy: Strategy = { new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, @@ -183,17 +183,15 @@ class Strategy ( } } -/** - * @since 1.2.0 - */ +@Since("1.2.0") @Experimental object Strategy { /** * Construct a default set of parameters for [[org.apache.spark.mllib.tree.DecisionTree]] * @param algo "Classification" or "Regression" - * @since 1.2.0 */ + @Since("1.2.0") def defaultStrategy(algo: String): Strategy = { defaultStrategy(Algo.fromString(algo)) } @@ -201,8 +199,8 @@ object Strategy { /** * Construct a default set of parameters for [[org.apache.spark.mllib.tree.DecisionTree]] * @param algo Algo.Classification or Algo.Regression - * @since 1.3.0 */ + @Since("1.3.0") def defaultStrategy(algo: Algo): Strategy = algo match { case Algo.Classification => new Strategy(algo = Classification, impurity = Gini, maxDepth = 10, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 0b6c7266dee05..73df6b054a8ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} /** * :: Experimental :: * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object Entropy extends Impurity { @@ -36,8 +36,8 @@ object Entropy extends Impurity { * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 - * @since 1.1.0 */ + @Since("1.1.0") @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { if (totalCount == 0) { @@ -64,8 +64,8 @@ object Entropy extends Impurity { * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value, or 0 if count = 0 - * @since 1.0.0 */ + @Since("1.0.0") @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") @@ -73,8 +73,8 @@ object Entropy extends Impurity { /** * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. - * @since 1.1.0 */ + @Since("1.1.0") def instance: this.type = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 3b0be428833ae..f21845b21a802 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,15 +17,15 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} /** * :: Experimental :: * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object Gini extends Impurity { @@ -35,8 +35,8 @@ object Gini extends Impurity { * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 - * @since 1.1.0 */ + @Since("1.1.0") @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { if (totalCount == 0) { @@ -60,8 +60,8 @@ object Gini extends Impurity { * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value, or 0 if count = 0 - * @since 1.0.0 */ + @Since("1.0.0") @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") @@ -69,8 +69,8 @@ object Gini extends Impurity { /** * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. - * @since 1.1.0 */ + @Since("1.1.0") def instance: this.type = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index dd297400058d2..4637dcceea7f8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} /** * :: Experimental :: @@ -25,8 +25,8 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} * This trait is used for * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] * (b) calculating impurity values from sufficient statistics. - * @since 1.0.0 */ +@Since("1.0.0") @Experimental trait Impurity extends Serializable { @@ -36,8 +36,8 @@ trait Impurity extends Serializable { * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 - * @since 1.1.0 */ + @Since("1.1.0") @DeveloperApi def calculate(counts: Array[Double], totalCount: Double): Double @@ -48,8 +48,8 @@ trait Impurity extends Serializable { * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value, or 0 if count = 0 - * @since 1.0.0 */ + @Since("1.0.0") @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index adbe05811f286..a74197278d6f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.tree.impurity -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} /** * :: Experimental :: * Class for calculating variance during regression - * @since 1.0.0 */ +@Since("1.0.0") @Experimental object Variance extends Impurity { @@ -33,8 +33,8 @@ object Variance extends Impurity { * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels * @return information value, or 0 if totalCount = 0 - * @since 1.1.0 */ + @Since("1.1.0") @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = throw new UnsupportedOperationException("Variance.calculate") @@ -46,8 +46,8 @@ object Variance extends Impurity { * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value, or 0 if count = 0 - * @since 1.0.0 */ + @Since("1.0.0") @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { if (count == 0) { @@ -60,8 +60,8 @@ object Variance extends Impurity { /** * Get this impurity instance. * This is useful for passing impurity parameters to a Strategy in Java. - * @since 1.0.0 */ + @Since("1.0.0") def instance: this.type = this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index c6e3d0d824dd7..bab7b8c6cadf2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -29,8 +29,8 @@ import org.apache.spark.mllib.tree.model.TreeEnsembleModel * The absolute (L1) error is defined as: * |y - F(x)| * where y is the label and F(x) is the model prediction for features x. - * @since 1.2.0 */ +@Since("1.2.0") @DeveloperApi object AbsoluteError extends Loss { @@ -41,8 +41,8 @@ object AbsoluteError extends Loss { * @param prediction Predicted label. * @param label True label. * @return Loss gradient - * @since 1.2.0 */ + @Since("1.2.0") override def gradient(prediction: Double, label: Double): Double = { if (label - prediction < 0) 1.0 else -1.0 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index eee58445a1ec1..b2b4594712f0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.mllib.util.MLUtils @@ -31,8 +31,8 @@ import org.apache.spark.mllib.util.MLUtils * The log loss is defined as: * 2 log(1 + exp(-2 y F(x))) * where y is a label in {-1, 1} and F(x) is the model prediction for features x. - * @since 1.2.0 */ +@Since("1.2.0") @DeveloperApi object LogLoss extends Loss { @@ -43,8 +43,8 @@ object LogLoss extends Loss { * @param prediction Predicted label. * @param label True label. * @return Loss gradient - * @since 1.2.0 */ + @Since("1.2.0") override def gradient(prediction: Double, label: Double): Double = { - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 7c9fb924645c8..687cde325ffed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel import org.apache.spark.rdd.RDD @@ -26,8 +26,8 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. - * @since 1.2.0 */ +@Since("1.2.0") @DeveloperApi trait Loss extends Serializable { @@ -36,8 +36,8 @@ trait Loss extends Serializable { * @param prediction Predicted feature * @param label true label. * @return Loss gradient. - * @since 1.2.0 */ + @Since("1.2.0") def gradient(prediction: Double, label: Double): Double /** @@ -47,8 +47,8 @@ trait Loss extends Serializable { * @param model Model of the weak learner. * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * @return Measure of model error on data - * @since 1.2.0 */ + @Since("1.2.0") def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { data.map(point => computeError(model.predict(point.features), point.label)).mean() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala index 47dc94cde7e03..2b112fbe12202 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala @@ -17,14 +17,12 @@ package org.apache.spark.mllib.tree.loss -/** - * @since 1.2.0 - */ +import org.apache.spark.annotation.Since + +@Since("1.2.0") object Losses { - /** - * @since 1.2.0 - */ + @Since("1.2.0") def fromString(name: String): Loss = name match { case "leastSquaresError" => SquaredError case "leastAbsoluteError" => AbsoluteError diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index ff8903d6956bd..3f7d3d38be16c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -29,8 +29,8 @@ import org.apache.spark.mllib.tree.model.TreeEnsembleModel * The squared (L2) error is defined as: * (y - F(x))**2 * where y is the label and F(x) is the model prediction for features x. - * @since 1.2.0 */ +@Since("1.2.0") @DeveloperApi object SquaredError extends Loss { @@ -41,8 +41,8 @@ object SquaredError extends Loss { * @param prediction Predicted label. * @param label True label. * @return Loss gradient - * @since 1.2.0 */ + @Since("1.2.0") override def gradient(prediction: Double, label: Double): Double = { - 2.0 * (label - prediction) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0f386a26601ce..3eefd135f7836 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.tree.configuration.{Algo, FeatureType} @@ -40,8 +40,8 @@ import org.apache.spark.util.Utils * This model stores the decision tree structure and parameters. * @param topNode root node * @param algo algorithm type -- classification or regression - * @since 1.0.0 */ +@Since("1.0.0") @Experimental class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable with Saveable { @@ -50,8 +50,8 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * * @param features array representing a single data point * @return Double prediction from the trained model - * @since 1.0.0 */ + @Since("1.0.0") def predict(features: Vector): Double = { topNode.predict(features) } @@ -61,8 +61,8 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * * @param features RDD representing data points to be predicted * @return RDD of predictions for each of the given data points - * @since 1.0.0 */ + @Since("1.0.0") def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) } @@ -72,16 +72,16 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * * @param features JavaRDD representing data points to be predicted * @return JavaRDD of predictions for each of the given data points - * @since 1.2.0 */ + @Since("1.2.0") def predict(features: JavaRDD[Vector]): JavaRDD[Double] = { predict(features.rdd) } /** * Get number of nodes in tree, including leaf nodes. - * @since 1.1.0 */ + @Since("1.1.0") def numNodes: Int = { 1 + topNode.numDescendants } @@ -89,8 +89,8 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable /** * Get depth of tree. * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. - * @since 1.1.0 */ + @Since("1.1.0") def depth: Int = { topNode.subtreeDepth } @@ -119,8 +119,8 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. * If the directory already exists, this method throws an exception. - * @since 1.3.0 */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) } @@ -128,9 +128,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable override protected def formatVersion: String = DecisionTreeModel.formatVersion } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { private[spark] def formatVersion: String = "1.0" @@ -317,8 +315,8 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. * @return Model instance - * @since 1.3.0 */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): DecisionTreeModel = { implicit val formats = DefaultFormats val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 23f0363639120..091a0462c204f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator /** @@ -29,8 +29,8 @@ import org.apache.spark.mllib.tree.impurity.ImpurityCalculator * @param rightImpurity right node impurity * @param leftPredict left node predict * @param rightPredict right node predict - * @since 1.0.0 */ +@Since("1.0.0") @DeveloperApi class InformationGainStats( val gain: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index aca3350c2e535..8c54c55107233 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vector @@ -38,8 +38,8 @@ import org.apache.spark.mllib.linalg.Vector * @param leftNode left child * @param rightNode right child * @param stats information gain stats - * @since 1.0.0 */ +@Since("1.0.0") @DeveloperApi class Node ( val id: Int, @@ -59,8 +59,8 @@ class Node ( /** * build the left node and right nodes if not leaf * @param nodes array of nodes - * @since 1.0.0 */ + @Since("1.0.0") @deprecated("build should no longer be used since trees are constructed on-the-fly in training", "1.2.0") def build(nodes: Array[Node]): Unit = { @@ -81,8 +81,8 @@ class Node ( * predict value if node is not leaf * @param features feature value * @return predicted value - * @since 1.1.0 */ + @Since("1.1.0") def predict(features: Vector) : Double = { if (isLeaf) { predict.predict diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index be819b59e7048..965784051ede5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} /** * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) - * @since 1.2.0 */ +@Since("1.2.0") @DeveloperApi class Predict( val predict: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 18d40530aee1e..45db83ae3a1f3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType import org.apache.spark.mllib.tree.configuration.FeatureType import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType @@ -30,8 +30,8 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType * Split left if feature <= threshold, else right. * @param featureType type of feature -- categorical or continuous * @param categories Split left if categorical feature value is in this set, else right. - * @since 1.0.0 */ +@Since("1.0.0") @DeveloperApi case class Split( feature: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 0c629b12a84df..19571447a2c56 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint @@ -45,8 +45,8 @@ import org.apache.spark.util.Utils * * @param algo algorithm for the ensemble model, either Classification or Regression * @param trees tree ensembles - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) extends TreeEnsembleModel(algo, trees, Array.fill(trees.length)(1.0), @@ -60,8 +60,8 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. * If the directory already exists, this method throws an exception. - * @since 1.3.0 */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, RandomForestModel.SaveLoadV1_0.thisClassName) @@ -70,9 +70,7 @@ class RandomForestModel(override val algo: Algo, override val trees: Array[Decis override protected def formatVersion: String = RandomForestModel.formatVersion } -/** - * @since 1.3.0 - */ +@Since("1.3.0") object RandomForestModel extends Loader[RandomForestModel] { private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion @@ -82,8 +80,8 @@ object RandomForestModel extends Loader[RandomForestModel] { * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. * @return Model instance - * @since 1.3.0 */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): RandomForestModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName @@ -114,8 +112,8 @@ object RandomForestModel extends Loader[RandomForestModel] { * @param algo algorithm for the ensemble model, either Classification or Regression * @param trees tree ensembles * @param treeWeights tree ensemble weights - * @since 1.2.0 */ +@Since("1.2.0") @Experimental class GradientBoostedTreesModel( override val algo: Algo, @@ -130,8 +128,8 @@ class GradientBoostedTreesModel( * @param sc Spark context used to save model data. * @param path Path specifying the directory in which to save this model. * If the directory already exists, this method throws an exception. - * @since 1.3.0 */ + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, GradientBoostedTreesModel.SaveLoadV1_0.thisClassName) @@ -143,8 +141,8 @@ class GradientBoostedTreesModel( * @param loss evaluation metric. * @return an array with index i having the losses or errors for the ensemble * containing the first i+1 trees - * @since 1.4.0 */ + @Since("1.4.0") def evaluateEachIteration( data: RDD[LabeledPoint], loss: Loss): Array[Double] = { @@ -186,8 +184,8 @@ class GradientBoostedTreesModel( } /** - * @since 1.3.0 */ +@Since("1.3.0") object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { /** @@ -199,8 +197,8 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * @param loss: evaluation metric. * @return a RDD with each element being a zip of the prediction and error * corresponding to every sample. - * @since 1.4.0 */ + @Since("1.4.0") def computeInitialPredictionAndError( data: RDD[LabeledPoint], initTreeWeight: Double, @@ -223,8 +221,8 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * @param loss: evaluation metric. * @return a RDD with each element being a zip of the prediction and error * corresponding to each sample. - * @since 1.4.0 */ + @Since("1.4.0") def updatePredictionError( data: RDD[LabeledPoint], predictionAndError: RDD[(Double, Double)], @@ -248,8 +246,8 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. * @return Model instance - * @since 1.3.0 */ + @Since("1.3.0") override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala index f520b3a1b7c72..bcaacc1b1f191 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala @@ -24,7 +24,6 @@ package org.apache.spark.mllib * - information loss calculation with entropy and Gini for classification and * variance for regression, * - both continuous and categorical features. - * @since 1.0.0 */ package object tree { } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 11ed23176fc12..4940974bf4f41 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD @@ -64,8 +64,8 @@ object MLUtils { * feature dimensions. * @param minPartitions min number of partitions * @return labeled data stored as an RDD[LabeledPoint] - * @since 1.0.0 */ + @Since("1.0.0") def loadLibSVMFile( sc: SparkContext, path: String, @@ -115,9 +115,7 @@ object MLUtils { // Convenient methods for `loadLibSVMFile`. - /** - * @since 1.0.0 - */ + @Since("1.0.0") @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, @@ -130,17 +128,15 @@ object MLUtils { /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of * partitions. - * @since 1.0.0 */ + @Since("1.0.0") def loadLibSVMFile( sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = loadLibSVMFile(sc, path, numFeatures, sc.defaultMinPartitions) - /** - * @since 1.0.0 - */ + @Since("1.0.0") @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, @@ -149,9 +145,7 @@ object MLUtils { numFeatures: Int): RDD[LabeledPoint] = loadLibSVMFile(sc, path, numFeatures) - /** - * @since 1.0.0 - */ + @Since("1.0.0") @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, @@ -162,8 +156,8 @@ object MLUtils { /** * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of * features determined automatically and the default number of partitions. - * @since 1.0.0 */ + @Since("1.0.0") def loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] = loadLibSVMFile(sc, path, -1) @@ -193,15 +187,15 @@ object MLUtils { * @param path file or directory path in any Hadoop-supported file system URI * @param minPartitions min number of partitions * @return vectors stored as an RDD[Vector] - * @since 1.1.0 */ + @Since("1.1.0") def loadVectors(sc: SparkContext, path: String, minPartitions: Int): RDD[Vector] = sc.textFile(path, minPartitions).map(Vectors.parse) /** * Loads vectors saved using `RDD[Vector].saveAsTextFile` with the default number of partitions. - * @since 1.1.0 */ + @Since("1.1.0") def loadVectors(sc: SparkContext, path: String): RDD[Vector] = sc.textFile(path, sc.defaultMinPartitions).map(Vectors.parse) @@ -211,16 +205,16 @@ object MLUtils { * @param path file or directory path in any Hadoop-supported file system URI * @param minPartitions min number of partitions * @return labeled points stored as an RDD[LabeledPoint] - * @since 1.1.0 */ + @Since("1.1.0") def loadLabeledPoints(sc: SparkContext, path: String, minPartitions: Int): RDD[LabeledPoint] = sc.textFile(path, minPartitions).map(LabeledPoint.parse) /** * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile` with the default number of * partitions. - * @since 1.1.0 */ + @Since("1.1.0") def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = loadLabeledPoints(sc, dir, sc.defaultMinPartitions) @@ -236,8 +230,8 @@ object MLUtils { * * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. - * @since 1.0.0 */ + @Since("1.0.0") @deprecated("Should use MLUtils.loadLabeledPoints instead.", "1.0.1") def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => @@ -258,8 +252,8 @@ object MLUtils { * * @deprecated Should use [[org.apache.spark.rdd.RDD#saveAsTextFile]] for saving and * [[org.apache.spark.mllib.util.MLUtils#loadLabeledPoints]] for loading. - * @since 1.0.0 */ + @Since("1.0.0") @deprecated("Should use RDD[LabeledPoint].saveAsTextFile instead.", "1.0.1") def saveLabeledData(data: RDD[LabeledPoint], dir: String) { val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) @@ -271,8 +265,8 @@ object MLUtils { * Return a k element array of pairs of RDDs with the first element of each pair * containing the training data, a complement of the validation data and the second * element, the validation data, containing a unique 1/kth of the data. Where k=numFolds. - * @since 1.0.0 */ + @Since("1.0.0") @Experimental def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat @@ -287,8 +281,8 @@ object MLUtils { /** * Returns a new vector with `1.0` (bias) appended to the input vector. - * @since 1.0.0 */ + @Since("1.0.0") def appendBias(vector: Vector): Vector = { vector match { case dv: DenseVector => From e3355090d4030daffed5efb0959bf1d724c13c13 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 21 Aug 2015 14:30:00 -0700 Subject: [PATCH 045/232] [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary. https://issues.apache.org/jira/browse/SPARK-10143 With this PR, we will set min split size to parquet's block size (row group size) set in the conf if the min split size is smaller. So, we can avoid have too many tasks and even useless tasks for reading parquet data. I tested it locally. The table I have has 343MB and it is in my local FS. Because I did not set any min/max split size, the default split size was 32MB and the map stage had 11 tasks. But there were only three tasks that actually read data. With my PR, there were only three tasks in the map stage. Here is the difference. Without this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399179/8587dba6-4765-11e5-9189-7ebba52a2b6d.png) With this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399185/a4735d74-4765-11e5-8848-1f1e361a6b4b.png) Even if the block size setting does match the actual block size of parquet file, I think it is still generally good to use parquet's block size setting if min split size is smaller than this block size. Tested it on a cluster using ``` val count = sqlContext.table("""store_sales""").groupBy().count().queryExecution.executedPlan(3).execute().count ``` Basically, it reads 0 column of table `store_sales`. My table has 1824 parquet files with size from 80MB to 280MB (1 to 3 row group sizes). Without this patch, in a 16 worker cluster, the job had 5023 tasks and spent 102s. With this patch, the job had 2893 tasks and spent 64s. It is still not as good as using one mapper per file (1824 tasks and 42s), but it is much better than our master. Author: Yin Huai Closes #8346 from yhuai/parquetMinSplit. --- .../datasources/parquet/ParquetRelation.scala | 41 ++++++++++++++++++- 1 file changed, 39 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 68169d48ac57c..bbf682aec0f9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import scala.util.{Failure, Try} import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -281,12 +282,18 @@ private[sql] class ParquetRelation( val assumeInt96IsTimestamp = sqlContext.conf.isParquetINT96AsTimestamp val followParquetFormatSpec = sqlContext.conf.followParquetFormatSpec + // Parquet row group size. We will use this value as the value for + // mapreduce.input.fileinputformat.split.minsize and mapred.min.split.size if the value + // of these flags are smaller than the parquet row group size. + val parquetBlockSize = ParquetOutputFormat.getLongBlockSize(broadcastedConf.value.value) + // Create the function to set variable Parquet confs at both driver and executor side. val initLocalJobFuncOpt = ParquetRelation.initializeLocalJobFunc( requiredColumns, filters, dataSchema, + parquetBlockSize, useMetadataCache, parquetFilterPushDown, assumeBinaryIsString, @@ -294,7 +301,8 @@ private[sql] class ParquetRelation( followParquetFormatSpec) _ // Create the function to set input paths at the driver side. - val setInputPaths = ParquetRelation.initializeDriverSideJobFunc(inputFiles) _ + val setInputPaths = + ParquetRelation.initializeDriverSideJobFunc(inputFiles, parquetBlockSize) _ Utils.withDummyCallSite(sqlContext.sparkContext) { new SqlNewHadoopRDD( @@ -482,11 +490,35 @@ private[sql] object ParquetRelation extends Logging { // internally. private[sql] val METASTORE_SCHEMA = "metastoreSchema" + /** + * If parquet's block size (row group size) setting is larger than the min split size, + * we use parquet's block size setting as the min split size. Otherwise, we will create + * tasks processing nothing (because a split does not cover the starting point of a + * parquet block). See https://issues.apache.org/jira/browse/SPARK-10143 for more information. + */ + private def overrideMinSplitSize(parquetBlockSize: Long, conf: Configuration): Unit = { + val minSplitSize = + math.max( + conf.getLong("mapred.min.split.size", 0L), + conf.getLong("mapreduce.input.fileinputformat.split.minsize", 0L)) + if (parquetBlockSize > minSplitSize) { + val message = + s"Parquet's block size (row group size) is larger than " + + s"mapred.min.split.size/mapreduce.input.fileinputformat.split.minsize. Setting " + + s"mapred.min.split.size and mapreduce.input.fileinputformat.split.minsize to " + + s"$parquetBlockSize." + logDebug(message) + conf.set("mapred.min.split.size", parquetBlockSize.toString) + conf.set("mapreduce.input.fileinputformat.split.minsize", parquetBlockSize.toString) + } + } + /** This closure sets various Parquet configurations at both driver side and executor side. */ private[parquet] def initializeLocalJobFunc( requiredColumns: Array[String], filters: Array[Filter], dataSchema: StructType, + parquetBlockSize: Long, useMetadataCache: Boolean, parquetFilterPushDown: Boolean, assumeBinaryIsString: Boolean, @@ -522,16 +554,21 @@ private[sql] object ParquetRelation extends Logging { conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, assumeBinaryIsString) conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, assumeInt96IsTimestamp) conf.setBoolean(SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key, followParquetFormatSpec) + + overrideMinSplitSize(parquetBlockSize, conf) } /** This closure sets input paths at the driver side. */ private[parquet] def initializeDriverSideJobFunc( - inputFiles: Array[FileStatus])(job: Job): Unit = { + inputFiles: Array[FileStatus], + parquetBlockSize: Long)(job: Job): Unit = { // We side the input paths at the driver side. logInfo(s"Reading Parquet file(s) from ${inputFiles.map(_.getPath).mkString(", ")}") if (inputFiles.nonEmpty) { FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) } + + overrideMinSplitSize(parquetBlockSize, job.getConfiguration) } private[parquet] def readSchema( From f01c4220d2b791f470fa6596ffe11baa51517fbe Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 21 Aug 2015 16:28:00 -0700 Subject: [PATCH 046/232] [SPARK-10163] [ML] Allow single-category features for GBT models Removed categorical feature info validation since no longer needed This is needed to make the ML user guide examples work (in another current PR). CC: mengxr Author: Joseph K. Bradley Closes #8367 from jkbradley/gbt-single-cat. --- .../org/apache/spark/mllib/tree/configuration/Strategy.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index a58f01ba8544e..b74e3f1f46523 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -158,11 +158,6 @@ class Strategy ( s" Valid values are integers >= 0.") require(maxBins >= 2, s"DecisionTree Strategy given invalid maxBins parameter: $maxBins." + s" Valid values are integers >= 2.") - categoricalFeaturesInfo.foreach { case (feature, arity) => - require(arity >= 2, - s"DecisionTree Strategy given invalid categoricalFeaturesInfo setting:" + - s" feature $feature has $arity categories. The number of categories should be >= 2.") - } require(minInstancesPerNode >= 1, s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") require(maxMemoryInMB <= 10240, From 630a994e6a9785d1704f8e7fb604f32f5dea24f8 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 21 Aug 2015 16:30:12 -0700 Subject: [PATCH 047/232] [SPARK-9893] User guide with Java test suite for VectorSlicer Add user guide for `VectorSlicer`, with Java test suite and Python version VectorSlicer. Note that Python version does not support selecting by names now. Author: Xusen Yin Closes #8267 from yinxusen/SPARK-9893. --- docs/ml-features.md | 133 ++++++++++++++++++ .../ml/feature/JavaVectorSlicerSuite.java | 85 +++++++++++ 2 files changed, 218 insertions(+) create mode 100644 mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java diff --git a/docs/ml-features.md b/docs/ml-features.md index 6309db97be4d0..642a4b4c53183 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1477,6 +1477,139 @@ print(output.select("features", "clicked").first()) +# Feature Selectors + +## VectorSlicer + +`VectorSlicer` is a transformer that takes a feature vector and outputs a new feature vector with a +sub-array of the original features. It is useful for extracting features from a vector column. + +`VectorSlicer` accepts a vector column with a specified indices, then outputs a new vector column +whose values are selected via those indices. There are two types of indices, + + 1. Integer indices that represents the indices into the vector, `setIndices()`; + + 2. String indices that represents the names of features into the vector, `setNames()`. + *This requires the vector column to have an `AttributeGroup` since the implementation matches on + the name field of an `Attribute`.* + +Specification by integer and string are both acceptable. Moreover, you can use integer index and +string name simultaneously. At least one feature must be selected. Duplicate features are not +allowed, so there can be no overlap between selected indices and names. Note that if names of +features are selected, an exception will be threw out when encountering with empty input attributes. + +The output vector will order features with the selected indices first (in the order given), +followed by the selected names (in the order given). + +**Examples** + +Suppose that we have a DataFrame with the column `userFeatures`: + +~~~ + userFeatures +------------------ + [0.0, 10.0, 0.5] +~~~ + +`userFeatures` is a vector column that contains three user features. Assuming that the first column +of `userFeatures` are all zeros, so we want to remove it and only the last two columns are selected. +The `VectorSlicer` selects the last two elements with `setIndices(1, 2)` then produces a new vector +column named `features`: + +~~~ + userFeatures | features +------------------|----------------------------- + [0.0, 10.0, 0.5] | [10.0, 0.5] +~~~ + +Suppose also that we have a potential input attributes for the `userFeatures`, i.e. +`["f1", "f2", "f3"]`, then we can use `setNames("f2", "f3")` to select them. + +~~~ + userFeatures | features +------------------|----------------------------- + [0.0, 10.0, 0.5] | [10.0, 0.5] + ["f1", "f2", "f3"] | ["f2", "f3"] +~~~ + +
+
+ +[`VectorSlicer`](api/scala/index.html#org.apache.spark.ml.feature.VectorSlicer) takes an input +column name with specified indices or names and an output column name. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup, NumericAttribute} +import org.apache.spark.ml.feature.VectorSlicer +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + +val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(-2.0, 2.3, 0.0) +) + +val defaultAttr = NumericAttribute.defaultAttr +val attrs = Array("f1", "f2", "f3").map(defaultAttr.withName) +val attrGroup = new AttributeGroup("userFeatures", attrs.asInstanceOf[Array[Attribute]]) + +val dataRDD = sc.parallelize(data).map(Row.apply) +val dataset = sqlContext.createDataFrame(dataRDD, StructType(attrGroup.toStructField())) + +val slicer = new VectorSlicer().setInputCol("userFeatures").setOutputCol("features") + +slicer.setIndices(1).setNames("f3") +// or slicer.setIndices(Array(1, 2)), or slicer.setNames(Array("f2", "f3")) + +val output = slicer.transform(dataset) +println(output.select("userFeatures", "features").first()) +{% endhighlight %} +
+ +
+ +[`VectorSlicer`](api/java/org/apache/spark/ml/feature/VectorSlicer.html) takes an input column name +with specified indices or names and an output column name. + +{% highlight java %} +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; +import static org.apache.spark.sql.types.DataTypes.*; + +Attribute[] attrs = new Attribute[]{ + NumericAttribute.defaultAttr().withName("f1"), + NumericAttribute.defaultAttr().withName("f2"), + NumericAttribute.defaultAttr().withName("f3") +}; +AttributeGroup group = new AttributeGroup("userFeatures", attrs); + +JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), + RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) +)); + +DataFrame dataset = jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); + +VectorSlicer vectorSlicer = new VectorSlicer() + .setInputCol("userFeatures").setOutputCol("features"); + +vectorSlicer.setIndices(new int[]{1}).setNames(new String[]{"f3"}); +// or slicer.setIndices(new int[]{1, 2}), or slicer.setNames(new String[]{"f2", "f3"}) + +DataFrame output = vectorSlicer.transform(dataset); + +System.out.println(output.select("userFeatures", "features").first()); +{% endhighlight %} +
+
+ ## RFormula `RFormula` selects columns specified by an [R model formula](https://stat.ethz.ch/R-manual/R-devel/library/stats/html/formula.html). It produces a vector column of features and a double column of labels. Like when formulas are used in R for linear regression, string input columns will be one-hot encoded, and numeric columns will be cast to doubles. If not already present in the DataFrame, the output label column will be created from the specified response variable in the formula. diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java new file mode 100644 index 0000000000000..56988b9fb29cb --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -0,0 +1,85 @@ +/* + * 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.ml.feature; + +import com.google.common.collect.Lists; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.attribute.Attribute; +import org.apache.spark.ml.attribute.AttributeGroup; +import org.apache.spark.ml.attribute.NumericAttribute; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.types.StructType; + + +public class JavaVectorSlicerSuite { + private transient JavaSparkContext jsc; + private transient SQLContext jsql; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaVectorSlicerSuite"); + jsql = new SQLContext(jsc); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void vectorSlice() { + Attribute[] attrs = new Attribute[]{ + NumericAttribute.defaultAttr().withName("f1"), + NumericAttribute.defaultAttr().withName("f2"), + NumericAttribute.defaultAttr().withName("f3") + }; + AttributeGroup group = new AttributeGroup("userFeatures", attrs); + + JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), + RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) + )); + + DataFrame dataset = jsql.createDataFrame(jrdd, (new StructType()).add(group.toStructField())); + + VectorSlicer vectorSlicer = new VectorSlicer() + .setInputCol("userFeatures").setOutputCol("features"); + + vectorSlicer.setIndices(new int[]{1}).setNames(new String[]{"f3"}); + + DataFrame output = vectorSlicer.transform(dataset); + + for (Row r : output.select("userFeatures", "features").take(2)) { + Vector features = r.getAs(1); + Assert.assertEquals(features.size(), 2); + } + } +} From 46fcb9e0dbb2b28110f68a3d9f6c0c47bfd197b1 Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Sat, 22 Aug 2015 02:38:10 -0700 Subject: [PATCH 048/232] Update programming-guide.md Update `lineLengths.persist();` to `lineLengths.persist(StorageLevel.MEMORY_ONLY());` because `JavaRDD#persist` needs a parameter of `StorageLevel`. Author: Keiji Yoshida Closes #8372 from yosssi/patch-1. --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 982c5eabe652b..4cf83bb392636 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -549,7 +549,7 @@ returning only its answer to the driver program. If we also wanted to use `lineLengths` again later, we could add: {% highlight java %} -lineLengths.persist(); +lineLengths.persist(StorageLevel.MEMORY_ONLY()); {% endhighlight %} before the `reduce`, which would cause `lineLengths` to be saved in memory after the first time it is computed. From 90cb9f05655a25b95b8f9fe81da14e5b9c8bcf44 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Sat, 22 Aug 2015 10:16:35 -0700 Subject: [PATCH 049/232] [SPARK-9401] [SQL] Fully implement code generation for ConcatWs This PR adds full codegen support for ConcatWs, is a substitute of #7782 JIRA: https://issues.apache.org/jira/browse/SPARK-9401 cc davies Author: Yijie Shen Closes #8353 from yjshen/concatws. --- .../expressions/stringExpressions.scala | 42 +++++++++++++++++-- 1 file changed, 39 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index b60d318534a41..48d02bb534501 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -72,7 +72,7 @@ case class Concat(children: Seq[Expression]) extends Expression with ImplicitCas * Returns null if the separator is null. Otherwise, concat_ws skips all null values. */ case class ConcatWs(children: Seq[Expression]) - extends Expression with ImplicitCastInputTypes with CodegenFallback { + extends Expression with ImplicitCastInputTypes { require(children.nonEmpty, s"$prettyName requires at least one argument.") @@ -114,8 +114,44 @@ case class ConcatWs(children: Seq[Expression]) boolean ${ev.isNull} = ${ev.primitive} == null; """ } else { - // Contains a mix of strings and arrays. Fall back to interpreted mode for now. - super.genCode(ctx, ev) + val array = ctx.freshName("array") + val varargNum = ctx.freshName("varargNum") + val idxInVararg = ctx.freshName("idxInVararg") + + val evals = children.map(_.gen(ctx)) + val (varargCount, varargBuild) = children.tail.zip(evals.tail).map { case (child, eval) => + child.dataType match { + case StringType => + ("", // we count all the StringType arguments num at once below. + s"$array[$idxInVararg ++] = ${eval.isNull} ? (UTF8String) null : ${eval.primitive};") + case _: ArrayType => + val size = ctx.freshName("n") + (s""" + if (!${eval.isNull}) { + $varargNum += ${eval.primitive}.numElements(); + } + """, + s""" + if (!${eval.isNull}) { + final int $size = ${eval.primitive}.numElements(); + for (int j = 0; j < $size; j ++) { + $array[$idxInVararg ++] = ${ctx.getValue(eval.primitive, StringType, "j")}; + } + } + """) + } + }.unzip + + evals.map(_.code).mkString("\n") + + s""" + int $varargNum = ${children.count(_.dataType == StringType) - 1}; + int $idxInVararg = 0; + ${varargCount.mkString("\n")} + UTF8String[] $array = new UTF8String[$varargNum]; + ${varargBuild.mkString("\n")} + UTF8String ${ev.primitive} = UTF8String.concatWs(${evals.head.primitive}, $array); + boolean ${ev.isNull} = ${ev.primitive} == null; + """ } } } From 623c675fde7a3a39957a62c7af26a54f4b01f8ce Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Sun, 23 Aug 2015 11:04:29 +0100 Subject: [PATCH 050/232] Update streaming-programming-guide.md Update `See the Scala example` to `See the Java example`. Author: Keiji Yoshida Closes #8376 from yosssi/patch-1. --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index c59d936b43c88..118ced298f4b0 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1702,7 +1702,7 @@ context.awaitTermination(); If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data. If the directory does not exist (i.e., running for the first time), then the function `contextFactory` will be called to create a new -context and set up the DStreams. See the Scala example +context and set up the DStreams. See the Java example [JavaRecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java). This example appends the word counts of network data into a file. From c6df5f66d9a8b9760f2cd46fcd930f977650c9c5 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 23 Aug 2015 17:41:49 -0700 Subject: [PATCH 051/232] [SPARK-10148] [STREAMING] Display active and inactive receiver numbers in Streaming page Added the active and inactive receiver numbers in the summary section of Streaming page. screen shot 2015-08-21 at 2 08 54 pm Author: zsxwing Closes #8351 from zsxwing/receiver-number. --- .../spark/streaming/ui/StreamingJobProgressListener.scala | 8 ++++++++ .../org/apache/spark/streaming/ui/StreamingPage.scala | 6 ++++++ 2 files changed, 14 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index b77c555c68b8b..78aeb004e18b1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -148,6 +148,14 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) receiverInfos.size } + def numActiveReceivers: Int = synchronized { + receiverInfos.count(_._2.active) + } + + def numInactiveReceivers: Int = { + ssc.graph.getReceiverInputStreams().size - numActiveReceivers + } + def numTotalCompletedBatches: Long = synchronized { totalCompletedBatches } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 87af902428ec8..96d943e75d272 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -303,6 +303,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val numCompletedBatches = listener.retainedCompletedBatches.size val numActiveBatches = batchTimes.length - numCompletedBatches + val numReceivers = listener.numInactiveReceivers + listener.numActiveReceivers val table = // scalastyle:off @@ -330,6 +331,11 @@ private[ui] class StreamingPage(parent: StreamingTab) } } + { + if (numReceivers > 0) { +
Receivers: {listener.numActiveReceivers} / {numReceivers} active
+ } + }
Avg: {eventRateForAllStreams.formattedAvg} events/sec
From b963c19a803c5a26c9b65655d40ca6621acf8bd4 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 23 Aug 2015 18:34:07 -0700 Subject: [PATCH 052/232] [SPARK-10164] [MLLIB] Fixed GMM distributed decomposition bug GaussianMixture now distributes matrix decompositions for certain problem sizes. Distributed computation actually fails, but this was not tested in unit tests. This PR adds a unit test which checks this. It failed previously but works with this fix. CC: mengxr Author: Joseph K. Bradley Closes #8370 from jkbradley/gmm-fix. --- .../mllib/clustering/GaussianMixture.scala | 22 +++++++++++++------ .../clustering/GaussianMixtureSuite.scala | 22 +++++++++++++++++-- 2 files changed, 35 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index fcc9dfecac54f..daa947e81d44d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -169,9 +169,7 @@ class GaussianMixture private ( // Get length of the input vectors val d = breezeData.first().length - // Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when - // d > 25 except for when k is very small - val distributeGaussians = ((k - 1.0) / k) * d > 25 + val shouldDistributeGaussians = GaussianMixture.shouldDistributeGaussians(k, d) // Determine initial weights and corresponding Gaussians. // If the user supplied an initial GMM, we use those values, otherwise @@ -205,15 +203,15 @@ class GaussianMixture private ( // (often referred to as the "M" step in literature) val sumWeights = sums.weights.sum - if (distributeGaussians) { + if (shouldDistributeGaussians) { val numPartitions = math.min(k, 1024) val tuples = Seq.tabulate(k)(i => (sums.means(i), sums.sigmas(i), sums.weights(i))) val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, sigma, weight) => updateWeightsAndGaussians(mean, sigma, weight, sumWeights) - }.collect.unzip - Array.copy(ws, 0, weights, 0, ws.length) - Array.copy(gs, 0, gaussians, 0, gs.length) + }.collect().unzip + Array.copy(ws.toArray, 0, weights, 0, ws.length) + Array.copy(gs.toArray, 0, gaussians, 0, gs.length) } else { var i = 0 while (i < k) { @@ -271,6 +269,16 @@ class GaussianMixture private ( } } +private[clustering] object GaussianMixture { + /** + * Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when + * d > 25 except for when k is very small. + * @param k Number of topics + * @param d Number of features + */ + def shouldDistributeGaussians(k: Int, d: Int): Boolean = ((k - 1.0) / k) * d > 25 +} + // companion class to provide zero constructor for ExpectationSum private object ExpectationSum { def zero(k: Int, d: Int): ExpectationSum = { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala index b636d02f786e6..a72723eb00daf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GaussianMixtureSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.clustering import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vectors, Matrices} +import org.apache.spark.mllib.linalg.{Vector, Vectors, Matrices} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -76,6 +76,20 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { assert(gmm.gaussians(1).sigma ~== Esigma(1) absTol 1E-3) } + test("two clusters with distributed decompositions") { + val data = sc.parallelize(GaussianTestData.data2, 2) + + val k = 5 + val d = data.first().size + assert(GaussianMixture.shouldDistributeGaussians(k, d)) + + val gmm = new GaussianMixture() + .setK(k) + .run(data) + + assert(gmm.k === k) + } + test("single cluster with sparse data") { val data = sc.parallelize(Array( Vectors.sparse(3, Array(0, 2), Array(4.0, 2.0)), @@ -116,7 +130,7 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { val sparseGMM = new GaussianMixture() .setK(2) .setInitialModel(initialGmm) - .run(data) + .run(sparseData) assert(sparseGMM.weights(0) ~== Ew(0) absTol 1E-3) assert(sparseGMM.weights(1) ~== Ew(1) absTol 1E-3) @@ -168,5 +182,9 @@ class GaussianMixtureSuite extends SparkFunSuite with MLlibTestSparkContext { Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734) ) + val data2: Array[Vector] = Array.tabulate(25){ i: Int => + Vectors.dense(Array.tabulate(50)(i + _.toDouble)) + } + } } From 053d94fcf32268369b5a40837271f15d6af41aa4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sun, 23 Aug 2015 19:24:32 -0700 Subject: [PATCH 053/232] [SPARK-10142] [STREAMING] Made python checkpoint recovery handle non-local checkpoint paths and existing SparkContexts The current code only checks checkpoint files in local filesystem, and always tries to create a new Python SparkContext (even if one already exists). The solution is to do the following: 1. Use the same code path as Java to check whether a valid checkpoint exists 2. Create a new Python SparkContext only if there no active one. There is not test for the path as its hard to test with distributed filesystem paths in a local unit test. I am going to test it with a distributed file system manually to verify that this patch works. Author: Tathagata Das Closes #8366 from tdas/SPARK-10142 and squashes the following commits: 3afa666 [Tathagata Das] Added tests 2dd4ae5 [Tathagata Das] Added the check to not create a context if one already exists 9bf151b [Tathagata Das] Made python checkpoint recovery use java to find the checkpoint files --- python/pyspark/streaming/context.py | 22 ++++++---- python/pyspark/streaming/tests.py | 43 ++++++++++++++++--- .../apache/spark/streaming/Checkpoint.scala | 9 ++++ 3 files changed, 58 insertions(+), 16 deletions(-) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index e3ba70e4e5e88..4069d7a149986 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -150,26 +150,30 @@ def getOrCreate(cls, checkpointPath, setupFunc): @param checkpointPath: Checkpoint directory used in an earlier streaming program @param setupFunc: Function to create a new context and setup DStreams """ - # TODO: support checkpoint in HDFS - if not os.path.exists(checkpointPath) or not os.listdir(checkpointPath): + cls._ensure_initialized() + gw = SparkContext._gateway + + # Check whether valid checkpoint information exists in the given path + if gw.jvm.CheckpointReader.read(checkpointPath).isEmpty(): ssc = setupFunc() ssc.checkpoint(checkpointPath) return ssc - cls._ensure_initialized() - gw = SparkContext._gateway - try: jssc = gw.jvm.JavaStreamingContext(checkpointPath) except Exception: print("failed to load StreamingContext from checkpoint", file=sys.stderr) raise - jsc = jssc.sparkContext() - conf = SparkConf(_jconf=jsc.getConf()) - sc = SparkContext(conf=conf, gateway=gw, jsc=jsc) + # If there is already an active instance of Python SparkContext use it, or create a new one + if not SparkContext._active_spark_context: + jsc = jssc.sparkContext() + conf = SparkConf(_jconf=jsc.getConf()) + SparkContext(conf=conf, gateway=gw, jsc=jsc) + + sc = SparkContext._active_spark_context + # update ctx in serializer - SparkContext._active_spark_context = sc cls._transformerSerializer.ctx = sc return StreamingContext(sc, None, jssc) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 214d5be439003..510a4f2b3e472 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -603,6 +603,10 @@ def tearDownClass(): def tearDown(self): if self.ssc is not None: self.ssc.stop(True) + if self.sc is not None: + self.sc.stop() + if self.cpd is not None: + shutil.rmtree(self.cpd) def test_get_or_create_and_get_active_or_create(self): inputd = tempfile.mkdtemp() @@ -622,8 +626,12 @@ def setup(): self.setupCalled = True return ssc - cpd = tempfile.mkdtemp("test_streaming_cps") - self.ssc = StreamingContext.getOrCreate(cpd, setup) + # Verify that getOrCreate() calls setup() in absence of checkpoint files + self.cpd = tempfile.mkdtemp("test_streaming_cps") + self.setupCalled = False + self.ssc = StreamingContext.getOrCreate(self.cpd, setup) + self.assertFalse(self.setupCalled) + self.ssc.start() def check_output(n): @@ -660,31 +668,52 @@ def check_output(n): self.ssc.stop(True, True) time.sleep(1) self.setupCalled = False - self.ssc = StreamingContext.getOrCreate(cpd, setup) + self.ssc = StreamingContext.getOrCreate(self.cpd, setup) self.assertFalse(self.setupCalled) self.ssc.start() check_output(3) + # Verify that getOrCreate() uses existing SparkContext + self.ssc.stop(True, True) + time.sleep(1) + sc = SparkContext(SparkConf()) + self.setupCalled = False + self.ssc = StreamingContext.getOrCreate(self.cpd, setup) + self.assertFalse(self.setupCalled) + self.assertTrue(self.ssc.sparkContext == sc) + # Verify the getActiveOrCreate() recovers from checkpoint files self.ssc.stop(True, True) time.sleep(1) self.setupCalled = False - self.ssc = StreamingContext.getActiveOrCreate(cpd, setup) + self.ssc = StreamingContext.getActiveOrCreate(self.cpd, setup) self.assertFalse(self.setupCalled) self.ssc.start() check_output(4) # Verify that getActiveOrCreate() returns active context self.setupCalled = False - self.assertEquals(StreamingContext.getActiveOrCreate(cpd, setup), self.ssc) + self.assertEquals(StreamingContext.getActiveOrCreate(self.cpd, setup), self.ssc) self.assertFalse(self.setupCalled) + # Verify that getActiveOrCreate() uses existing SparkContext + self.ssc.stop(True, True) + time.sleep(1) + self.sc = SparkContext(SparkConf()) + self.setupCalled = False + self.ssc = StreamingContext.getActiveOrCreate(self.cpd, setup) + self.assertFalse(self.setupCalled) + self.assertTrue(self.ssc.sparkContext == sc) + # Verify that getActiveOrCreate() calls setup() in absence of checkpoint files self.ssc.stop(True, True) - shutil.rmtree(cpd) # delete checkpoint directory + shutil.rmtree(self.cpd) # delete checkpoint directory + time.sleep(1) self.setupCalled = False - self.ssc = StreamingContext.getActiveOrCreate(cpd, setup) + self.ssc = StreamingContext.getActiveOrCreate(self.cpd, setup) self.assertTrue(self.setupCalled) + + # Stop everything self.ssc.stop(True, True) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 6f6b449accc3c..cd5d960369c05 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -286,6 +286,15 @@ class CheckpointWriter( private[streaming] object CheckpointReader extends Logging { + /** + * Read checkpoint files present in the given checkpoint directory. If there are no checkpoint + * files, then return None, else try to return the latest valid checkpoint object. If no + * checkpoint files could be read correctly, then return None. + */ + def read(checkpointDir: String): Option[Checkpoint] = { + read(checkpointDir, new SparkConf(), SparkHadoopUtil.get.conf, ignoreReadError = true) + } + /** * Read checkpoint files present in the given checkpoint directory. If there are no checkpoint * files, then return None, else try to return the latest valid checkpoint object. If no From 4e0395ddb764d092b5b38447af49e196e590e0f0 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 24 Aug 2015 12:38:01 -0700 Subject: [PATCH 054/232] [SPARK-10168] [STREAMING] Fix the issue that maven publishes wrong artifact jars This PR removed the `outputFile` configuration from pom.xml and updated `tests.py` to search jars for both sbt build and maven build. I ran ` mvn -Pkinesis-asl -DskipTests clean install` locally, and verified the jars in my local repository were correct. I also checked Python tests for maven build, and it passed all tests. Author: zsxwing Closes #8373 from zsxwing/SPARK-10168 and squashes the following commits: e0b5818 [zsxwing] Fix the sbt build c697627 [zsxwing] Add the jar pathes to the exception message be1d8a5 [zsxwing] Fix the issue that maven publishes wrong artifact jars --- external/flume-assembly/pom.xml | 1 - external/kafka-assembly/pom.xml | 1 - external/mqtt-assembly/pom.xml | 1 - extras/kinesis-asl-assembly/pom.xml | 1 - python/pyspark/streaming/tests.py | 47 ++++++++++++++++------------- 5 files changed, 26 insertions(+), 25 deletions(-) diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index e05e4318969ce..561ed4babe5d0 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -115,7 +115,6 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-flume-assembly-${project.version}.jar *:* diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 36342f37bb2ea..6f4e2a89e9af7 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -142,7 +142,6 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-kafka-assembly-${project.version}.jar *:* diff --git a/external/mqtt-assembly/pom.xml b/external/mqtt-assembly/pom.xml index f3e3f93e7ed50..8412600633734 100644 --- a/external/mqtt-assembly/pom.xml +++ b/external/mqtt-assembly/pom.xml @@ -132,7 +132,6 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-mqtt-assembly-${project.version}.jar *:* diff --git a/extras/kinesis-asl-assembly/pom.xml b/extras/kinesis-asl-assembly/pom.xml index 3ca538608f694..51af3e6f2225f 100644 --- a/extras/kinesis-asl-assembly/pom.xml +++ b/extras/kinesis-asl-assembly/pom.xml @@ -137,7 +137,6 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-kinesis-asl-assembly-${project.version}.jar *:* diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 510a4f2b3e472..cfea95b0dec71 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -1162,11 +1162,20 @@ def get_output(_, rdd): kinesisTestUtils.deleteDynamoDBTable(kinesisAppName) +# Search jar in the project dir using the jar name_prefix for both sbt build and maven build because +# the artifact jars are in different directories. +def search_jar(dir, name_prefix): + # We should ignore the following jars + ignored_jar_suffixes = ("javadoc.jar", "sources.jar", "test-sources.jar", "tests.jar") + jars = (glob.glob(os.path.join(dir, "target/scala-*/" + name_prefix + "-*.jar")) + # sbt build + glob.glob(os.path.join(dir, "target/" + name_prefix + "_*.jar"))) # maven build + return [jar for jar in jars if not jar.endswith(ignored_jar_suffixes)] + + def search_kafka_assembly_jar(): SPARK_HOME = os.environ["SPARK_HOME"] kafka_assembly_dir = os.path.join(SPARK_HOME, "external/kafka-assembly") - jars = glob.glob( - os.path.join(kafka_assembly_dir, "target/scala-*/spark-streaming-kafka-assembly-*.jar")) + jars = search_jar(kafka_assembly_dir, "spark-streaming-kafka-assembly") if not jars: raise Exception( ("Failed to find Spark Streaming kafka assembly jar in %s. " % kafka_assembly_dir) + @@ -1174,8 +1183,8 @@ def search_kafka_assembly_jar(): "'build/sbt assembly/assembly streaming-kafka-assembly/assembly' or " "'build/mvn package' before running this test.") elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming Kafka assembly JARs in %s; please " - "remove all but one") % kafka_assembly_dir) + raise Exception(("Found multiple Spark Streaming Kafka assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) else: return jars[0] @@ -1183,8 +1192,7 @@ def search_kafka_assembly_jar(): def search_flume_assembly_jar(): SPARK_HOME = os.environ["SPARK_HOME"] flume_assembly_dir = os.path.join(SPARK_HOME, "external/flume-assembly") - jars = glob.glob( - os.path.join(flume_assembly_dir, "target/scala-*/spark-streaming-flume-assembly-*.jar")) + jars = search_jar(flume_assembly_dir, "spark-streaming-flume-assembly") if not jars: raise Exception( ("Failed to find Spark Streaming Flume assembly jar in %s. " % flume_assembly_dir) + @@ -1192,8 +1200,8 @@ def search_flume_assembly_jar(): "'build/sbt assembly/assembly streaming-flume-assembly/assembly' or " "'build/mvn package' before running this test.") elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming Flume assembly JARs in %s; please " - "remove all but one") % flume_assembly_dir) + raise Exception(("Found multiple Spark Streaming Flume assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) else: return jars[0] @@ -1201,8 +1209,7 @@ def search_flume_assembly_jar(): def search_mqtt_assembly_jar(): SPARK_HOME = os.environ["SPARK_HOME"] mqtt_assembly_dir = os.path.join(SPARK_HOME, "external/mqtt-assembly") - jars = glob.glob( - os.path.join(mqtt_assembly_dir, "target/scala-*/spark-streaming-mqtt-assembly-*.jar")) + jars = search_jar(mqtt_assembly_dir, "spark-streaming-mqtt-assembly") if not jars: raise Exception( ("Failed to find Spark Streaming MQTT assembly jar in %s. " % mqtt_assembly_dir) + @@ -1210,8 +1217,8 @@ def search_mqtt_assembly_jar(): "'build/sbt assembly/assembly streaming-mqtt-assembly/assembly' or " "'build/mvn package' before running this test") elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming MQTT assembly JARs in %s; please " - "remove all but one") % mqtt_assembly_dir) + raise Exception(("Found multiple Spark Streaming MQTT assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) else: return jars[0] @@ -1227,8 +1234,8 @@ def search_mqtt_test_jar(): "You need to build Spark with " "'build/sbt assembly/assembly streaming-mqtt/test:assembly'") elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming MQTT test JARs in %s; please " - "remove all but one") % mqtt_test_dir) + raise Exception(("Found multiple Spark Streaming MQTT test JARs: %s; please " + "remove all but one") % (", ".join(jars))) else: return jars[0] @@ -1236,14 +1243,12 @@ def search_mqtt_test_jar(): def search_kinesis_asl_assembly_jar(): SPARK_HOME = os.environ["SPARK_HOME"] kinesis_asl_assembly_dir = os.path.join(SPARK_HOME, "extras/kinesis-asl-assembly") - jars = glob.glob( - os.path.join(kinesis_asl_assembly_dir, - "target/scala-*/spark-streaming-kinesis-asl-assembly-*.jar")) + jars = search_jar(kinesis_asl_assembly_dir, "spark-streaming-kinesis-asl-assembly") if not jars: return None elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming Kinesis ASL assembly JARs in %s; please " - "remove all but one") % kinesis_asl_assembly_dir) + raise Exception(("Found multiple Spark Streaming Kinesis ASL assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) else: return jars[0] @@ -1269,8 +1274,8 @@ def search_kinesis_asl_assembly_jar(): mqtt_test_jar, kinesis_asl_assembly_jar) os.environ["PYSPARK_SUBMIT_ARGS"] = "--jars %s pyspark-shell" % jars - testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, - CheckpointTests, KafkaStreamTests, FlumeStreamTests, FlumePollingStreamTests] + testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, CheckpointTests, + KafkaStreamTests, FlumeStreamTests, FlumePollingStreamTests, MQTTStreamTests] if kinesis_jar_present is True: testcases.append(KinesisStreamTests) From 7478c8b66d6a2b1179f20c38b49e27e37b0caec3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Aug 2015 12:40:09 -0700 Subject: [PATCH 055/232] [SPARK-9791] [PACKAGE] Change private class to private class to prevent unnecessary classes from showing up in the docs In addition, some random cleanup of import ordering Author: Tathagata Das Closes #8387 from tdas/SPARK-9791 and squashes the following commits: 67f3ee9 [Tathagata Das] Change private class to private[package] class to prevent them from showing up in the docs --- .../spark/streaming/flume/FlumeUtils.scala | 2 +- .../apache/spark/streaming/kafka/Broker.scala | 6 ++-- .../streaming/kafka/KafkaTestUtils.scala | 10 +++--- .../spark/streaming/kafka/KafkaUtils.scala | 36 +++++-------------- .../spark/streaming/kafka/OffsetRange.scala | 8 ----- .../spark/streaming/mqtt/MQTTUtils.scala | 6 ++-- .../spark/streaming/mqtt/MQTTTestUtils.scala | 2 +- .../streaming/kinesis/KinesisTestUtils.scala | 2 +- .../spark/streaming/util/WriteAheadLog.java | 2 ++ .../util/WriteAheadLogRecordHandle.java | 2 ++ .../receiver/ReceivedBlockHandler.scala | 2 +- .../streaming/scheduler/ReceiverTracker.scala | 2 +- .../apache/spark/streaming/ui/BatchPage.scala | 2 +- 13 files changed, 28 insertions(+), 54 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 095bfb0c73a9a..a65a9b921aafa 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -247,7 +247,7 @@ object FlumeUtils { * This is a helper class that wraps the methods in FlumeUtils into more Python-friendly class and * function so that it can be easily instantiated and called from Python's FlumeUtils. */ -private class FlumeUtilsPythonHelper { +private[flume] class FlumeUtilsPythonHelper { def createStream( jssc: JavaStreamingContext, diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala index 5a74febb4bd46..9159051ba06e4 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala @@ -20,11 +20,9 @@ package org.apache.spark.streaming.kafka import org.apache.spark.annotation.Experimental /** - * :: Experimental :: - * Represent the host and port info for a Kafka broker. - * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID + * Represents the host and port info for a Kafka broker. + * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID. */ -@Experimental final class Broker private( /** Broker's hostname */ val host: String, diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index b608b75952721..79a9db4291bef 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -20,9 +20,8 @@ package org.apache.spark.streaming.kafka import java.io.File import java.lang.{Integer => JInt} import java.net.InetSocketAddress -import java.util.{Map => JMap} -import java.util.Properties import java.util.concurrent.TimeoutException +import java.util.{Map => JMap, Properties} import scala.annotation.tailrec import scala.language.postfixOps @@ -30,17 +29,16 @@ import scala.util.control.NonFatal import kafka.admin.AdminUtils import kafka.api.Request -import kafka.common.TopicAndPartition import kafka.producer.{KeyedMessage, Producer, ProducerConfig} import kafka.serializer.StringEncoder import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.{ZKStringSerializer, ZkUtils} -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} -import org.apache.spark.{Logging, SparkConf} import org.apache.spark.streaming.Time import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf} /** * This is a helper class for Kafka test suites. This has the functionality to set up @@ -48,7 +46,7 @@ import org.apache.spark.util.Utils * * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. */ -private class KafkaTestUtils extends Logging { +private[kafka] class KafkaTestUtils extends Logging { // Zookeeper related configurations private val zkHost = "localhost" diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index f3b01bd60b178..388dbb8184106 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -17,29 +17,25 @@ package org.apache.spark.streaming.kafka -import java.lang.{Integer => JInt} -import java.lang.{Long => JLong} -import java.util.{Map => JMap} -import java.util.{Set => JSet} -import java.util.{List => JList} +import java.lang.{Integer => JInt, Long => JLong} +import java.util.{List => JList, Map => JMap, Set => JSet} -import scala.reflect.ClassTag import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairInputDStream, JavaInputDStream, JavaPairReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.api.java.{JavaInputDStream, JavaPairInputDStream, JavaPairReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} -import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import org.apache.spark.streaming.util.WriteAheadLogUtils +import org.apache.spark.{SparkContext, SparkException} object KafkaUtils { /** @@ -196,7 +192,6 @@ object KafkaUtils { * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition */ - @Experimental def createRDD[ K: ClassTag, V: ClassTag, @@ -214,7 +209,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you * specify the Kafka leader to connect to (to optimize fetching) and access the message as well * as the metadata. @@ -230,7 +224,6 @@ object KafkaUtils { * in which case leaders will be looked up on the driver. * @param messageHandler Function for translating each message and metadata into the desired type */ - @Experimental def createRDD[ K: ClassTag, V: ClassTag, @@ -268,7 +261,6 @@ object KafkaUtils { * @param offsetRanges Each OffsetRange in the batch corresponds to a * range of offsets for a given Kafka topic/partition */ - @Experimental def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( jsc: JavaSparkContext, keyClass: Class[K], @@ -287,7 +279,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you * specify the Kafka leader to connect to (to optimize fetching) and access the message as well * as the metadata. @@ -303,7 +294,6 @@ object KafkaUtils { * in which case leaders will be looked up on the driver. * @param messageHandler Function for translating each message and metadata into the desired type */ - @Experimental def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( jsc: JavaSparkContext, keyClass: Class[K], @@ -327,7 +317,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create an input stream that directly pulls messages from Kafka Brokers * without using any receiver. This stream can guarantee that each message * from Kafka is included in transformations exactly once (see points below). @@ -357,7 +346,6 @@ object KafkaUtils { * starting point of the stream * @param messageHandler Function for translating each message and metadata into the desired type */ - @Experimental def createDirectStream[ K: ClassTag, V: ClassTag, @@ -375,7 +363,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create an input stream that directly pulls messages from Kafka Brokers * without using any receiver. This stream can guarantee that each message * from Kafka is included in transformations exactly once (see points below). @@ -405,7 +392,6 @@ object KafkaUtils { * to determine where the stream starts (defaults to "largest") * @param topics Names of the topics to consume */ - @Experimental def createDirectStream[ K: ClassTag, V: ClassTag, @@ -437,7 +423,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create an input stream that directly pulls messages from Kafka Brokers * without using any receiver. This stream can guarantee that each message * from Kafka is included in transformations exactly once (see points below). @@ -472,7 +457,6 @@ object KafkaUtils { * starting point of the stream * @param messageHandler Function for translating each message and metadata into the desired type */ - @Experimental def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( jssc: JavaStreamingContext, keyClass: Class[K], @@ -499,7 +483,6 @@ object KafkaUtils { } /** - * :: Experimental :: * Create an input stream that directly pulls messages from Kafka Brokers * without using any receiver. This stream can guarantee that each message * from Kafka is included in transformations exactly once (see points below). @@ -533,7 +516,6 @@ object KafkaUtils { * to determine where the stream starts (defaults to "largest") * @param topics Names of the topics to consume */ - @Experimental def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( jssc: JavaStreamingContext, keyClass: Class[K], @@ -564,7 +546,7 @@ object KafkaUtils { * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() * takes care of known parameters instead of passing them from Python */ -private class KafkaUtilsPythonHelper { +private[kafka] class KafkaUtilsPythonHelper { def createStream( jssc: JavaStreamingContext, kafkaParams: JMap[String, String], diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 2f8981d4898bd..8a5f371494511 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -19,10 +19,7 @@ package org.apache.spark.streaming.kafka import kafka.common.TopicAndPartition -import org.apache.spark.annotation.Experimental - /** - * :: Experimental :: * Represents any object that has a collection of [[OffsetRange]]s. This can be used access the * offset ranges in RDDs generated by the direct Kafka DStream (see * [[KafkaUtils.createDirectStream()]]). @@ -33,13 +30,11 @@ import org.apache.spark.annotation.Experimental * } * }}} */ -@Experimental trait HasOffsetRanges { def offsetRanges: Array[OffsetRange] } /** - * :: Experimental :: * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class * can be created with `OffsetRange.create()`. * @param topic Kafka topic name @@ -47,7 +42,6 @@ trait HasOffsetRanges { * @param fromOffset Inclusive starting offset * @param untilOffset Exclusive ending offset */ -@Experimental final class OffsetRange private( val topic: String, val partition: Int, @@ -84,10 +78,8 @@ final class OffsetRange private( } /** - * :: Experimental :: * Companion object the provides methods to create instances of [[OffsetRange]]. */ -@Experimental object OffsetRange { def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = new OffsetRange(topic, partition, fromOffset, untilOffset) diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index 38a1114863d15..7b8d56d6faf2d 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -21,8 +21,8 @@ import scala.reflect.ClassTag import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream object MQTTUtils { /** @@ -79,7 +79,7 @@ object MQTTUtils { * This is a helper class that wraps the methods in MQTTUtils into more Python-friendly class and * function so that it can be easily instantiated and called from Python's MQTTUtils. */ -private class MQTTUtilsPythonHelper { +private[mqtt] class MQTTUtilsPythonHelper { def createStream( jssc: JavaStreamingContext, diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala index 1a371b7008824..1618e2c088b70 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala @@ -33,7 +33,7 @@ import org.apache.spark.{Logging, SparkConf} /** * Share codes for Scala and Python unit tests */ -private class MQTTTestUtils extends Logging { +private[mqtt] class MQTTTestUtils extends Logging { private val persistenceDir = Utils.createTempDir() private val brokerHost = "localhost" diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 711aade182945..c8eec13ec7dc7 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -36,7 +36,7 @@ import org.apache.spark.Logging /** * Shared utility methods for performing Kinesis tests that actually transfer data */ -private class KinesisTestUtils extends Logging { +private[kinesis] class KinesisTestUtils extends Logging { val endpointUrl = KinesisTestUtils.endpointUrl val regionName = RegionUtils.getRegionByEndpoint(endpointUrl).getName() diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java index 8c0fdfa9c7478..3738fc1a235c2 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLog.java @@ -21,6 +21,8 @@ import java.util.Iterator; /** + * :: DeveloperApi :: + * * This abstract class represents a write ahead log (aka journal) that is used by Spark Streaming * to save the received data (by receivers) and associated metadata to a reliable storage, so that * they can be recovered after driver failures. See the Spark documentation for more information diff --git a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java index 02324189b7822..662889e779fb2 100644 --- a/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java +++ b/streaming/src/main/java/org/apache/spark/streaming/util/WriteAheadLogRecordHandle.java @@ -18,6 +18,8 @@ package org.apache.spark.streaming.util; /** + * :: DeveloperApi :: + * * This abstract class represents a handle that refers to a record written in a * {@link org.apache.spark.streaming.util.WriteAheadLog WriteAheadLog}. * It must contain all the information necessary for the record to be read and returned by diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index c8dd6e06812dc..5f6c5b024085c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -222,7 +222,7 @@ private[streaming] object WriteAheadLogBasedBlockHandler { /** * A utility that will wrap the Iterator to get the count */ -private class CountingIterator[T](iterator: Iterator[T]) extends Iterator[T] { +private[streaming] class CountingIterator[T](iterator: Iterator[T]) extends Iterator[T] { private var _count = 0 private def isFullyConsumed: Boolean = !iterator.hasNext diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index aae3acf7aba3e..30d25a64e307a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -546,7 +546,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false * Function to start the receiver on the worker node. Use a class instead of closure to avoid * the serialization issue. */ -private class StartReceiverFunc( +private[streaming] class StartReceiverFunc( checkpointDirOption: Option[String], serializableHadoopConf: SerializableConfiguration) extends (Iterator[Receiver[_]] => Unit) with Serializable { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 0c891662c264f..90d1b0fadecfc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -28,7 +28,7 @@ import org.apache.spark.ui.{UIUtils => SparkUIUtils, WebUIPage} import org.apache.spark.streaming.ui.StreamingJobProgressListener.{SparkJobId, OutputOpId} import org.apache.spark.ui.jobs.UIData.JobUIData -private case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) +private[ui] case class SparkJobIdWithUIData(sparkJobId: SparkJobId, jobUIData: Option[JobUIData]) private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { private val streamingListener = parent.listener From 9ce0c7ad333f4a3c01207e5e9ed42bcafb99d894 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Mon, 24 Aug 2015 13:48:01 -0700 Subject: [PATCH 056/232] [SPARK-7710] [SPARK-7998] [DOCS] Docs for DataFrameStatFunctions This PR contains examples on how to use some of the Stat Functions available for DataFrames under `df.stat`. rxin Author: Burak Yavuz Closes #8378 from brkyvz/update-sql-docs. --- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../spark/sql/DataFrameStatFunctions.scala | 101 ++++++++++++++++++ 2 files changed, 102 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index d6688b24ae7d6..791c10c3d7ce7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -684,7 +684,7 @@ class DataFrame private[sql]( // make it a NamedExpression. case Column(u: UnresolvedAttribute) => UnresolvedAlias(u) case Column(expr: NamedExpression) => expr - // Leave an unaliased explode with an empty list of names since the analzyer will generate the + // Leave an unaliased explode with an empty list of names since the analyzer will generate the // correct defaults after the nested expression's type has been resolved. case Column(explode: Explode) => MultiAlias(explode, Nil) case Column(expr: Expression) => Alias(expr, expr.prettyString)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 2e68e358f2f1f..69c984717526d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -39,6 +39,13 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @param col2 the name of the second column * @return the covariance of the two columns. * + * {{{ + * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) + * .withColumn("rand2", rand(seed=27)) + * df.stat.cov("rand1", "rand2") + * res1: Double = 0.065... + * }}} + * * @since 1.4.0 */ def cov(col1: String, col2: String): Double = { @@ -54,6 +61,13 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @param col2 the name of the column to calculate the correlation against * @return The Pearson Correlation Coefficient as a Double. * + * {{{ + * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) + * .withColumn("rand2", rand(seed=27)) + * df.stat.corr("rand1", "rand2") + * res1: Double = 0.613... + * }}} + * * @since 1.4.0 */ def corr(col1: String, col2: String, method: String): Double = { @@ -69,6 +83,13 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @param col2 the name of the column to calculate the correlation against * @return The Pearson Correlation Coefficient as a Double. * + * {{{ + * val df = sc.parallelize(0 until 10).toDF("id").withColumn("rand1", rand(seed=10)) + * .withColumn("rand2", rand(seed=27)) + * df.stat.corr("rand1", "rand2", "pearson") + * res1: Double = 0.613... + * }}} + * * @since 1.4.0 */ def corr(col1: String, col2: String): Double = { @@ -92,6 +113,20 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * of the DataFrame. * @return A DataFrame containing for the contingency table. * + * {{{ + * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), + * (3, 3))).toDF("key", "value") + * val ct = df.stat.crosstab("key", "value") + * ct.show() + * +---------+---+---+---+ + * |key_value| 1| 2| 3| + * +---------+---+---+---+ + * | 2| 2| 0| 1| + * | 1| 1| 1| 0| + * | 3| 0| 1| 1| + * +---------+---+---+---+ + * }}} + * * @since 1.4.0 */ def crosstab(col1: String, col2: String): DataFrame = { @@ -112,6 +147,32 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * than 1e-4. * @return A Local DataFrame with the Array of frequent items for each column. * + * {{{ + * val rows = Seq.tabulate(100) { i => + * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) + * } + * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns + * // "a" and "b" + * val freqSingles = df.stat.freqItems(Array("a", "b"), 0.4) + * freqSingles.show() + * +-----------+-------------+ + * |a_freqItems| b_freqItems| + * +-----------+-------------+ + * | [1, 99]|[-1.0, -99.0]| + * +-----------+-------------+ + * // find the pair of items with a frequency greater than 0.1 in columns "a" and "b" + * val pairDf = df.select(struct("a", "b").as("a-b")) + * val freqPairs = pairDf.stat.freqItems(Array("a-b"), 0.1) + * freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show() + * +----------+ + * | freq_ab| + * +----------+ + * | [1,-1.0]| + * | ... | + * +----------+ + * }}} + * * @since 1.4.0 */ def freqItems(cols: Array[String], support: Double): DataFrame = { @@ -147,6 +208,32 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @param cols the names of the columns to search frequent items in. * @return A Local DataFrame with the Array of frequent items for each column. * + * {{{ + * val rows = Seq.tabulate(100) { i => + * if (i % 2 == 0) (1, -1.0) else (i, i * -1.0) + * } + * val df = sqlContext.createDataFrame(rows).toDF("a", "b") + * // find the items with a frequency greater than 0.4 (observed 40% of the time) for columns + * // "a" and "b" + * val freqSingles = df.stat.freqItems(Seq("a", "b"), 0.4) + * freqSingles.show() + * +-----------+-------------+ + * |a_freqItems| b_freqItems| + * +-----------+-------------+ + * | [1, 99]|[-1.0, -99.0]| + * +-----------+-------------+ + * // find the pair of items with a frequency greater than 0.1 in columns "a" and "b" + * val pairDf = df.select(struct("a", "b").as("a-b")) + * val freqPairs = pairDf.stat.freqItems(Seq("a-b"), 0.1) + * freqPairs.select(explode($"a-b_freqItems").as("freq_ab")).show() + * +----------+ + * | freq_ab| + * +----------+ + * | [1,-1.0]| + * | ... | + * +----------+ + * }}} + * * @since 1.4.0 */ def freqItems(cols: Seq[String], support: Double): DataFrame = { @@ -180,6 +267,20 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * @tparam T stratum type * @return a new [[DataFrame]] that represents the stratified sample * + * {{{ + * val df = sqlContext.createDataFrame(Seq((1, 1), (1, 2), (2, 1), (2, 1), (2, 3), (3, 2), + * (3, 3))).toDF("key", "value") + * val fractions = Map(1 -> 1.0, 3 -> 0.5) + * df.stat.sampleBy("key", fractions, 36L).show() + * +---+-----+ + * |key|value| + * +---+-----+ + * | 1| 1| + * | 1| 2| + * | 3| 2| + * +---+-----+ + * }}} + * * @since 1.5.0 */ def sampleBy[T](col: String, fractions: Map[T, Double], seed: Long): DataFrame = { From 662bb9667669cb07cf6d2ccee0d8e76bb561cd89 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 24 Aug 2015 14:10:50 -0700 Subject: [PATCH 057/232] [SPARK-10144] [UI] Actually show peak execution memory by default The peak execution memory metric was introduced in SPARK-8735. That was before Tungsten was enabled by default, so it assumed that `spark.sql.unsafe.enabled` must be explicitly set to true. The result is that the memory is not displayed by default. Author: Andrew Or Closes #8345 from andrewor14/show-memory-default. --- .../main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++---- .../test/scala/org/apache/spark/ui/StagePageSuite.scala | 8 ++++++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index fb4556b836859..4adc6596ba21c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -68,8 +68,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // if we find that it's okay. private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) - private val displayPeakExecutionMemory = - parent.conf.getOption("spark.sql.unsafe.enabled").exists(_.toBoolean) + private val displayPeakExecutionMemory = parent.conf.getBoolean("spark.sql.unsafe.enabled", true) def render(request: HttpServletRequest): Seq[Node] = { progressListener.synchronized { @@ -1193,8 +1192,7 @@ private[ui] class TaskPagedTable( desc: Boolean) extends PagedTable[TaskTableRowData] { // We only track peak memory used for unsafe operators - private val displayPeakExecutionMemory = - conf.getOption("spark.sql.unsafe.enabled").exists(_.toBoolean) + private val displayPeakExecutionMemory = conf.getBoolean("spark.sql.unsafe.enabled", true) override def tableId: String = "task-table" diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala index 98f9314f31dff..3388c6dca81f1 100644 --- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala @@ -33,14 +33,18 @@ class StagePageSuite extends SparkFunSuite with LocalSparkContext { test("peak execution memory only displayed if unsafe is enabled") { val unsafeConf = "spark.sql.unsafe.enabled" - val conf = new SparkConf().set(unsafeConf, "true") + val conf = new SparkConf(false).set(unsafeConf, "true") val html = renderStagePage(conf).toString().toLowerCase val targetString = "peak execution memory" assert(html.contains(targetString)) // Disable unsafe and make sure it's not there - val conf2 = new SparkConf().set(unsafeConf, "false") + val conf2 = new SparkConf(false).set(unsafeConf, "false") val html2 = renderStagePage(conf2).toString().toLowerCase assert(!html2.contains(targetString)) + // Avoid setting anything; it should be displayed by default + val conf3 = new SparkConf(false) + val html3 = renderStagePage(conf3).toString().toLowerCase + assert(html3.contains(targetString)) } /** From a2f4cdceba32aaa0df59df335ca0ce1ac73fc6c2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 24 Aug 2015 14:11:19 -0700 Subject: [PATCH 058/232] [SPARK-8580] [SQL] Refactors ParquetHiveCompatibilitySuite and adds more test cases This PR refactors `ParquetHiveCompatibilitySuite` so that it's easier to add new test cases. Hit two bugs, SPARK-10177 and HIVE-11625, while working on this, added test cases for them and marked as ignored for now. SPARK-10177 will be addressed in a separate PR. Author: Cheng Lian Closes #8392 from liancheng/spark-8580/parquet-hive-compat-tests. --- .../hive/ParquetHiveCompatibilitySuite.scala | 132 ++++++++++++------ 1 file changed, 93 insertions(+), 39 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index 13452e71a1b3b..bc30180cf0917 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -17,15 +17,17 @@ package org.apache.spark.sql.hive +import java.sql.Timestamp +import java.util.{Locale, TimeZone} + import org.apache.hadoop.hive.conf.HiveConf +import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.execution.datasources.parquet.ParquetCompatibilityTest +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.{Row, SQLConf, SQLContext} -class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest { - import ParquetCompatibilityTest.makeNullable - +class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with BeforeAndAfterAll { override def _sqlContext: SQLContext = TestHive private val sqlContext = _sqlContext @@ -35,69 +37,121 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest { */ private val stagingDir = new HiveConf().getVar(HiveConf.ConfVars.STAGINGDIR) - test("Read Parquet file generated by parquet-hive") { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + + protected override def beforeAll(): Unit = { + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + Locale.setDefault(Locale.US) + } + + override protected def afterAll(): Unit = { + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + } + + override protected def logParquetSchema(path: String): Unit = { + val schema = readParquetSchema(path, { path => + !path.getName.startsWith("_") && !path.getName.startsWith(stagingDir) + }) + + logInfo( + s"""Schema of the Parquet file written by parquet-avro: + |$schema + """.stripMargin) + } + + private def testParquetHiveCompatibility(row: Row, hiveTypes: String*): Unit = { withTable("parquet_compat") { withTempPath { dir => val path = dir.getCanonicalPath + // Hive columns are always nullable, so here we append a all-null row. + val rows = row :: Row(Seq.fill(row.length)(null): _*) :: Nil + + // Don't convert Hive metastore Parquet tables to let Hive write those Parquet files. withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { withTempTable("data") { - sqlContext.sql( + val fields = hiveTypes.zipWithIndex.map { case (typ, index) => s" col_$index $typ" } + + val ddl = s"""CREATE TABLE parquet_compat( - | bool_column BOOLEAN, - | byte_column TINYINT, - | short_column SMALLINT, - | int_column INT, - | long_column BIGINT, - | float_column FLOAT, - | double_column DOUBLE, - | - | strings_column ARRAY, - | int_to_string_column MAP + |${fields.mkString(",\n")} |) |STORED AS PARQUET |LOCATION '$path' + """.stripMargin + + logInfo( + s"""Creating testing Parquet table with the following DDL: + |$ddl """.stripMargin) + sqlContext.sql(ddl) + val schema = sqlContext.table("parquet_compat").schema - val rowRDD = sqlContext.sparkContext.parallelize(makeRows).coalesce(1) + val rowRDD = sqlContext.sparkContext.parallelize(rows).coalesce(1) sqlContext.createDataFrame(rowRDD, schema).registerTempTable("data") sqlContext.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") } } - val schema = readParquetSchema(path, { path => - !path.getName.startsWith("_") && !path.getName.startsWith(stagingDir) - }) - - logInfo( - s"""Schema of the Parquet file written by parquet-hive: - |$schema - """.stripMargin) + logParquetSchema(path) // Unfortunately parquet-hive doesn't add `UTF8` annotation to BINARY when writing strings. // Have to assume all BINARY values are strings here. withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true") { - checkAnswer(sqlContext.read.parquet(path), makeRows) + checkAnswer(sqlContext.read.parquet(path), rows) } } } } - def makeRows: Seq[Row] = { - (0 until 10).map { i => - def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + test("simple primitives") { + testParquetHiveCompatibility( + Row(true, 1.toByte, 2.toShort, 3, 4.toLong, 5.1f, 6.1d, "foo"), + "BOOLEAN", "TINYINT", "SMALLINT", "INT", "BIGINT", "FLOAT", "DOUBLE", "STRING") + } + ignore("SPARK-10177 timestamp") { + testParquetHiveCompatibility(Row(Timestamp.valueOf("2015-08-24 00:31:00")), "TIMESTAMP") + } + + test("array") { + testParquetHiveCompatibility( Row( - nullable(i % 2 == 0: java.lang.Boolean), - nullable(i.toByte: java.lang.Byte), - nullable((i + 1).toShort: java.lang.Short), - nullable(i + 2: Integer), - nullable(i.toLong * 10: java.lang.Long), - nullable(i.toFloat + 0.1f: java.lang.Float), - nullable(i.toDouble + 0.2d: java.lang.Double), - nullable(Seq.tabulate(3)(n => s"arr_${i + n}")), - nullable(Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap)) - } + Seq[Integer](1: Integer, null, 2: Integer, null), + Seq[String]("foo", null, "bar", null), + Seq[Seq[Integer]]( + Seq[Integer](1: Integer, null), + Seq[Integer](2: Integer, null))), + "ARRAY", + "ARRAY", + "ARRAY>") + } + + test("map") { + testParquetHiveCompatibility( + Row( + Map[Integer, String]( + (1: Integer) -> "foo", + (2: Integer) -> null)), + "MAP") + } + + // HIVE-11625: Parquet map entries with null keys are dropped by Hive + ignore("map entries with null keys") { + testParquetHiveCompatibility( + Row( + Map[Integer, String]( + null.asInstanceOf[Integer] -> "bar", + null.asInstanceOf[Integer] -> null)), + "MAP") + } + + test("struct") { + testParquetHiveCompatibility( + Row(Row(1, Seq("foo", "bar", null))), + "STRUCT>") } } From cb2d2e15844d7ae34b5dd7028b55e11586ed93fa Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 24 Aug 2015 22:35:21 +0100 Subject: [PATCH 059/232] [SPARK-9758] [TEST] [SQL] Compilation issue for hive test / wrong package? Move `test.org.apache.spark.sql.hive` package tests to apparent intended `org.apache.spark.sql.hive` as they don't intend to test behavior from outside org.apache.spark.* Alternate take, per discussion at https://github.com/apache/spark/pull/8051 I think this is what vanzin and I had in mind but also CC rxin to cross-check, as this does indeed depend on whether these tests were accidentally in this package or not. Testing from a `test.org.apache.spark` package is legitimate but didn't seem to be the intent here. Author: Sean Owen Closes #8307 from srowen/SPARK-9758. --- .../org/apache/spark/sql/hive/JavaDataFrameSuite.java | 6 ++---- .../spark/sql/hive/JavaMetastoreDataSourcesSuite.java | 3 +-- .../org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java | 2 +- .../org/apache/spark/sql/hive/aggregate/MyDoubleSum.java | 2 +- .../apache/spark/sql/hive/execution/UDFIntegerToString.java | 0 .../org/apache/spark/sql/hive/execution/UDFListListInt.java | 0 .../org/apache/spark/sql/hive/execution/UDFListString.java | 0 .../apache/spark/sql/hive/execution/UDFStringString.java | 0 .../org/apache/spark/sql/hive/execution/UDFTwoListList.java | 0 .../spark/sql/hive/execution/AggregationQuerySuite.scala | 2 +- 10 files changed, 6 insertions(+), 9 deletions(-) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/JavaDataFrameSuite.java (94%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java (98%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java (98%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java (98%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/execution/UDFIntegerToString.java (100%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/execution/UDFListListInt.java (100%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/execution/UDFListString.java (100%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/execution/UDFStringString.java (100%) rename sql/hive/src/test/java/{test => }/org/apache/spark/sql/hive/execution/UDFTwoListList.java (100%) diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java similarity index 94% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java index a30dfa554eabc..019d8a30266e2 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaDataFrameSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaDataFrameSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.hive; +package org.apache.spark.sql.hive; import java.io.IOException; import java.util.ArrayList; @@ -31,10 +31,8 @@ import org.apache.spark.sql.expressions.Window; import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; import static org.apache.spark.sql.functions.*; -import org.apache.spark.sql.hive.HiveContext; import org.apache.spark.sql.hive.test.TestHive$; -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; -import test.org.apache.spark.sql.hive.aggregate.MyDoubleSum; +import org.apache.spark.sql.hive.aggregate.MyDoubleSum; public class JavaDataFrameSuite { private transient JavaSparkContext sc; diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java similarity index 98% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 15c2c3deb0d83..4192155975c47 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.hive; +package org.apache.spark.sql.hive; import java.io.File; import java.io.IOException; @@ -37,7 +37,6 @@ import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.QueryTest$; import org.apache.spark.sql.Row; -import org.apache.spark.sql.hive.HiveContext; import org.apache.spark.sql.hive.test.TestHive$; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java similarity index 98% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java index 2961b803f14aa..5a167edd89592 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleAvg.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.hive.aggregate; +package org.apache.spark.sql.hive.aggregate; import java.util.ArrayList; import java.util.List; diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java similarity index 98% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java index c71882a6e7bed..c3b7768e71bf8 100644 --- a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/aggregate/MyDoubleSum.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package test.org.apache.spark.sql.hive.aggregate; +package org.apache.spark.sql.hive.aggregate; import java.util.ArrayList; import java.util.List; diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFIntegerToString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java similarity index 100% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFIntegerToString.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFListListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java similarity index 100% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFListListInt.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFListString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java similarity index 100% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFListString.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFStringString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java similarity index 100% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFStringString.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java diff --git a/sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFTwoListList.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java similarity index 100% rename from sql/hive/src/test/java/test/org/apache/spark/sql/hive/execution/UDFTwoListList.java rename to sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 119663af1887a..4886a85948367 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} -import _root_.test.org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum} +import org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum} abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with BeforeAndAfterAll { override def _sqlContext: SQLContext = TestHive From 13db11cb08eb90eb0ea3402c9fe0270aa282f247 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 24 Aug 2015 15:38:54 -0700 Subject: [PATCH 060/232] [SPARK-10061] [DOC] ML ensemble docs User guide for spark.ml GBTs and Random Forests. The examples are copied from the decision tree guide and modified to run. I caught some issues I had somehow missed in the tree guide as well. I have run all examples, including Java ones. (Of course, I thought I had previously as well...) CC: mengxr manishamde yanboliang Author: Joseph K. Bradley Closes #8369 from jkbradley/ml-ensemble-docs. --- docs/ml-decision-tree.md | 75 ++- docs/ml-ensembles.md | 952 ++++++++++++++++++++++++++++++++++++++- 2 files changed, 976 insertions(+), 51 deletions(-) diff --git a/docs/ml-decision-tree.md b/docs/ml-decision-tree.md index 958c6f5e4716c..542819e93e6dc 100644 --- a/docs/ml-decision-tree.md +++ b/docs/ml-decision-tree.md @@ -30,7 +30,7 @@ The Pipelines API for Decision Trees offers a bit more functionality than the or Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described in the [Ensembles guide](ml-ensembles.html). -# Inputs and Outputs (Predictions) +# Inputs and Outputs We list the input and output (prediction) column types here. All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. @@ -234,7 +234,7 @@ IndexToString labelConverter = new IndexToString() // Chain indexers and tree in a Pipeline Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); + .setStages(new PipelineStage[] {labelIndexer, featureIndexer, dt, labelConverter}); // Train model. This also runs the indexers. PipelineModel model = pipeline.fit(trainingData); @@ -315,10 +315,13 @@ print treeModel # summary only ## Regression +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the Decision Tree algorithm can recognize. +
-More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.classification.DecisionTreeClassifier). +More details on parameters can be found in the [Scala API documentation](api/scala/index.html#org.apache.spark.ml.regression.DecisionTreeRegressor). {% highlight scala %} import org.apache.spark.ml.Pipeline @@ -347,7 +350,7 @@ val dt = new DecisionTreeRegressor() .setLabelCol("label") .setFeaturesCol("indexedFeatures") -// Chain indexers and tree in a Pipeline +// Chain indexer and tree in a Pipeline val pipeline = new Pipeline() .setStages(Array(featureIndexer, dt)) @@ -365,9 +368,7 @@ val evaluator = new RegressionEvaluator() .setLabelCol("label") .setPredictionCol("prediction") .setMetricName("rmse") -// We negate the RMSE value since RegressionEvalutor returns negated RMSE -// (since evaluation metrics are meant to be maximized by CrossValidator). -val rmse = - evaluator.evaluate(predictions) +val rmse = evaluator.evaluate(predictions) println("Root Mean Squared Error (RMSE) on test data = " + rmse) val treeModel = model.stages(1).asInstanceOf[DecisionTreeRegressionModel] @@ -377,14 +378,15 @@ println("Learned regression tree model:\n" + treeModel.toDebugString)
-More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/classification/DecisionTreeClassifier.html). +More details on parameters can be found in the [Java API documentation](api/java/org/apache/spark/ml/regression/DecisionTreeRegressor.html). {% highlight java %} import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; import org.apache.spark.ml.evaluation.RegressionEvaluator; -import org.apache.spark.ml.feature.*; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; import org.apache.spark.ml.regression.DecisionTreeRegressionModel; import org.apache.spark.ml.regression.DecisionTreeRegressor; import org.apache.spark.mllib.regression.LabeledPoint; @@ -396,17 +398,12 @@ import org.apache.spark.sql.DataFrame; RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); -// Index labels, adding metadata to the label column. -// Fit on whole dataset to include all labels in index. -StringIndexerModel labelIndexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("indexedLabel") - .fit(data); // Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. VectorIndexerModel featureIndexer = new VectorIndexer() .setInputCol("features") .setOutputCol("indexedFeatures") - .setMaxCategories(4) // features with > 4 distinct values are treated as continuous + .setMaxCategories(4) .fit(data); // Split the data into training and test sets (30% held out for testing) @@ -416,61 +413,49 @@ DataFrame testData = splits[1]; // Train a DecisionTree model. DecisionTreeRegressor dt = new DecisionTreeRegressor() - .setLabelCol("indexedLabel") .setFeaturesCol("indexedFeatures"); -// Convert indexed labels back to original labels. -IndexToString labelConverter = new IndexToString() - .setInputCol("prediction") - .setOutputCol("predictedLabel") - .setLabels(labelIndexer.labels()); - -// Chain indexers and tree in a Pipeline +// Chain indexer and tree in a Pipeline Pipeline pipeline = new Pipeline() - .setStages(new PipelineStage[]{labelIndexer, featureIndexer, dt, labelConverter}); + .setStages(new PipelineStage[] {featureIndexer, dt}); -// Train model. This also runs the indexers. +// Train model. This also runs the indexer. PipelineModel model = pipeline.fit(trainingData); // Make predictions. DataFrame predictions = model.transform(testData); // Select example rows to display. -predictions.select("predictedLabel", "label", "features").show(5); +predictions.select("label", "features").show(5); // Select (prediction, true label) and compute test error RegressionEvaluator evaluator = new RegressionEvaluator() - .setLabelCol("indexedLabel") + .setLabelCol("label") .setPredictionCol("prediction") .setMetricName("rmse"); -// We negate the RMSE value since RegressionEvalutor returns negated RMSE -// (since evaluation metrics are meant to be maximized by CrossValidator). -double rmse = - evaluator.evaluate(predictions); +double rmse = evaluator.evaluate(predictions); System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); DecisionTreeRegressionModel treeModel = - (DecisionTreeRegressionModel)(model.stages()[2]); + (DecisionTreeRegressionModel)(model.stages()[1]); System.out.println("Learned regression tree model:\n" + treeModel.toDebugString()); {% endhighlight %}
-More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.classification.DecisionTreeClassifier). +More details on parameters can be found in the [Python API documentation](api/python/pyspark.ml.html#pyspark.ml.regression.DecisionTreeRegressor). {% highlight python %} from pyspark.ml import Pipeline from pyspark.ml.regression import DecisionTreeRegressor -from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.feature import VectorIndexer from pyspark.ml.evaluation import RegressionEvaluator from pyspark.mllib.util import MLUtils # Load and parse the data file, converting it to a DataFrame. data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() -# Index labels, adding metadata to the label column. -# Fit on whole dataset to include all labels in index. -labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) # Automatically identify categorical features, and index them. # We specify maxCategories so features with > 4 distinct values are treated as continuous. featureIndexer =\ @@ -480,26 +465,24 @@ featureIndexer =\ (trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a DecisionTree model. -dt = DecisionTreeRegressor(labelCol="indexedLabel", featuresCol="indexedFeatures") +dt = DecisionTreeRegressor(featuresCol="indexedFeatures") -# Chain indexers and tree in a Pipeline -pipeline = Pipeline(stages=[labelIndexer, featureIndexer, dt]) +# Chain indexer and tree in a Pipeline +pipeline = Pipeline(stages=[featureIndexer, dt]) -# Train model. This also runs the indexers. +# Train model. This also runs the indexer. model = pipeline.fit(trainingData) # Make predictions. predictions = model.transform(testData) # Select example rows to display. -predictions.select("prediction", "indexedLabel", "features").show(5) +predictions.select("prediction", "label", "features").show(5) # Select (prediction, true label) and compute test error evaluator = RegressionEvaluator( - labelCol="indexedLabel", predictionCol="prediction", metricName="rmse") -# We negate the RMSE value since RegressionEvalutor returns negated RMSE -# (since evaluation metrics are meant to be maximized by CrossValidator). -rmse = -evaluator.evaluate(predictions) + labelCol="label", predictionCol="prediction", metricName="rmse") +rmse = evaluator.evaluate(predictions) print "Root Mean Squared Error (RMSE) on test data = %g" % rmse treeModel = model.stages[1] diff --git a/docs/ml-ensembles.md b/docs/ml-ensembles.md index 9ff50e95fc479..62749909e01dc 100644 --- a/docs/ml-ensembles.md +++ b/docs/ml-ensembles.md @@ -11,11 +11,947 @@ displayTitle: ML - Ensembles An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) is a learning algorithm which creates a model composed of a set of other base models. -The Pipelines API supports the following ensemble algorithms: [`OneVsRest`](api/scala/index.html#org.apache.spark.ml.classifier.OneVsRest) -## OneVsRest +## Tree Ensembles -[OneVsRest](http://en.wikipedia.org/wiki/Multiclass_classification#One-vs.-rest) is an example of a machine learning reduction for performing multiclass classification given a base classifier that can perform binary classification efficiently. +The Pipelines API supports two major tree ensemble algorithms: [Random Forests](http://en.wikipedia.org/wiki/Random_forest) and [Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting). +Both use [MLlib decision trees](ml-decision-tree.html) as their base models. + +Users can find more information about ensemble algorithms in the [MLlib Ensemble guide](mllib-ensembles.html). In this section, we demonstrate the Pipelines API for ensembles. + +The main differences between this API and the [original MLlib ensembles API](mllib-ensembles.html) are: +* support for ML Pipelines +* separation of classification vs. regression +* use of DataFrame metadata to distinguish continuous and categorical features +* a bit more functionality for random forests: estimates of feature importance, as well as the predicted probability of each class (a.k.a. class conditional probabilities) for classification. + +### Random Forests + +[Random forests](http://en.wikipedia.org/wiki/Random_forest) +are ensembles of [decision trees](ml-decision-tree.html). +Random forests combine many decision trees in order to reduce the risk of overfitting. +MLlib supports random forests for binary and multiclass classification and for regression, +using both continuous and categorical features. + +This section gives examples of using random forests with the Pipelines API. +For more information on the algorithm, please see the [main MLlib docs on random forests](mllib-ensembles.html). + +#### Inputs and Outputs + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +##### Input Columns + +
+ + + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescription
labelColDouble"label"Label to predict
featuresColVector"features"Feature vector
+ +##### Output Columns (Predictions) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescriptionNotes
predictionColDouble"prediction"Predicted label
rawPredictionColVector"rawPrediction"Vector of length # classes, with the counts of training instance labels at the tree node which makes the predictionClassification only
probabilityColVector"probability"Vector of length # classes equal to rawPrediction normalized to a multinomial distributionClassification only
+ +#### Example: Classification + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. + +
+
+ +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.RandomForestClassifier) for more details. + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.RandomForestClassifier +import org.apache.spark.ml.classification.RandomForestClassificationModel +import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a RandomForest model. +val rf = new RandomForestClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setNumTrees(10) + +// Convert indexed labels back to original labels. +val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + +// Chain indexers and forest in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, rf, labelConverter)) + +// Train model. This also runs the indexers. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision") +val accuracy = evaluator.evaluate(predictions) +println("Test Error = " + (1.0 - accuracy)) + +val rfModel = model.stages(2).asInstanceOf[RandomForestClassificationModel] +println("Learned classification forest model:\n" + rfModel.toDebugString) +{% endhighlight %} +
+ +
+ +Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/RandomForestClassifier.html) for more details. + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.RandomForestClassifier; +import org.apache.spark.ml.classification.RandomForestClassificationModel; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a RandomForest model. +RandomForestClassifier rf = new RandomForestClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures"); + +// Convert indexed labels back to original labels. +IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + +// Chain indexers and forest in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {labelIndexer, featureIndexer, rf, labelConverter}); + +// Train model. This also runs the indexers. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision"); +double accuracy = evaluator.evaluate(predictions); +System.out.println("Test Error = " + (1.0 - accuracy)); + +RandomForestClassificationModel rfModel = + (RandomForestClassificationModel)(model.stages()[2]); +System.out.println("Learned classification forest model:\n" + rfModel.toDebugString()); +{% endhighlight %} +
+ +
+ +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.RandomForestClassifier) for more details. + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.classification import RandomForestClassifier +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Index labels, adding metadata to the label column. +# Fit on whole dataset to include all labels in index. +labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) +# Automatically identify categorical features, and index them. +# Set maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +rf = RandomForestClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures") + +# Chain indexers and forest in a Pipeline +pipeline = Pipeline(stages=[labelIndexer, featureIndexer, rf]) + +# Train model. This also runs the indexers. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "indexedLabel", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="precision") +accuracy = evaluator.evaluate(predictions) +print "Test Error = %g" % (1.0 - accuracy) + +rfModel = model.stages[2] +print rfModel # summary only +{% endhighlight %} +
+
+ +#### Example: Regression + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use a feature transformer to index categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. + +
+
+ +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.RandomForestRegressor) for more details. + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.regression.RandomForestRegressor +import org.apache.spark.ml.regression.RandomForestRegressionModel +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a RandomForest model. +val rf = new RandomForestRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + +// Chain indexer and forest in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(featureIndexer, rf)) + +// Train model. This also runs the indexer. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("prediction", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") +val rmse = evaluator.evaluate(predictions) +println("Root Mean Squared Error (RMSE) on test data = " + rmse) + +val rfModel = model.stages(1).asInstanceOf[RandomForestRegressionModel] +println("Learned regression forest model:\n" + rfModel.toDebugString) +{% endhighlight %} +
+ +
+ +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/RandomForestRegressor.html) for more details. + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.ml.regression.RandomForestRegressionModel; +import org.apache.spark.ml.regression.RandomForestRegressor; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a RandomForest model. +RandomForestRegressor rf = new RandomForestRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures"); + +// Chain indexer and forest in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {featureIndexer, rf}); + +// Train model. This also runs the indexer. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("prediction", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); +double rmse = evaluator.evaluate(predictions); +System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + +RandomForestRegressionModel rfModel = + (RandomForestRegressionModel)(model.stages()[1]); +System.out.println("Learned regression forest model:\n" + rfModel.toDebugString()); +{% endhighlight %} +
+ +
+ +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.RandomForestRegressor) for more details. + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.regression import RandomForestRegressor +from pyspark.ml.feature import VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Automatically identify categorical features, and index them. +# Set maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +rf = RandomForestRegressor(featuresCol="indexedFeatures") + +# Chain indexer and forest in a Pipeline +pipeline = Pipeline(stages=[featureIndexer, rf]) + +# Train model. This also runs the indexer. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "label", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") +rmse = evaluator.evaluate(predictions) +print "Root Mean Squared Error (RMSE) on test data = %g" % rmse + +rfModel = model.stages[1] +print rfModel # summary only +{% endhighlight %} +
+
+ +### Gradient-Boosted Trees (GBTs) + +[Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) +are ensembles of [decision trees](ml-decision-tree.html). +GBTs iteratively train decision trees in order to minimize a loss function. +MLlib supports GBTs for binary classification and for regression, +using both continuous and categorical features. + +This section gives examples of using GBTs with the Pipelines API. +For more information on the algorithm, please see the [main MLlib docs on GBTs](mllib-ensembles.html). + +#### Inputs and Outputs + +We list the input and output (prediction) column types here. +All output columns are optional; to exclude an output column, set its corresponding Param to an empty string. + +##### Input Columns + + + + + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescription
labelColDouble"label"Label to predict
featuresColVector"features"Feature vector
+ +Note that `GBTClassifier` currently only supports binary labels. + +##### Output Columns (Predictions) + + + + + + + + + + + + + + + + + + + + +
Param nameType(s)DefaultDescriptionNotes
predictionColDouble"prediction"Predicted label
+ +In the future, `GBTClassifier` will also output columns for `rawPrediction` and `probability`, just as `RandomForestClassifier` does. + +#### Example: Classification + +The following examples load a dataset in LibSVM format, split it into training and test sets, train on the first dataset, and then evaluate on the held-out test set. +We use two feature transformers to prepare the data; these help index categories for the label and categorical features, adding metadata to the `DataFrame` which the tree-based algorithms can recognize. + +
+
+ +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classification.GBTClassifier) for more details. + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.GBTClassifier +import org.apache.spark.ml.classification.GBTClassificationModel +import org.apache.spark.ml.feature.{StringIndexer, IndexToString, VectorIndexer} +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +val labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data) +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a GBT model. +val gbt = new GBTClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10) + +// Convert indexed labels back to original labels. +val labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels) + +// Chain indexers and GBT in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(labelIndexer, featureIndexer, gbt, labelConverter)) + +// Train model. This also runs the indexers. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision") +val accuracy = evaluator.evaluate(predictions) +println("Test Error = " + (1.0 - accuracy)) + +val gbtModel = model.stages(2).asInstanceOf[GBTClassificationModel] +println("Learned classification GBT model:\n" + gbtModel.toDebugString) +{% endhighlight %} +
+ +
+ +Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/GBTClassifier.html) for more details. + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.GBTClassifier; +import org.apache.spark.ml.classification.GBTClassificationModel; +import org.apache.spark.ml.evaluation.MulticlassClassificationEvaluator; +import org.apache.spark.ml.feature.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Index labels, adding metadata to the label column. +// Fit on whole dataset to include all labels in index. +StringIndexerModel labelIndexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("indexedLabel") + .fit(data); +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a GBT model. +GBTClassifier gbt = new GBTClassifier() + .setLabelCol("indexedLabel") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10); + +// Convert indexed labels back to original labels. +IndexToString labelConverter = new IndexToString() + .setInputCol("prediction") + .setOutputCol("predictedLabel") + .setLabels(labelIndexer.labels()); + +// Chain indexers and GBT in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {labelIndexer, featureIndexer, gbt, labelConverter}); + +// Train model. This also runs the indexers. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("predictedLabel", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +MulticlassClassificationEvaluator evaluator = new MulticlassClassificationEvaluator() + .setLabelCol("indexedLabel") + .setPredictionCol("prediction") + .setMetricName("precision"); +double accuracy = evaluator.evaluate(predictions); +System.out.println("Test Error = " + (1.0 - accuracy)); + +GBTClassificationModel gbtModel = + (GBTClassificationModel)(model.stages()[2]); +System.out.println("Learned classification GBT model:\n" + gbtModel.toDebugString()); +{% endhighlight %} +
+ +
+ +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.classification.GBTClassifier) for more details. + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.classification import GBTClassifier +from pyspark.ml.feature import StringIndexer, VectorIndexer +from pyspark.ml.evaluation import MulticlassClassificationEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Index labels, adding metadata to the label column. +# Fit on whole dataset to include all labels in index. +labelIndexer = StringIndexer(inputCol="label", outputCol="indexedLabel").fit(data) +# Automatically identify categorical features, and index them. +# Set maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a GBT model. +gbt = GBTClassifier(labelCol="indexedLabel", featuresCol="indexedFeatures", maxIter=10) + +# Chain indexers and GBT in a Pipeline +pipeline = Pipeline(stages=[labelIndexer, featureIndexer, gbt]) + +# Train model. This also runs the indexers. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "indexedLabel", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = MulticlassClassificationEvaluator( + labelCol="indexedLabel", predictionCol="prediction", metricName="precision") +accuracy = evaluator.evaluate(predictions) +print "Test Error = %g" % (1.0 - accuracy) + +gbtModel = model.stages[2] +print gbtModel # summary only +{% endhighlight %} +
+
+ +#### Example: Regression + +Note: For this example dataset, `GBTRegressor` actually only needs 1 iteration, but that will not +be true in general. + +
+
+ +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.regression.GBTRegressor) for more details. + +{% highlight scala %} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.regression.GBTRegressor +import org.apache.spark.ml.regression.GBTRegressionModel +import org.apache.spark.ml.feature.VectorIndexer +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file, converting it to a DataFrame. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +val featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data) + +// Split the data into training and test sets (30% held out for testing) +val Array(trainingData, testData) = data.randomSplit(Array(0.7, 0.3)) + +// Train a GBT model. +val gbt = new GBTRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10) + +// Chain indexer and GBT in a Pipeline +val pipeline = new Pipeline() + .setStages(Array(featureIndexer, gbt)) + +// Train model. This also runs the indexer. +val model = pipeline.fit(trainingData) + +// Make predictions. +val predictions = model.transform(testData) + +// Select example rows to display. +predictions.select("prediction", "label", "features").show(5) + +// Select (prediction, true label) and compute test error +val evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse") +val rmse = evaluator.evaluate(predictions) +println("Root Mean Squared Error (RMSE) on test data = " + rmse) + +val gbtModel = model.stages(1).asInstanceOf[GBTRegressionModel] +println("Learned regression GBT model:\n" + gbtModel.toDebugString) +{% endhighlight %} +
+ +
+ +Refer to the [Java API docs](api/java/org/apache/spark/ml/regression/GBTRegressor.html) for more details. + +{% highlight java %} +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.feature.VectorIndexer; +import org.apache.spark.ml.feature.VectorIndexerModel; +import org.apache.spark.ml.regression.GBTRegressionModel; +import org.apache.spark.ml.regression.GBTRegressor; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +// Load and parse the data file, converting it to a DataFrame. +RDD rdd = MLUtils.loadLibSVMFile(sc.sc(), "data/mllib/sample_libsvm_data.txt"); +DataFrame data = jsql.createDataFrame(rdd, LabeledPoint.class); + +// Automatically identify categorical features, and index them. +// Set maxCategories so features with > 4 distinct values are treated as continuous. +VectorIndexerModel featureIndexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexedFeatures") + .setMaxCategories(4) + .fit(data); + +// Split the data into training and test sets (30% held out for testing) +DataFrame[] splits = data.randomSplit(new double[] {0.7, 0.3}); +DataFrame trainingData = splits[0]; +DataFrame testData = splits[1]; + +// Train a GBT model. +GBTRegressor gbt = new GBTRegressor() + .setLabelCol("label") + .setFeaturesCol("indexedFeatures") + .setMaxIter(10); + +// Chain indexer and GBT in a Pipeline +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {featureIndexer, gbt}); + +// Train model. This also runs the indexer. +PipelineModel model = pipeline.fit(trainingData); + +// Make predictions. +DataFrame predictions = model.transform(testData); + +// Select example rows to display. +predictions.select("prediction", "label", "features").show(5); + +// Select (prediction, true label) and compute test error +RegressionEvaluator evaluator = new RegressionEvaluator() + .setLabelCol("label") + .setPredictionCol("prediction") + .setMetricName("rmse"); +double rmse = evaluator.evaluate(predictions); +System.out.println("Root Mean Squared Error (RMSE) on test data = " + rmse); + +GBTRegressionModel gbtModel = + (GBTRegressionModel)(model.stages()[1]); +System.out.println("Learned regression GBT model:\n" + gbtModel.toDebugString()); +{% endhighlight %} +
+ +
+ +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.regression.GBTRegressor) for more details. + +{% highlight python %} +from pyspark.ml import Pipeline +from pyspark.ml.regression import GBTRegressor +from pyspark.ml.feature import VectorIndexer +from pyspark.ml.evaluation import RegressionEvaluator +from pyspark.mllib.util import MLUtils + +# Load and parse the data file, converting it to a DataFrame. +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +# Automatically identify categorical features, and index them. +# Set maxCategories so features with > 4 distinct values are treated as continuous. +featureIndexer =\ + VectorIndexer(inputCol="features", outputCol="indexedFeatures", maxCategories=4).fit(data) + +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a GBT model. +gbt = GBTRegressor(featuresCol="indexedFeatures", maxIter=10) + +# Chain indexer and GBT in a Pipeline +pipeline = Pipeline(stages=[featureIndexer, gbt]) + +# Train model. This also runs the indexer. +model = pipeline.fit(trainingData) + +# Make predictions. +predictions = model.transform(testData) + +# Select example rows to display. +predictions.select("prediction", "label", "features").show(5) + +# Select (prediction, true label) and compute test error +evaluator = RegressionEvaluator( + labelCol="label", predictionCol="prediction", metricName="rmse") +rmse = evaluator.evaluate(predictions) +print "Root Mean Squared Error (RMSE) on test data = %g" % rmse + +gbtModel = model.stages[1] +print gbtModel # summary only +{% endhighlight %} +
+
+ + +## One-vs-Rest (a.k.a. One-vs-All) + +[OneVsRest](http://en.wikipedia.org/wiki/Multiclass_classification#One-vs.-rest) is an example of a machine learning reduction for performing multiclass classification given a base classifier that can perform binary classification efficiently. It is also known as "One-vs-All." `OneVsRest` is implemented as an `Estimator`. For the base classifier it takes instances of `Classifier` and creates a binary classification problem for each of the k classes. The classifier for class i is trained to predict whether the label is i or not, distinguishing class i from all other classes. @@ -28,6 +964,9 @@ The example below demonstrates how to load the
+ +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.classifier.OneVsRest) for more details. + {% highlight scala %} import org.apache.spark.ml.classification.{LogisticRegression, OneVsRest} import org.apache.spark.mllib.evaluation.MulticlassMetrics @@ -64,9 +1003,12 @@ println("label\tfpr\n") } {% endhighlight %}
+
-{% highlight java %} +Refer to the [Java API docs](api/java/org/apache/spark/ml/classification/OneVsRest.html) for more details. + +{% highlight java %} import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegression; @@ -88,7 +1030,7 @@ RDD data = MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_multiclass_classification_data.txt"); DataFrame dataFrame = jsql.createDataFrame(data, LabeledPoint.class); -DataFrame[] splits = dataFrame.randomSplit(new double[]{0.7, 0.3}, 12345); +DataFrame[] splits = dataFrame.randomSplit(new double[] {0.7, 0.3}, 12345); DataFrame train = splits[0]; DataFrame test = splits[1]; From d7b4c095271c36fcc7f9ded267ecf5ec66fac803 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 24 Aug 2015 16:17:45 -0700 Subject: [PATCH 061/232] [SPARK-10190] Fix NPE in CatalystTypeConverters Decimal toScala converter This adds a missing null check to the Decimal `toScala` converter in `CatalystTypeConverters`, fixing an NPE. Author: Josh Rosen Closes #8401 from JoshRosen/SPARK-10190. --- .../apache/spark/sql/catalyst/CatalystTypeConverters.scala | 5 ++++- .../spark/sql/catalyst/CatalystTypeConvertersSuite.scala | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 8d0c64eae4774..966623ed017ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -329,7 +329,10 @@ object CatalystTypeConverters { null } } - override def toScala(catalystValue: Decimal): JavaBigDecimal = catalystValue.toJavaBigDecimal + override def toScala(catalystValue: Decimal): JavaBigDecimal = { + if (catalystValue == null) null + else catalystValue.toJavaBigDecimal + } override def toScalaImpl(row: InternalRow, column: Int): JavaBigDecimal = row.getDecimal(column, dataType.precision, dataType.scale).toJavaBigDecimal } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala index df0f04563edcf..03bb102c67fe7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala @@ -32,7 +32,9 @@ class CatalystTypeConvertersSuite extends SparkFunSuite { IntegerType, LongType, FloatType, - DoubleType) + DoubleType, + DecimalType.SYSTEM_DEFAULT, + DecimalType.USER_DEFAULT) test("null handling in rows") { val schema = StructType(simpleTypes.map(t => StructField(t.getClass.getName, t))) From 2bf338c626e9d97ccc033cfadae8b36a82c66fd1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Aug 2015 18:10:51 -0700 Subject: [PATCH 062/232] [SPARK-10165] [SQL] Await child resolution in ResolveFunctions Currently, we eagerly attempt to resolve functions, even before their children are resolved. However, this is not valid in cases where we need to know the types of the input arguments (i.e. when resolving Hive UDFs). As a fix, this PR delays function resolution until the functions children are resolved. This change also necessitates a change to the way we resolve aggregate expressions that are not in aggregate operators (e.g., in `HAVING` or `ORDER BY` clauses). Specifically, we can't assume that these misplaced functions will be resolved, allowing us to differentiate aggregate functions from normal functions. To compensate for this change we now attempt to resolve these unresolved expressions in the context of the aggregate operator, before checking to see if any aggregate expressions are present. Author: Michael Armbrust Closes #8371 from marmbrus/hiveUDFResolution. --- .../sql/catalyst/analysis/Analyzer.scala | 116 +++++++++++------- .../sql/hive/execution/HiveUDFSuite.scala | 5 + 2 files changed, 77 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d0eb9c2c90bdf..1a5de15c61f86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -78,7 +78,7 @@ class Analyzer( ResolveAliases :: ExtractWindowExpressions :: GlobalAggregates :: - UnresolvedHavingClauseAttributes :: + ResolveAggregateFunctions :: HiveTypeCoercion.typeCoercionRules ++ extendedResolutionRules : _*), Batch("Nondeterministic", Once, @@ -452,37 +452,6 @@ class Analyzer( logDebug(s"Failed to find $missing in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } - case s @ Sort(ordering, global, a @ Aggregate(grouping, aggs, child)) - if !s.resolved && a.resolved => - // A small hack to create an object that will allow us to resolve any references that - // refer to named expressions that are present in the grouping expressions. - val groupingRelation = LocalRelation( - grouping.collect { case ne: NamedExpression => ne.toAttribute } - ) - - // Find sort attributes that are projected away so we can temporarily add them back in. - val (newOrdering, missingAttr) = resolveAndFindMissing(ordering, a, groupingRelation) - - // Find aggregate expressions and evaluate them early, since they can't be evaluated in a - // Sort. - val (withAggsRemoved, aliasedAggregateList) = newOrdering.map { - case aggOrdering if aggOrdering.collect { case a: AggregateExpression => a }.nonEmpty => - val aliased = Alias(aggOrdering.child, "_aggOrdering")() - (aggOrdering.copy(child = aliased.toAttribute), Some(aliased)) - - case other => (other, None) - }.unzip - - val missing = missingAttr ++ aliasedAggregateList.flatten - - if (missing.nonEmpty) { - // Add missing grouping exprs and then project them away after the sort. - Project(a.output, - Sort(withAggsRemoved, global, - Aggregate(grouping, aggs ++ missing, child))) - } else { - s // Nothing we can do here. Return original plan. - } } /** @@ -515,6 +484,7 @@ class Analyzer( def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case q: LogicalPlan => q transformExpressions { + case u if !u.childrenResolved => u // Skip until children are resolved. case u @ UnresolvedFunction(name, children, isDistinct) => withPosition(u) { registry.lookupFunction(name, children) match { @@ -559,21 +529,79 @@ class Analyzer( } /** - * This rule finds expressions in HAVING clause filters that depend on - * unresolved attributes. It pushes these expressions down to the underlying - * aggregates and then projects them away above the filter. + * This rule finds aggregate expressions that are not in an aggregate operator. For example, + * those in a HAVING clause or ORDER BY clause. These expressions are pushed down to the + * underlying aggregate operator and then projected away after the original operator. */ - object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { + object ResolveAggregateFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) - if aggregate.resolved && containsAggregate(havingCondition) => - - val evaluatedCondition = Alias(havingCondition, "havingCondition")() - val aggExprsWithHaving = evaluatedCondition +: originalAggExprs + case filter @ Filter(havingCondition, + aggregate @ Aggregate(grouping, originalAggExprs, child)) + if aggregate.resolved && !filter.resolved => + + // Try resolving the condition of the filter as though it is in the aggregate clause + val aggregatedCondition = + Aggregate(grouping, Alias(havingCondition, "havingCondition")() :: Nil, child) + val resolvedOperator = execute(aggregatedCondition) + def resolvedAggregateFilter = + resolvedOperator + .asInstanceOf[Aggregate] + .aggregateExpressions.head + + // If resolution was successful and we see the filter has an aggregate in it, add it to + // the original aggregate operator. + if (resolvedOperator.resolved && containsAggregate(resolvedAggregateFilter)) { + val aggExprsWithHaving = resolvedAggregateFilter +: originalAggExprs + + Project(aggregate.output, + Filter(resolvedAggregateFilter.toAttribute, + aggregate.copy(aggregateExpressions = aggExprsWithHaving))) + } else { + filter + } - Project(aggregate.output, - Filter(evaluatedCondition.toAttribute, - aggregate.copy(aggregateExpressions = aggExprsWithHaving))) + case sort @ Sort(sortOrder, global, + aggregate @ Aggregate(grouping, originalAggExprs, child)) + if aggregate.resolved && !sort.resolved => + + // Try resolving the ordering as though it is in the aggregate clause. + try { + val aliasedOrder = sortOrder.map(o => Alias(o.child, "aggOrder")()) + val aggregatedOrdering = Aggregate(grouping, aliasedOrder, child) + val resolvedOperator: Aggregate = execute(aggregatedOrdering).asInstanceOf[Aggregate] + def resolvedAggregateOrdering = resolvedOperator.aggregateExpressions + + // Expressions that have an aggregate can be pushed down. + val needsAggregate = resolvedAggregateOrdering.exists(containsAggregate) + + // Attribute references, that are missing from the order but are present in the grouping + // expressions can also be pushed down. + val requiredAttributes = resolvedAggregateOrdering.map(_.references).reduce(_ ++ _) + val missingAttributes = requiredAttributes -- aggregate.outputSet + val validPushdownAttributes = + missingAttributes.filter(a => grouping.exists(a.semanticEquals)) + + // If resolution was successful and we see the ordering either has an aggregate in it or + // it is missing something that is projected away by the aggregate, add the ordering + // the original aggregate operator. + if (resolvedOperator.resolved && (needsAggregate || validPushdownAttributes.nonEmpty)) { + val evaluatedOrderings: Seq[SortOrder] = sortOrder.zip(resolvedAggregateOrdering).map { + case (order, evaluated) => order.copy(child = evaluated.toAttribute) + } + val aggExprsWithOrdering: Seq[NamedExpression] = + resolvedAggregateOrdering ++ originalAggExprs + + Project(aggregate.output, + Sort(evaluatedOrderings, global, + aggregate.copy(aggregateExpressions = aggExprsWithOrdering))) + } else { + sort + } + } catch { + // Attempting to resolve in the aggregate can result in ambiguity. When this happens, + // just return the original plan. + case ae: AnalysisException => sort + } } protected def containsAggregate(condition: Expression): Boolean = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 10f2902e5eef0..b03a35132325d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -276,6 +276,11 @@ class HiveUDFSuite extends QueryTest { checkAnswer( sql("SELECT testStringStringUDF(\"hello\", s) FROM stringTable"), Seq(Row("hello world"), Row("hello goodbye"))) + + checkAnswer( + sql("SELECT testStringStringUDF(\"\", testStringStringUDF(\"hello\", s)) FROM stringTable"), + Seq(Row(" hello world"), Row(" hello goodbye"))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUDF") TestHive.reset() From 6511bf559b736d8e23ae398901c8d78938e66869 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Mon, 24 Aug 2015 18:17:51 -0700 Subject: [PATCH 063/232] [SPARK-10118] [SPARKR] [DOCS] Improve SparkR API docs for 1.5 release cc: shivaram ## Summary - Modify `tdname` of expression functions. i.e. `ascii`: `rdname functions` => `rdname ascii` - Replace the dynamical function definitions to the static ones because of thir documentations. ## Generated PDF File https://drive.google.com/file/d/0B9biIZIU47lLX2t6ZjRoRnBTSEU/view?usp=sharing ## JIRA [[SPARK-10118] Improve SparkR API docs for 1.5 release - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10118) Author: Yu ISHIKAWA Author: Yuu ISHIKAWA Closes #8386 from yu-iskw/SPARK-10118. --- R/create-docs.sh | 2 +- R/pkg/R/column.R | 5 +- R/pkg/R/functions.R | 1603 +++++++++++++++++++++++++++++++++++++++---- R/pkg/R/generics.R | 214 +++--- 4 files changed, 1596 insertions(+), 228 deletions(-) diff --git a/R/create-docs.sh b/R/create-docs.sh index 6a4687b06ecb9..d2ae160b50021 100755 --- a/R/create-docs.sh +++ b/R/create-docs.sh @@ -39,7 +39,7 @@ pushd $FWDIR mkdir -p pkg/html pushd pkg/html -Rscript -e 'library(SparkR, lib.loc="../../lib"); library(knitr); knit_rd("SparkR")' +Rscript -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knitr); knit_rd("SparkR", links = tools::findHTMLlinks(paste(libDir, "SparkR", sep="/")))' popd diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index 5a07ebd308296..a1f50c383367c 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -169,8 +169,7 @@ setMethod("between", signature(x = "Column"), #' #' @rdname column #' -#' @examples -#' \dontrun{ +#' @examples \dontrun{ #' cast(df$age, "string") #' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) #' } @@ -192,7 +191,7 @@ setMethod("cast", #' #' @rdname column #' @return a matched values as a result of comparing with given values. -#' \dontrun{ +#' @examples \dontrun{ #' filter(df, "age in (10, 30)") #' where(df, df$age %in% c(10, 30)) #' } diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index b5879bd9ad553..d848730e70433 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -18,69 +18,1298 @@ #' @include generics.R column.R NULL -#' @title S4 expression functions for DataFrame column(s) -#' @description These are expression functions on DataFrame columns - -functions1 <- c( - "abs", "acos", "approxCountDistinct", "ascii", "asin", "atan", - "avg", "base64", "bin", "bitwiseNOT", "cbrt", "ceil", "cos", "cosh", "count", - "crc32", "dayofmonth", "dayofyear", "exp", "explode", "expm1", "factorial", - "first", "floor", "hex", "hour", "initcap", "isNaN", "last", "last_day", - "length", "log", "log10", "log1p", "log2", "lower", "ltrim", "max", "md5", - "mean", "min", "minute", "month", "negate", "quarter", "reverse", - "rint", "round", "rtrim", "second", "sha1", "signum", "sin", "sinh", "size", - "soundex", "sqrt", "sum", "sumDistinct", "tan", "tanh", "toDegrees", - "toRadians", "to_date", "trim", "unbase64", "unhex", "upper", "weekofyear", - "year") -functions2 <- c( - "atan2", "datediff", "hypot", "levenshtein", "months_between", "nanvl", "pmod") - -createFunction1 <- function(name) { - setMethod(name, - signature(x = "Column"), - function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", name, x@jc) - column(jc) - }) -} - -createFunction2 <- function(name) { - setMethod(name, - signature(y = "Column"), - function(y, x) { - if (class(x) == "Column") { - x <- x@jc - } - jc <- callJStatic("org.apache.spark.sql.functions", name, y@jc, x) - column(jc) - }) -} +#' Creates a \code{Column} of literal value. +#' +#' The passed in object is returned directly if it is already a \linkS4class{Column}. +#' If the object is a Scala Symbol, it is converted into a \linkS4class{Column} also. +#' Otherwise, a new \linkS4class{Column} is created to represent the literal value. +#' +#' @family normal_funcs +#' @rdname lit +#' @name lit +#' @export +setMethod("lit", signature("ANY"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", + "lit", + ifelse(class(x) == "Column", x@jc, x)) + column(jc) + }) + +#' abs +#' +#' Computes the absolute value. +#' +#' @rdname abs +#' @name abs +#' @family normal_funcs +#' @export +#' @examples \dontrun{abs(df$c)} +setMethod("abs", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "abs", x@jc) + column(jc) + }) + +#' acos +#' +#' Computes the cosine inverse of the given value; the returned angle is in the range +#' 0.0 through pi. +#' +#' @rdname acos +#' @name acos +#' @family math_funcs +#' @export +#' @examples \dontrun{acos(df$c)} +setMethod("acos", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "acos", x@jc) + column(jc) + }) + +#' approxCountDistinct +#' +#' Aggregate function: returns the approximate number of distinct items in a group. +#' +#' @rdname approxCountDistinct +#' @name approxCountDistinct +#' @family agg_funcs +#' @export +#' @examples \dontrun{approxCountDistinct(df$c)} +setMethod("approxCountDistinct", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc) + column(jc) + }) + +#' ascii +#' +#' Computes the numeric value of the first character of the string column, and returns the +#' result as a int column. +#' +#' @rdname ascii +#' @name ascii +#' @family string_funcs +#' @export +#' @examples \dontrun{\dontrun{ascii(df$c)}} +setMethod("ascii", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "ascii", x@jc) + column(jc) + }) + +#' asin +#' +#' Computes the sine inverse of the given value; the returned angle is in the range +#' -pi/2 through pi/2. +#' +#' @rdname asin +#' @name asin +#' @family math_funcs +#' @export +#' @examples \dontrun{asin(df$c)} +setMethod("asin", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "asin", x@jc) + column(jc) + }) + +#' atan +#' +#' Computes the tangent inverse of the given value. +#' +#' @rdname atan +#' @name atan +#' @family math_funcs +#' @export +#' @examples \dontrun{atan(df$c)} +setMethod("atan", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "atan", x@jc) + column(jc) + }) + +#' avg +#' +#' Aggregate function: returns the average of the values in a group. +#' +#' @rdname avg +#' @name avg +#' @family agg_funcs +#' @export +#' @examples \dontrun{avg(df$c)} +setMethod("avg", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "avg", x@jc) + column(jc) + }) + +#' base64 +#' +#' Computes the BASE64 encoding of a binary column and returns it as a string column. +#' This is the reverse of unbase64. +#' +#' @rdname base64 +#' @name base64 +#' @family string_funcs +#' @export +#' @examples \dontrun{base64(df$c)} +setMethod("base64", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "base64", x@jc) + column(jc) + }) + +#' bin +#' +#' An expression that returns the string representation of the binary value of the given long +#' column. For example, bin("12") returns "1100". +#' +#' @rdname bin +#' @name bin +#' @family math_funcs +#' @export +#' @examples \dontrun{bin(df$c)} +setMethod("bin", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "bin", x@jc) + column(jc) + }) + +#' bitwiseNOT +#' +#' Computes bitwise NOT. +#' +#' @rdname bitwiseNOT +#' @name bitwiseNOT +#' @family normal_funcs +#' @export +#' @examples \dontrun{bitwiseNOT(df$c)} +setMethod("bitwiseNOT", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "bitwiseNOT", x@jc) + column(jc) + }) + +#' cbrt +#' +#' Computes the cube-root of the given value. +#' +#' @rdname cbrt +#' @name cbrt +#' @family math_funcs +#' @export +#' @examples \dontrun{cbrt(df$c)} +setMethod("cbrt", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "cbrt", x@jc) + column(jc) + }) + +#' ceil +#' +#' Computes the ceiling of the given value. +#' +#' @rdname ceil +#' @name ceil +#' @family math_funcs +#' @export +#' @examples \dontrun{ceil(df$c)} +setMethod("ceil", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "ceil", x@jc) + column(jc) + }) + +#' cos +#' +#' Computes the cosine of the given value. +#' +#' @rdname cos +#' @name cos +#' @family math_funcs +#' @export +#' @examples \dontrun{cos(df$c)} +setMethod("cos", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "cos", x@jc) + column(jc) + }) + +#' cosh +#' +#' Computes the hyperbolic cosine of the given value. +#' +#' @rdname cosh +#' @name cosh +#' @family math_funcs +#' @export +#' @examples \dontrun{cosh(df$c)} +setMethod("cosh", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "cosh", x@jc) + column(jc) + }) + +#' count +#' +#' Aggregate function: returns the number of items in a group. +#' +#' @rdname count +#' @name count +#' @family agg_funcs +#' @export +#' @examples \dontrun{count(df$c)} +setMethod("count", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "count", x@jc) + column(jc) + }) + +#' crc32 +#' +#' Calculates the cyclic redundancy check value (CRC32) of a binary column and +#' returns the value as a bigint. +#' +#' @rdname crc32 +#' @name crc32 +#' @family misc_funcs +#' @export +#' @examples \dontrun{crc32(df$c)} +setMethod("crc32", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "crc32", x@jc) + column(jc) + }) + +#' dayofmonth +#' +#' Extracts the day of the month as an integer from a given date/timestamp/string. +#' +#' @rdname dayofmonth +#' @name dayofmonth +#' @family datetime_funcs +#' @export +#' @examples \dontrun{dayofmonth(df$c)} +setMethod("dayofmonth", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "dayofmonth", x@jc) + column(jc) + }) + +#' dayofyear +#' +#' Extracts the day of the year as an integer from a given date/timestamp/string. +#' +#' @rdname dayofyear +#' @name dayofyear +#' @family datetime_funcs +#' @export +#' @examples \dontrun{dayofyear(df$c)} +setMethod("dayofyear", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "dayofyear", x@jc) + column(jc) + }) + +#' exp +#' +#' Computes the exponential of the given value. +#' +#' @rdname exp +#' @name exp +#' @family math_funcs +#' @export +#' @examples \dontrun{exp(df$c)} +setMethod("exp", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "exp", x@jc) + column(jc) + }) + +#' explode +#' +#' Creates a new row for each element in the given array or map column. +#' +#' @rdname explode +#' @name explode +#' @family collection_funcs +#' @export +#' @examples \dontrun{explode(df$c)} +setMethod("explode", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "explode", x@jc) + column(jc) + }) + +#' expm1 +#' +#' Computes the exponential of the given value minus one. +#' +#' @rdname expm1 +#' @name expm1 +#' @family math_funcs +#' @export +#' @examples \dontrun{expm1(df$c)} +setMethod("expm1", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "expm1", x@jc) + column(jc) + }) + +#' factorial +#' +#' Computes the factorial of the given value. +#' +#' @rdname factorial +#' @name factorial +#' @family math_funcs +#' @export +#' @examples \dontrun{factorial(df$c)} +setMethod("factorial", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "factorial", x@jc) + column(jc) + }) + +#' first +#' +#' Aggregate function: returns the first value in a group. +#' +#' @rdname first +#' @name first +#' @family agg_funcs +#' @export +#' @examples \dontrun{first(df$c)} +setMethod("first", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "first", x@jc) + column(jc) + }) + +#' floor +#' +#' Computes the floor of the given value. +#' +#' @rdname floor +#' @name floor +#' @family math_funcs +#' @export +#' @examples \dontrun{floor(df$c)} +setMethod("floor", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "floor", x@jc) + column(jc) + }) + +#' hex +#' +#' Computes hex value of the given column. +#' +#' @rdname hex +#' @name hex +#' @family math_funcs +#' @export +#' @examples \dontrun{hex(df$c)} +setMethod("hex", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "hex", x@jc) + column(jc) + }) + +#' hour +#' +#' Extracts the hours as an integer from a given date/timestamp/string. +#' +#' @rdname hour +#' @name hour +#' @family datetime_funcs +#' @export +#' @examples \dontrun{hour(df$c)} +setMethod("hour", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "hour", x@jc) + column(jc) + }) + +#' initcap +#' +#' Returns a new string column by converting the first letter of each word to uppercase. +#' Words are delimited by whitespace. +#' +#' For example, "hello world" will become "Hello World". +#' +#' @rdname initcap +#' @name initcap +#' @family string_funcs +#' @export +#' @examples \dontrun{initcap(df$c)} +setMethod("initcap", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "initcap", x@jc) + column(jc) + }) + +#' isNaN +#' +#' Return true iff the column is NaN. +#' +#' @rdname isNaN +#' @name isNaN +#' @family normal_funcs +#' @export +#' @examples \dontrun{isNaN(df$c)} +setMethod("isNaN", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "isNaN", x@jc) + column(jc) + }) + +#' last +#' +#' Aggregate function: returns the last value in a group. +#' +#' @rdname last +#' @name last +#' @family agg_funcs +#' @export +#' @examples \dontrun{last(df$c)} +setMethod("last", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "last", x@jc) + column(jc) + }) + +#' last_day +#' +#' Given a date column, returns the last day of the month which the given date belongs to. +#' For example, input "2015-07-27" returns "2015-07-31" since July 31 is the last day of the +#' month in July 2015. +#' +#' @rdname last_day +#' @name last_day +#' @family datetime_funcs +#' @export +#' @examples \dontrun{last_day(df$c)} +setMethod("last_day", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "last_day", x@jc) + column(jc) + }) + +#' length +#' +#' Computes the length of a given string or binary column. +#' +#' @rdname length +#' @name length +#' @family string_funcs +#' @export +#' @examples \dontrun{length(df$c)} +setMethod("length", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "length", x@jc) + column(jc) + }) + +#' log +#' +#' Computes the natural logarithm of the given value. +#' +#' @rdname log +#' @name log +#' @family math_funcs +#' @export +#' @examples \dontrun{log(df$c)} +setMethod("log", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "log", x@jc) + column(jc) + }) + +#' log10 +#' +#' Computes the logarithm of the given value in base 10. +#' +#' @rdname log10 +#' @name log10 +#' @family math_funcs +#' @export +#' @examples \dontrun{log10(df$c)} +setMethod("log10", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "log10", x@jc) + column(jc) + }) + +#' log1p +#' +#' Computes the natural logarithm of the given value plus one. +#' +#' @rdname log1p +#' @name log1p +#' @family math_funcs +#' @export +#' @examples \dontrun{log1p(df$c)} +setMethod("log1p", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "log1p", x@jc) + column(jc) + }) + +#' log2 +#' +#' Computes the logarithm of the given column in base 2. +#' +#' @rdname log2 +#' @name log2 +#' @family math_funcs +#' @export +#' @examples \dontrun{log2(df$c)} +setMethod("log2", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "log2", x@jc) + column(jc) + }) + +#' lower +#' +#' Converts a string column to lower case. +#' +#' @rdname lower +#' @name lower +#' @family string_funcs +#' @export +#' @examples \dontrun{lower(df$c)} +setMethod("lower", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "lower", x@jc) + column(jc) + }) + +#' ltrim +#' +#' Trim the spaces from left end for the specified string value. +#' +#' @rdname ltrim +#' @name ltrim +#' @family string_funcs +#' @export +#' @examples \dontrun{ltrim(df$c)} +setMethod("ltrim", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "ltrim", x@jc) + column(jc) + }) + +#' max +#' +#' Aggregate function: returns the maximum value of the expression in a group. +#' +#' @rdname max +#' @name max +#' @family agg_funcs +#' @export +#' @examples \dontrun{max(df$c)} +setMethod("max", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "max", x@jc) + column(jc) + }) + +#' md5 +#' +#' Calculates the MD5 digest of a binary column and returns the value +#' as a 32 character hex string. +#' +#' @rdname md5 +#' @name md5 +#' @family misc_funcs +#' @export +#' @examples \dontrun{md5(df$c)} +setMethod("md5", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "md5", x@jc) + column(jc) + }) + +#' mean +#' +#' Aggregate function: returns the average of the values in a group. +#' Alias for avg. +#' +#' @rdname mean +#' @name mean +#' @family agg_funcs +#' @export +#' @examples \dontrun{mean(df$c)} +setMethod("mean", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "mean", x@jc) + column(jc) + }) + +#' min +#' +#' Aggregate function: returns the minimum value of the expression in a group. +#' +#' @rdname min +#' @name min +#' @family agg_funcs +#' @export +#' @examples \dontrun{min(df$c)} +setMethod("min", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "min", x@jc) + column(jc) + }) + +#' minute +#' +#' Extracts the minutes as an integer from a given date/timestamp/string. +#' +#' @rdname minute +#' @name minute +#' @family datetime_funcs +#' @export +#' @examples \dontrun{minute(df$c)} +setMethod("minute", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "minute", x@jc) + column(jc) + }) + +#' month +#' +#' Extracts the month as an integer from a given date/timestamp/string. +#' +#' @rdname month +#' @name month +#' @family datetime_funcs +#' @export +#' @examples \dontrun{month(df$c)} +setMethod("month", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "month", x@jc) + column(jc) + }) + +#' negate +#' +#' Unary minus, i.e. negate the expression. +#' +#' @rdname negate +#' @name negate +#' @family normal_funcs +#' @export +#' @examples \dontrun{negate(df$c)} +setMethod("negate", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "negate", x@jc) + column(jc) + }) -createFunctions <- function() { - for (name in functions1) { - createFunction1(name) - } - for (name in functions2) { - createFunction2(name) - } -} +#' quarter +#' +#' Extracts the quarter as an integer from a given date/timestamp/string. +#' +#' @rdname quarter +#' @name quarter +#' @family datetime_funcs +#' @export +#' @examples \dontrun{quarter(df$c)} +setMethod("quarter", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "quarter", x@jc) + column(jc) + }) -createFunctions() +#' reverse +#' +#' Reverses the string column and returns it as a new string column. +#' +#' @rdname reverse +#' @name reverse +#' @family string_funcs +#' @export +#' @examples \dontrun{reverse(df$c)} +setMethod("reverse", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "reverse", x@jc) + column(jc) + }) -#' @rdname functions -#' @return Creates a Column class of literal value. -setMethod("lit", signature("ANY"), +#' rint +#' +#' Returns the double value that is closest in value to the argument and +#' is equal to a mathematical integer. +#' +#' @rdname rint +#' @name rint +#' @family math_funcs +#' @export +#' @examples \dontrun{rint(df$c)} +setMethod("rint", + signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", - "lit", - ifelse(class(x) == "Column", x@jc, x)) + jc <- callJStatic("org.apache.spark.sql.functions", "rint", x@jc) + column(jc) + }) + +#' round +#' +#' Returns the value of the column `e` rounded to 0 decimal places. +#' +#' @rdname round +#' @name round +#' @family math_funcs +#' @export +#' @examples \dontrun{round(df$c)} +setMethod("round", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "round", x@jc) + column(jc) + }) + +#' rtrim +#' +#' Trim the spaces from right end for the specified string value. +#' +#' @rdname rtrim +#' @name rtrim +#' @family string_funcs +#' @export +#' @examples \dontrun{rtrim(df$c)} +setMethod("rtrim", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "rtrim", x@jc) + column(jc) + }) + +#' second +#' +#' Extracts the seconds as an integer from a given date/timestamp/string. +#' +#' @rdname second +#' @name second +#' @family datetime_funcs +#' @export +#' @examples \dontrun{second(df$c)} +setMethod("second", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "second", x@jc) + column(jc) + }) + +#' sha1 +#' +#' Calculates the SHA-1 digest of a binary column and returns the value +#' as a 40 character hex string. +#' +#' @rdname sha1 +#' @name sha1 +#' @family misc_funcs +#' @export +#' @examples \dontrun{sha1(df$c)} +setMethod("sha1", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sha1", x@jc) + column(jc) + }) + +#' signum +#' +#' Computes the signum of the given value. +#' +#' @rdname signum +#' @name signum +#' @family math_funcs +#' @export +#' @examples \dontrun{signum(df$c)} +setMethod("signum", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "signum", x@jc) + column(jc) + }) + +#' sin +#' +#' Computes the sine of the given value. +#' +#' @rdname sin +#' @name sin +#' @family math_funcs +#' @export +#' @examples \dontrun{sin(df$c)} +setMethod("sin", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sin", x@jc) + column(jc) + }) + +#' sinh +#' +#' Computes the hyperbolic sine of the given value. +#' +#' @rdname sinh +#' @name sinh +#' @family math_funcs +#' @export +#' @examples \dontrun{sinh(df$c)} +setMethod("sinh", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sinh", x@jc) + column(jc) + }) + +#' size +#' +#' Returns length of array or map. +#' +#' @rdname size +#' @name size +#' @family collection_funcs +#' @export +#' @examples \dontrun{size(df$c)} +setMethod("size", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "size", x@jc) + column(jc) + }) + +#' soundex +#' +#' Return the soundex code for the specified expression. +#' +#' @rdname soundex +#' @name soundex +#' @family string_funcs +#' @export +#' @examples \dontrun{soundex(df$c)} +setMethod("soundex", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "soundex", x@jc) + column(jc) + }) + +#' sqrt +#' +#' Computes the square root of the specified float value. +#' +#' @rdname sqrt +#' @name sqrt +#' @family math_funcs +#' @export +#' @examples \dontrun{sqrt(df$c)} +setMethod("sqrt", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sqrt", x@jc) + column(jc) + }) + +#' sum +#' +#' Aggregate function: returns the sum of all values in the expression. +#' +#' @rdname sum +#' @name sum +#' @family agg_funcs +#' @export +#' @examples \dontrun{sum(df$c)} +setMethod("sum", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sum", x@jc) + column(jc) + }) + +#' sumDistinct +#' +#' Aggregate function: returns the sum of distinct values in the expression. +#' +#' @rdname sumDistinct +#' @name sumDistinct +#' @family agg_funcs +#' @export +#' @examples \dontrun{sumDistinct(df$c)} +setMethod("sumDistinct", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "sumDistinct", x@jc) + column(jc) + }) + +#' tan +#' +#' Computes the tangent of the given value. +#' +#' @rdname tan +#' @name tan +#' @family math_funcs +#' @export +#' @examples \dontrun{tan(df$c)} +setMethod("tan", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "tan", x@jc) + column(jc) + }) + +#' tanh +#' +#' Computes the hyperbolic tangent of the given value. +#' +#' @rdname tanh +#' @name tanh +#' @family math_funcs +#' @export +#' @examples \dontrun{tanh(df$c)} +setMethod("tanh", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "tanh", x@jc) + column(jc) + }) + +#' toDegrees +#' +#' Converts an angle measured in radians to an approximately equivalent angle measured in degrees. +#' +#' @rdname toDegrees +#' @name toDegrees +#' @family math_funcs +#' @export +#' @examples \dontrun{toDegrees(df$c)} +setMethod("toDegrees", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "toDegrees", x@jc) + column(jc) + }) + +#' toRadians +#' +#' Converts an angle measured in degrees to an approximately equivalent angle measured in radians. +#' +#' @rdname toRadians +#' @name toRadians +#' @family math_funcs +#' @export +#' @examples \dontrun{toRadians(df$c)} +setMethod("toRadians", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "toRadians", x@jc) + column(jc) + }) + +#' to_date +#' +#' Converts the column into DateType. +#' +#' @rdname to_date +#' @name to_date +#' @family datetime_funcs +#' @export +#' @examples \dontrun{to_date(df$c)} +setMethod("to_date", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "to_date", x@jc) + column(jc) + }) + +#' trim +#' +#' Trim the spaces from both ends for the specified string column. +#' +#' @rdname trim +#' @name trim +#' @family string_funcs +#' @export +#' @examples \dontrun{trim(df$c)} +setMethod("trim", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "trim", x@jc) + column(jc) + }) + +#' unbase64 +#' +#' Decodes a BASE64 encoded string column and returns it as a binary column. +#' This is the reverse of base64. +#' +#' @rdname unbase64 +#' @name unbase64 +#' @family string_funcs +#' @export +#' @examples \dontrun{unbase64(df$c)} +setMethod("unbase64", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "unbase64", x@jc) + column(jc) + }) + +#' unhex +#' +#' Inverse of hex. Interprets each pair of characters as a hexadecimal number +#' and converts to the byte representation of number. +#' +#' @rdname unhex +#' @name unhex +#' @family math_funcs +#' @export +#' @examples \dontrun{unhex(df$c)} +setMethod("unhex", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "unhex", x@jc) + column(jc) + }) + +#' upper +#' +#' Converts a string column to upper case. +#' +#' @rdname upper +#' @name upper +#' @family string_funcs +#' @export +#' @examples \dontrun{upper(df$c)} +setMethod("upper", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "upper", x@jc) + column(jc) + }) + +#' weekofyear +#' +#' Extracts the week number as an integer from a given date/timestamp/string. +#' +#' @rdname weekofyear +#' @name weekofyear +#' @family datetime_funcs +#' @export +#' @examples \dontrun{weekofyear(df$c)} +setMethod("weekofyear", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "weekofyear", x@jc) + column(jc) + }) + +#' year +#' +#' Extracts the year as an integer from a given date/timestamp/string. +#' +#' @rdname year +#' @name year +#' @family datetime_funcs +#' @export +#' @examples \dontrun{year(df$c)} +setMethod("year", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "year", x@jc) column(jc) }) +#' atan2 +#' +#' Returns the angle theta from the conversion of rectangular coordinates (x, y) to +#' polar coordinates (r, theta). +#' +#' @rdname atan2 +#' @name atan2 +#' @family math_funcs +#' @export +#' @examples \dontrun{atan2(df$c, x)} +setMethod("atan2", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "atan2", y@jc, x) + column(jc) + }) + +#' datediff +#' +#' Returns the number of days from `start` to `end`. +#' +#' @rdname datediff +#' @name datediff +#' @family datetime_funcs +#' @export +#' @examples \dontrun{datediff(df$c, x)} +setMethod("datediff", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "datediff", y@jc, x) + column(jc) + }) + +#' hypot +#' +#' Computes `sqrt(a^2^ + b^2^)` without intermediate overflow or underflow. +#' +#' @rdname hypot +#' @name hypot +#' @family math_funcs +#' @export +#' @examples \dontrun{hypot(df$c, x)} +setMethod("hypot", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "hypot", y@jc, x) + column(jc) + }) + +#' levenshtein +#' +#' Computes the Levenshtein distance of the two given string columns. +#' +#' @rdname levenshtein +#' @name levenshtein +#' @family string_funcs +#' @export +#' @examples \dontrun{levenshtein(df$c, x)} +setMethod("levenshtein", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "levenshtein", y@jc, x) + column(jc) + }) + +#' months_between +#' +#' Returns number of months between dates `date1` and `date2`. +#' +#' @rdname months_between +#' @name months_between +#' @family datetime_funcs +#' @export +#' @examples \dontrun{months_between(df$c, x)} +setMethod("months_between", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x) + column(jc) + }) + +#' nanvl +#' +#' Returns col1 if it is not NaN, or col2 if col1 is NaN. +#' hhBoth inputs should be floating point columns (DoubleType or FloatType). +#' +#' @rdname nanvl +#' @name nanvl +#' @family normal_funcs +#' @export +#' @examples \dontrun{nanvl(df$c, x)} +setMethod("nanvl", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "nanvl", y@jc, x) + column(jc) + }) + +#' pmod +#' +#' Returns the positive value of dividend mod divisor. +#' +#' @rdname pmod +#' @name pmod +#' @docType methods +#' @family math_funcs +#' @export +#' @examples \dontrun{pmod(df$c, x)} +setMethod("pmod", signature(y = "Column"), + function(y, x) { + if (class(x) == "Column") { + x <- x@jc + } + jc <- callJStatic("org.apache.spark.sql.functions", "pmod", y@jc, x) + column(jc) + }) + + #' Approx Count Distinct #' -#' @rdname functions +#' @family agg_funcs +#' @rdname approxCountDistinct +#' @name approxCountDistinct #' @return the approximate number of distinct items in a group. +#' @export setMethod("approxCountDistinct", signature(x = "Column"), function(x, rsd = 0.95) { @@ -90,8 +1319,11 @@ setMethod("approxCountDistinct", #' Count Distinct #' -#' @rdname functions +#' @family agg_funcs +#' @rdname countDistinct +#' @name countDistinct #' @return the number of distinct items in a group. +#' @export setMethod("countDistinct", signature(x = "Column"), function(x, ...) { @@ -103,8 +1335,15 @@ setMethod("countDistinct", column(jc) }) -#' @rdname functions -#' @return Concatenates multiple input string columns together into a single string column. + +#' concat +#' +#' Concatenates multiple input string columns together into a single string column. +#' +#' @family string_funcs +#' @rdname concat +#' @name concat +#' @export setMethod("concat", signature(x = "Column"), function(x, ...) { @@ -113,9 +1352,15 @@ setMethod("concat", column(jc) }) -#' @rdname functions -#' @return Returns the greatest value of the list of column names, skipping null values. -#' This function takes at least 2 parameters. It will return null if all parameters are null. +#' greatest +#' +#' Returns the greatest value of the list of column names, skipping null values. +#' This function takes at least 2 parameters. It will return null if all parameters are null. +#' +#' @family normal_funcs +#' @rdname greatest +#' @name greatest +#' @export setMethod("greatest", signature(x = "Column"), function(x, ...) { @@ -125,9 +1370,15 @@ setMethod("greatest", column(jc) }) -#' @rdname functions -#' @return Returns the least value of the list of column names, skipping null values. -#' This function takes at least 2 parameters. It will return null iff all parameters are null. +#' least +#' +#' Returns the least value of the list of column names, skipping null values. +#' This function takes at least 2 parameters. It will return null iff all parameters are null. +#' +#' @family normal_funcs +#' @rdname least +#' @name least +#' @export setMethod("least", signature(x = "Column"), function(x, ...) { @@ -137,30 +1388,58 @@ setMethod("least", column(jc) }) -#' @rdname functions +#' ceiling +#' +#' Computes the ceiling of the given value. +#' +#' @family math_funcs +#' @rdname ceil +#' @name ceil #' @aliases ceil +#' @export setMethod("ceiling", signature(x = "Column"), function(x) { ceil(x) }) -#' @rdname functions +#' sign +#' +#' Computes the signum of the given value. +#' +#' @family math_funcs +#' @rdname signum +#' @name signum #' @aliases signum +#' @export setMethod("sign", signature(x = "Column"), function(x) { signum(x) }) -#' @rdname functions +#' n_distinct +#' +#' Aggregate function: returns the number of distinct items in a group. +#' +#' @family agg_funcs +#' @rdname countDistinct +#' @name countDistinct #' @aliases countDistinct +#' @export setMethod("n_distinct", signature(x = "Column"), function(x, ...) { countDistinct(x, ...) }) -#' @rdname functions +#' n +#' +#' Aggregate function: returns the number of items in a group. +#' +#' @family agg_funcs +#' @rdname count +#' @name count #' @aliases count +#' @export setMethod("n", signature(x = "Column"), function(x) { count(x) @@ -171,13 +1450,16 @@ setMethod("n", signature(x = "Column"), #' Converts a date/timestamp/string to a value of string in the format specified by the date #' format given by the second argument. #' -#' A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All -#' pattern letters of `java.text.SimpleDateFormat` can be used. +#' A pattern could be for instance \preformatted{dd.MM.yyyy} and could return a string like '18.03.1993'. All +#' pattern letters of \code{java.text.SimpleDateFormat} can be used. #' -#' NOTE: Use when ever possible specialized functions like `year`. These benefit from a +#' NOTE: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname date_format +#' @name date_format +#' @export setMethod("date_format", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_format", y@jc, x) @@ -188,7 +1470,10 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' #' Assumes given timestamp is UTC and converts to given timezone. #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname from_utc_timestamp +#' @name from_utc_timestamp +#' @export setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) @@ -203,7 +1488,10 @@ setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), #' NOTE: The position is not zero based, but 1 based index, returns 0 if substr #' could not be found in str. #' -#' @rdname functions +#' @family string_funcs +#' @rdname instr +#' @name instr +#' @export setMethod("instr", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "instr", y@jc, x) @@ -215,13 +1503,16 @@ setMethod("instr", signature(y = "Column", x = "character"), #' Given a date column, returns the first date which is later than the value of the date column #' that is on the specified day of the week. #' -#' For example, `next <- day('2015-07-27', "Sunday")` returns 2015-08-02 because that is the first +#' For example, \code{next_day('2015-07-27', "Sunday")} returns 2015-08-02 because that is the first #' Sunday after 2015-07-27. #' #' Day of the week parameter is case insensitive, and accepts: #' "Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun". #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname next_day +#' @name next_day +#' @export setMethod("next_day", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "next_day", y@jc, x) @@ -232,7 +1523,10 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' #' Assumes given timestamp is in given timezone and converts to UTC. #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname to_utc_timestamp +#' @name to_utc_timestamp +#' @export setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) @@ -243,7 +1537,11 @@ setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), #' #' Returns the date that is numMonths after startDate. #' -#' @rdname functions +#' @name add_months +#' @family datetime_funcs +#' @rdname add_months +#' @name add_months +#' @export setMethod("add_months", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "add_months", y@jc, as.integer(x)) @@ -254,7 +1552,10 @@ setMethod("add_months", signature(y = "Column", x = "numeric"), #' #' Returns the date that is `days` days after `start` #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname date_add +#' @name date_add +#' @export setMethod("date_add", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_add", y@jc, as.integer(x)) @@ -265,7 +1566,10 @@ setMethod("date_add", signature(y = "Column", x = "numeric"), #' #' Returns the date that is `days` days before `start` #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname date_sub +#' @name date_sub +#' @export setMethod("date_sub", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "date_sub", y@jc, as.integer(x)) @@ -280,7 +1584,10 @@ setMethod("date_sub", signature(y = "Column", x = "numeric"), #' If d is 0, the result has no decimal point or fractional part. #' If d < 0, the result will be null.' #' -#' @rdname functions +#' @family string_funcs +#' @rdname format_number +#' @name format_number +#' @export setMethod("format_number", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -294,9 +1601,12 @@ setMethod("format_number", signature(y = "Column", x = "numeric"), #' Calculates the SHA-2 family of hash functions of a binary column and #' returns the value as a hex string. #' -#' @rdname functions #' @param y column to compute SHA-2 on. #' @param x one of 224, 256, 384, or 512. +#' @family misc_funcs +#' @rdname sha2 +#' @name sha2 +#' @export setMethod("sha2", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", "sha2", y@jc, as.integer(x)) @@ -308,7 +1618,10 @@ setMethod("sha2", signature(y = "Column", x = "numeric"), #' Shift the the given value numBits left. If the given value is a long value, this function #' will return a long value else it will return an integer value. #' -#' @rdname functions +#' @family math_funcs +#' @rdname shiftLeft +#' @name shiftLeft +#' @export setMethod("shiftLeft", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -322,7 +1635,10 @@ setMethod("shiftLeft", signature(y = "Column", x = "numeric"), #' Shift the the given value numBits right. If the given value is a long value, it will return #' a long value else it will return an integer value. #' -#' @rdname functions +#' @family math_funcs +#' @rdname shiftRight +#' @name shiftRight +#' @export setMethod("shiftRight", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -336,7 +1652,10 @@ setMethod("shiftRight", signature(y = "Column", x = "numeric"), #' Unsigned shift the the given value numBits right. If the given value is a long value, #' it will return a long value else it will return an integer value. #' -#' @rdname functions +#' @family math_funcs +#' @rdname shiftRightUnsigned +#' @name shiftRightUnsigned +#' @export setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), function(y, x) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -350,7 +1669,10 @@ setMethod("shiftRightUnsigned", signature(y = "Column", x = "numeric"), #' Concatenates multiple input string columns together into a single string column, #' using the given separator. #' -#' @rdname functions +#' @family string_funcs +#' @rdname concat_ws +#' @name concat_ws +#' @export setMethod("concat_ws", signature(sep = "character", x = "Column"), function(sep, x, ...) { jcols <- listToSeq(lapply(list(x, ...), function(x) { x@jc })) @@ -362,7 +1684,10 @@ setMethod("concat_ws", signature(sep = "character", x = "Column"), #' #' Convert a number in a string column from one base to another. #' -#' @rdname functions +#' @family math_funcs +#' @rdname conv +#' @name conv +#' @export setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeric"), function(x, fromBase, toBase) { fromBase <- as.integer(fromBase) @@ -378,7 +1703,10 @@ setMethod("conv", signature(x = "Column", fromBase = "numeric", toBase = "numeri #' Parses the expression string into the column that it represents, similar to #' DataFrame.selectExpr #' -#' @rdname functions +#' @family normal_funcs +#' @rdname expr +#' @name expr +#' @export setMethod("expr", signature(x = "character"), function(x) { jc <- callJStatic("org.apache.spark.sql.functions", "expr", x) @@ -389,7 +1717,10 @@ setMethod("expr", signature(x = "character"), #' #' Formats the arguments in printf-style and returns the result as a string column. #' -#' @rdname functions +#' @family string_funcs +#' @rdname format_string +#' @name format_string +#' @export setMethod("format_string", signature(format = "character", x = "Column"), function(format, x, ...) { jcols <- listToSeq(lapply(list(x, ...), function(arg) { arg@jc })) @@ -405,7 +1736,10 @@ setMethod("format_string", signature(format = "character", x = "Column"), #' representing the timestamp of that moment in the current system time zone in the given #' format. #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname from_unixtime +#' @name from_unixtime +#' @export setMethod("from_unixtime", signature(x = "Column"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", @@ -420,7 +1754,10 @@ setMethod("from_unixtime", signature(x = "Column"), #' NOTE: The position is not zero based, but 1 based index, returns 0 if substr #' could not be found in str. #' -#' @rdname functions +#' @family string_funcs +#' @rdname locate +#' @name locate +#' @export setMethod("locate", signature(substr = "character", str = "Column"), function(substr, str, pos = 0) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -433,7 +1770,10 @@ setMethod("locate", signature(substr = "character", str = "Column"), #' #' Left-pad the string column with #' -#' @rdname functions +#' @family string_funcs +#' @rdname lpad +#' @name lpad +#' @export setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -446,12 +1786,19 @@ setMethod("lpad", signature(x = "Column", len = "numeric", pad = "character"), #' #' Generate a random column with i.i.d. samples from U[0.0, 1.0]. #' -#' @rdname functions +#' @family normal_funcs +#' @rdname rand +#' @name rand +#' @export setMethod("rand", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand") column(jc) }) +#' @family normal_funcs +#' @rdname rand +#' @name rand +#' @export setMethod("rand", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "rand", as.integer(seed)) @@ -462,12 +1809,19 @@ setMethod("rand", signature(seed = "numeric"), #' #' Generate a column with i.i.d. samples from the standard normal distribution. #' -#' @rdname functions +#' @family normal_funcs +#' @rdname randn +#' @name randn +#' @export setMethod("randn", signature(seed = "missing"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn") column(jc) }) +#' @family normal_funcs +#' @rdname randn +#' @name randn +#' @export setMethod("randn", signature(seed = "numeric"), function(seed) { jc <- callJStatic("org.apache.spark.sql.functions", "randn", as.integer(seed)) @@ -478,7 +1832,10 @@ setMethod("randn", signature(seed = "numeric"), #' #' Extract a specific(idx) group identified by a java regex, from the specified string column. #' -#' @rdname functions +#' @family string_funcs +#' @rdname regexp_extract +#' @name regexp_extract +#' @export setMethod("regexp_extract", signature(x = "Column", pattern = "character", idx = "numeric"), function(x, pattern, idx) { @@ -492,7 +1849,10 @@ setMethod("regexp_extract", #' #' Replace all substrings of the specified string value that match regexp with rep. #' -#' @rdname functions +#' @family string_funcs +#' @rdname regexp_replace +#' @name regexp_replace +#' @export setMethod("regexp_replace", signature(x = "Column", pattern = "character", replacement = "character"), function(x, pattern, replacement) { @@ -506,7 +1866,10 @@ setMethod("regexp_replace", #' #' Right-padded with pad to a length of len. #' -#' @rdname functions +#' @family string_funcs +#' @rdname rpad +#' @name rpad +#' @export setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), function(x, len, pad) { jc <- callJStatic("org.apache.spark.sql.functions", @@ -522,7 +1885,10 @@ setMethod("rpad", signature(x = "Column", len = "numeric", pad = "character"), #' returned. If count is negative, every to the right of the final delimiter (counting from the #' right) is returned. substring <- index performs a case-sensitive match when searching for delim. #' -#' @rdname functions +#' @family string_funcs +#' @rdname substring_index +#' @name substring_index +#' @export setMethod("substring_index", signature(x = "Column", delim = "character", count = "numeric"), function(x, delim, count) { @@ -539,7 +1905,10 @@ setMethod("substring_index", #' The translate will happen when any character in the string matching with the character #' in the matchingString. #' -#' @rdname functions +#' @family string_funcs +#' @rdname translate +#' @name translate +#' @export setMethod("translate", signature(x = "Column", matchingString = "character", replaceString = "character"), function(x, matchingString, replaceString) { @@ -552,30 +1921,28 @@ setMethod("translate", #' #' Gets current Unix timestamp in seconds. #' -#' @rdname functions +#' @family datetime_funcs +#' @rdname unix_timestamp +#' @name unix_timestamp +#' @export setMethod("unix_timestamp", signature(x = "missing", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp") column(jc) }) -#' unix_timestamp -#' -#' Converts time string in format yyyy-MM-dd HH:mm:ss to Unix timestamp (in seconds), -#' using the default timezone and the default locale, return null if fail. -#' -#' @rdname functions +#' @family datetime_funcs +#' @rdname unix_timestamp +#' @name unix_timestamp +#' @export setMethod("unix_timestamp", signature(x = "Column", format = "missing"), function(x, format) { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc) column(jc) }) -#' unix_timestamp -#' -#' Convert time string with given pattern -#' (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) -#' to Unix time stamp (in seconds), return null if fail. -#' -#' @rdname functions +#' @family datetime_funcs +#' @rdname unix_timestamp +#' @name unix_timestamp +#' @export setMethod("unix_timestamp", signature(x = "Column", format = "character"), function(x, format = "yyyy-MM-dd HH:mm:ss") { jc <- callJStatic("org.apache.spark.sql.functions", "unix_timestamp", x@jc, format) @@ -586,7 +1953,10 @@ setMethod("unix_timestamp", signature(x = "Column", format = "character"), #' Evaluates a list of conditions and returns one of multiple possible result expressions. #' For unmatched expressions null is returned. #' -#' @rdname column +#' @family normal_funcs +#' @rdname when +#' @name when +#' @export setMethod("when", signature(condition = "Column", value = "ANY"), function(condition, value) { condition <- condition@jc @@ -597,10 +1967,13 @@ setMethod("when", signature(condition = "Column", value = "ANY"), #' ifelse #' -#' Evaluates a list of conditions and returns `yes` if the conditions are satisfied. -#' Otherwise `no` is returned for unmatched conditions. +#' Evaluates a list of conditions and returns \code{yes} if the conditions are satisfied. +#' Otherwise \code{no} is returned for unmatched conditions. #' -#' @rdname column +#' @family normal_funcs +#' @rdname ifelse +#' @name ifelse +#' @export setMethod("ifelse", signature(test = "Column", yes = "ANY", no = "ANY"), function(test, yes, no) { diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 84cb8dfdaa2dd..610a8c31223cd 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -567,10 +567,6 @@ setGeneric("withColumnRenamed", ###################### Column Methods ########################## -#' @rdname column -#' @export -setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) - #' @rdname column #' @export setGeneric("asc", function(x) { standardGeneric("asc") }) @@ -587,10 +583,6 @@ setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) #' @export setGeneric("contains", function(x, ...) { standardGeneric("contains") }) -#' @rdname column -#' @export -setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) - #' @rdname column #' @export setGeneric("desc", function(x) { standardGeneric("desc") }) @@ -607,10 +599,6 @@ setGeneric("getField", function(x, ...) { standardGeneric("getField") }) #' @export setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) -#' @rdname column -#' @export -setGeneric("hypot", function(y, x) { standardGeneric("hypot") }) - #' @rdname column #' @export setGeneric("isNull", function(x) { standardGeneric("isNull") }) @@ -619,30 +607,10 @@ setGeneric("isNull", function(x) { standardGeneric("isNull") }) #' @export setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) -#' @rdname column -#' @export -setGeneric("last", function(x) { standardGeneric("last") }) - #' @rdname column #' @export setGeneric("like", function(x, ...) { standardGeneric("like") }) -#' @rdname column -#' @export -setGeneric("lower", function(x) { standardGeneric("lower") }) - -#' @rdname column -#' @export -setGeneric("n", function(x) { standardGeneric("n") }) - -#' @rdname column -#' @export -setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) - -#' @rdname column -#' @export -setGeneric("rint", function(x, ...) { standardGeneric("rint") }) - #' @rdname column #' @export setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) @@ -662,312 +630,340 @@ setGeneric("otherwise", function(x, value) { standardGeneric("otherwise") }) ###################### Expression Function Methods ########################## -#' @rdname functions +#' @rdname add_months #' @export setGeneric("add_months", function(y, x) { standardGeneric("add_months") }) -#' @rdname functions +#' @rdname approxCountDistinct +#' @export +setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) + +#' @rdname ascii #' @export setGeneric("ascii", function(x) { standardGeneric("ascii") }) -#' @rdname functions +#' @rdname avg #' @export setGeneric("avg", function(x, ...) { standardGeneric("avg") }) -#' @rdname functions +#' @rdname base64 #' @export setGeneric("base64", function(x) { standardGeneric("base64") }) -#' @rdname functions +#' @rdname bin #' @export setGeneric("bin", function(x) { standardGeneric("bin") }) -#' @rdname functions +#' @rdname bitwiseNOT #' @export setGeneric("bitwiseNOT", function(x) { standardGeneric("bitwiseNOT") }) -#' @rdname functions +#' @rdname cbrt #' @export setGeneric("cbrt", function(x) { standardGeneric("cbrt") }) -#' @rdname functions +#' @rdname ceil #' @export setGeneric("ceil", function(x) { standardGeneric("ceil") }) -#' @rdname functions +#' @rdname concat #' @export setGeneric("concat", function(x, ...) { standardGeneric("concat") }) -#' @rdname functions +#' @rdname concat_ws #' @export setGeneric("concat_ws", function(sep, x, ...) { standardGeneric("concat_ws") }) -#' @rdname functions +#' @rdname conv #' @export setGeneric("conv", function(x, fromBase, toBase) { standardGeneric("conv") }) -#' @rdname functions +#' @rdname countDistinct +#' @export +setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) + +#' @rdname crc32 #' @export setGeneric("crc32", function(x) { standardGeneric("crc32") }) -#' @rdname functions +#' @rdname datediff #' @export setGeneric("datediff", function(y, x) { standardGeneric("datediff") }) -#' @rdname functions +#' @rdname date_add #' @export setGeneric("date_add", function(y, x) { standardGeneric("date_add") }) -#' @rdname functions +#' @rdname date_format #' @export setGeneric("date_format", function(y, x) { standardGeneric("date_format") }) -#' @rdname functions +#' @rdname date_sub #' @export setGeneric("date_sub", function(y, x) { standardGeneric("date_sub") }) -#' @rdname functions +#' @rdname dayofmonth #' @export setGeneric("dayofmonth", function(x) { standardGeneric("dayofmonth") }) -#' @rdname functions +#' @rdname dayofyear #' @export setGeneric("dayofyear", function(x) { standardGeneric("dayofyear") }) -#' @rdname functions +#' @rdname explode #' @export setGeneric("explode", function(x) { standardGeneric("explode") }) -#' @rdname functions +#' @rdname expr #' @export setGeneric("expr", function(x) { standardGeneric("expr") }) -#' @rdname functions +#' @rdname from_utc_timestamp #' @export setGeneric("from_utc_timestamp", function(y, x) { standardGeneric("from_utc_timestamp") }) -#' @rdname functions +#' @rdname format_number #' @export setGeneric("format_number", function(y, x) { standardGeneric("format_number") }) -#' @rdname functions +#' @rdname format_string #' @export setGeneric("format_string", function(format, x, ...) { standardGeneric("format_string") }) -#' @rdname functions +#' @rdname from_unixtime #' @export setGeneric("from_unixtime", function(x, ...) { standardGeneric("from_unixtime") }) -#' @rdname functions +#' @rdname greatest #' @export setGeneric("greatest", function(x, ...) { standardGeneric("greatest") }) -#' @rdname functions +#' @rdname hex #' @export setGeneric("hex", function(x) { standardGeneric("hex") }) -#' @rdname functions +#' @rdname hour #' @export setGeneric("hour", function(x) { standardGeneric("hour") }) -#' @rdname functions +#' @rdname hypot +#' @export +setGeneric("hypot", function(y, x) { standardGeneric("hypot") }) + +#' @rdname initcap #' @export setGeneric("initcap", function(x) { standardGeneric("initcap") }) -#' @rdname functions +#' @rdname instr #' @export setGeneric("instr", function(y, x) { standardGeneric("instr") }) -#' @rdname functions +#' @rdname isNaN #' @export setGeneric("isNaN", function(x) { standardGeneric("isNaN") }) -#' @rdname functions +#' @rdname last +#' @export +setGeneric("last", function(x) { standardGeneric("last") }) + +#' @rdname last_day #' @export setGeneric("last_day", function(x) { standardGeneric("last_day") }) -#' @rdname functions +#' @rdname least #' @export setGeneric("least", function(x, ...) { standardGeneric("least") }) -#' @rdname functions +#' @rdname levenshtein #' @export setGeneric("levenshtein", function(y, x) { standardGeneric("levenshtein") }) -#' @rdname functions +#' @rdname lit #' @export setGeneric("lit", function(x) { standardGeneric("lit") }) -#' @rdname functions +#' @rdname locate #' @export setGeneric("locate", function(substr, str, ...) { standardGeneric("locate") }) -#' @rdname functions +#' @rdname lower #' @export setGeneric("lower", function(x) { standardGeneric("lower") }) -#' @rdname functions +#' @rdname lpad #' @export setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) -#' @rdname functions +#' @rdname ltrim #' @export setGeneric("ltrim", function(x) { standardGeneric("ltrim") }) -#' @rdname functions +#' @rdname md5 #' @export setGeneric("md5", function(x) { standardGeneric("md5") }) -#' @rdname functions +#' @rdname minute #' @export setGeneric("minute", function(x) { standardGeneric("minute") }) -#' @rdname functions +#' @rdname month #' @export setGeneric("month", function(x) { standardGeneric("month") }) -#' @rdname functions +#' @rdname months_between #' @export setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) -#' @rdname functions +#' @rdname count +#' @export +setGeneric("n", function(x) { standardGeneric("n") }) + +#' @rdname nanvl #' @export setGeneric("nanvl", function(y, x) { standardGeneric("nanvl") }) -#' @rdname functions +#' @rdname negate #' @export setGeneric("negate", function(x) { standardGeneric("negate") }) -#' @rdname functions +#' @rdname next_day #' @export setGeneric("next_day", function(y, x) { standardGeneric("next_day") }) -#' @rdname functions +#' @rdname countDistinct +#' @export +setGeneric("n_distinct", function(x, ...) { standardGeneric("n_distinct") }) + +#' @rdname pmod #' @export setGeneric("pmod", function(y, x) { standardGeneric("pmod") }) -#' @rdname functions +#' @rdname quarter #' @export setGeneric("quarter", function(x) { standardGeneric("quarter") }) -#' @rdname functions +#' @rdname rand #' @export setGeneric("rand", function(seed) { standardGeneric("rand") }) -#' @rdname functions +#' @rdname randn #' @export setGeneric("randn", function(seed) { standardGeneric("randn") }) -#' @rdname functions +#' @rdname regexp_extract #' @export setGeneric("regexp_extract", function(x, pattern, idx) { standardGeneric("regexp_extract") }) -#' @rdname functions +#' @rdname regexp_replace #' @export setGeneric("regexp_replace", function(x, pattern, replacement) { standardGeneric("regexp_replace") }) -#' @rdname functions +#' @rdname reverse #' @export setGeneric("reverse", function(x) { standardGeneric("reverse") }) -#' @rdname functions +#' @rdname rint +#' @export +setGeneric("rint", function(x, ...) { standardGeneric("rint") }) + +#' @rdname rpad #' @export setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) -#' @rdname functions +#' @rdname rtrim #' @export setGeneric("rtrim", function(x) { standardGeneric("rtrim") }) -#' @rdname functions +#' @rdname second #' @export setGeneric("second", function(x) { standardGeneric("second") }) -#' @rdname functions +#' @rdname sha1 #' @export setGeneric("sha1", function(x) { standardGeneric("sha1") }) -#' @rdname functions +#' @rdname sha2 #' @export setGeneric("sha2", function(y, x) { standardGeneric("sha2") }) -#' @rdname functions +#' @rdname shiftLeft #' @export setGeneric("shiftLeft", function(y, x) { standardGeneric("shiftLeft") }) -#' @rdname functions +#' @rdname shiftRight #' @export setGeneric("shiftRight", function(y, x) { standardGeneric("shiftRight") }) -#' @rdname functions +#' @rdname shiftRightUnsigned #' @export setGeneric("shiftRightUnsigned", function(y, x) { standardGeneric("shiftRightUnsigned") }) -#' @rdname functions +#' @rdname signum #' @export setGeneric("signum", function(x) { standardGeneric("signum") }) -#' @rdname functions +#' @rdname size #' @export setGeneric("size", function(x) { standardGeneric("size") }) -#' @rdname functions +#' @rdname soundex #' @export setGeneric("soundex", function(x) { standardGeneric("soundex") }) -#' @rdname functions +#' @rdname substring_index #' @export setGeneric("substring_index", function(x, delim, count) { standardGeneric("substring_index") }) -#' @rdname functions +#' @rdname sumDistinct #' @export setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) -#' @rdname functions +#' @rdname toDegrees #' @export setGeneric("toDegrees", function(x) { standardGeneric("toDegrees") }) -#' @rdname functions +#' @rdname toRadians #' @export setGeneric("toRadians", function(x) { standardGeneric("toRadians") }) -#' @rdname functions +#' @rdname to_date #' @export setGeneric("to_date", function(x) { standardGeneric("to_date") }) -#' @rdname functions +#' @rdname to_utc_timestamp #' @export setGeneric("to_utc_timestamp", function(y, x) { standardGeneric("to_utc_timestamp") }) -#' @rdname functions +#' @rdname translate #' @export setGeneric("translate", function(x, matchingString, replaceString) { standardGeneric("translate") }) -#' @rdname functions +#' @rdname trim #' @export setGeneric("trim", function(x) { standardGeneric("trim") }) -#' @rdname functions +#' @rdname unbase64 #' @export setGeneric("unbase64", function(x) { standardGeneric("unbase64") }) -#' @rdname functions +#' @rdname unhex #' @export setGeneric("unhex", function(x) { standardGeneric("unhex") }) -#' @rdname functions +#' @rdname unix_timestamp #' @export setGeneric("unix_timestamp", function(x, format) { standardGeneric("unix_timestamp") }) -#' @rdname functions +#' @rdname upper #' @export setGeneric("upper", function(x) { standardGeneric("upper") }) -#' @rdname functions +#' @rdname weekofyear #' @export setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) -#' @rdname functions +#' @rdname year #' @export setGeneric("year", function(x) { standardGeneric("year") }) From 642c43c81c835139e3f35dfd6a215d668a474203 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Mon, 24 Aug 2015 19:45:41 -0700 Subject: [PATCH 064/232] [SQL] [MINOR] [DOC] Clarify docs for inferring DataFrame from RDD of Products * Makes `SQLImplicits.rddToDataFrameHolder` scaladoc consistent with `SQLContext.createDataFrame[A <: Product](rdd: RDD[A])` since the former is essentially a wrapper for the latter * Clarifies `createDataFrame[A <: Product]` scaladoc to apply for any `RDD[Product]`, not just case classes Author: Feynman Liang Closes #8406 from feynmanliang/sql-doc-fixes. --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 126c9c6f839c7..a1eea09e0477b 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 @@ -350,7 +350,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: - * Creates a DataFrame from an RDD of case classes. + * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). * * @group dataframes * @since 1.3.0 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 47b6f80bed483..bf03c61088426 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -40,7 +40,7 @@ private[sql] abstract class SQLImplicits { implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) /** - * Creates a DataFrame from an RDD of case classes or tuples. + * Creates a DataFrame from an RDD of Product (e.g. case classes, tuples). * @since 1.3.0 */ implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = { From a0c0aae1defe5e1e57704065631d201f8e3f6bac Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 25 Aug 2015 12:49:50 +0800 Subject: [PATCH 065/232] [SPARK-10121] [SQL] Thrift server always use the latest class loader provided by the conf of executionHive's state https://issues.apache.org/jira/browse/SPARK-10121 Looks like the problem is that if we add a jar through another thread, the thread handling the JDBC session will not get the latest classloader. Author: Yin Huai Closes #8368 from yhuai/SPARK-10121. --- .../SparkExecuteStatementOperation.scala | 6 +++ .../HiveThriftServer2Suites.scala | 54 +++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 833bf62d47d07..02cc7e5efa521 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -159,6 +159,12 @@ private[hive] class SparkExecuteStatementOperation( // User information is part of the metastore client member in Hive hiveContext.setSession(currentSqlSession) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = + hiveContext.executionHive.state.getConf.getClassLoader + sessionHive.getConf.setClassLoader(executionHiveClassLoader) + parentSessionState.getConf.setClassLoader(executionHiveClassLoader) + Hive.set(sessionHive) SessionState.setCurrentSessionState(parentSessionState) try { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ded42bca9971e..b72249b3bf8c0 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -377,6 +377,60 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { rs2.close() } } + + test("test add jar") { + withMultipleConnectionJdbcStatement( + { + statement => + val jarFile = + "../hive/src/test/resources/hive-hcatalog-core-0.13.1.jar" + .split("/") + .mkString(File.separator) + + statement.executeQuery(s"ADD JAR $jarFile") + }, + + { + statement => + val queries = Seq( + "DROP TABLE IF EXISTS smallKV", + "CREATE TABLE smallKV(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE smallKV", + "DROP TABLE IF EXISTS addJar", + """CREATE TABLE addJar(key string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + """.stripMargin) + + queries.foreach(statement.execute) + + statement.executeQuery( + """ + |INSERT INTO TABLE addJar SELECT 'k1' as key FROM smallKV limit 1 + """.stripMargin) + + val actualResult = + statement.executeQuery("SELECT key FROM addJar") + val actualResultBuffer = new collection.mutable.ArrayBuffer[String]() + while (actualResult.next()) { + actualResultBuffer += actualResult.getString(1) + } + actualResult.close() + + val expectedResult = + statement.executeQuery("SELECT 'k1'") + val expectedResultBuffer = new collection.mutable.ArrayBuffer[String]() + while (expectedResult.next()) { + expectedResultBuffer += expectedResult.getString(1) + } + expectedResult.close() + + assert(expectedResultBuffer === actualResultBuffer) + + statement.executeQuery("DROP TABLE IF EXISTS addJar") + statement.executeQuery("DROP TABLE IF EXISTS smallKV") + } + ) + } } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { From 5175ca0c85b10045d12c3fb57b1e52278a413ecf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Aug 2015 23:15:27 -0700 Subject: [PATCH 066/232] [SPARK-10178] [SQL] HiveComparisionTest should print out dependent tables In `HiveComparisionTest`s it is possible to fail a query of the form `SELECT * FROM dest1`, where `dest1` is the query that is actually computing the incorrect results. To aid debugging this patch improves the harness to also print these query plans and their results. Author: Michael Armbrust Closes #8388 from marmbrus/generatedTables. --- .../hive/execution/HiveComparisonTest.scala | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 2bdb0e11878e5..4d45249d9c6b8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.execution import java.io._ +import scala.util.control.NonFatal + import org.scalatest.{BeforeAndAfterAll, GivenWhenThen} import org.apache.spark.{Logging, SparkFunSuite} @@ -386,11 +388,45 @@ abstract class HiveComparisonTest hiveCacheFiles.foreach(_.delete()) } + // If this query is reading other tables that were created during this test run + // also print out the query plans and results for those. + val computedTablesMessages: String = try { + val tablesRead = new TestHive.QueryExecution(query).executedPlan.collect { + case ts: HiveTableScan => ts.relation.tableName + }.toSet + + TestHive.reset() + val executions = queryList.map(new TestHive.QueryExecution(_)) + executions.foreach(_.toRdd) + val tablesGenerated = queryList.zip(executions).flatMap { + case (q, e) => e.executedPlan.collect { + case i: InsertIntoHiveTable if tablesRead contains i.table.tableName => + (q, e, i) + } + } + + tablesGenerated.map { case (hiveql, execution, insert) => + s""" + |=== Generated Table === + |$hiveql + |$execution + |== Results == + |${insert.child.execute().collect().mkString("\n")} + """.stripMargin + }.mkString("\n") + + } catch { + case NonFatal(e) => + logError("Failed to compute generated tables", e) + s"Couldn't compute dependent tables: $e" + } + val errorMessage = s""" |Results do not match for $testCaseName: |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} |$resultComparison + |$computedTablesMessages """.stripMargin stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) From d9c25dec87e6da7d66a47ff94e7eefa008081b9d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 24 Aug 2015 23:26:14 -0700 Subject: [PATCH 067/232] =?UTF-8?q?[SPARK-9786]=20[STREAMING]=20[KAFKA]=20?= =?UTF-8?q?fix=20backpressure=20so=20it=20works=20with=20defa=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …ult maxRatePerPartition setting of 0 Author: cody koeninger Closes #8413 from koeninger/backpressure-testing-master. --- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 8a177077775c6..1000094e93cb3 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -95,8 +95,13 @@ class DirectKafkaInputDStream[ val effectiveRateLimitPerPartition = estimatedRateLimit .filter(_ > 0) - .map(limit => Math.min(maxRateLimitPerPartition, (limit / numPartitions))) - .getOrElse(maxRateLimitPerPartition) + .map { limit => + if (maxRateLimitPerPartition > 0) { + Math.min(maxRateLimitPerPartition, (limit / numPartitions)) + } else { + limit / numPartitions + } + }.getOrElse(maxRateLimitPerPartition) if (effectiveRateLimitPerPartition > 0) { val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 From f023aa2fcc1d1dbb82aee568be0a8f2457c309ae Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 24 Aug 2015 23:34:50 -0700 Subject: [PATCH 068/232] [SPARK-10137] [STREAMING] Avoid to restart receivers if scheduleReceivers returns balanced results This PR fixes the following cases for `ReceiverSchedulingPolicy`. 1) Assume there are 4 executors: host1, host2, host3, host4, and 5 receivers: r1, r2, r3, r4, r5. Then `ReceiverSchedulingPolicy.scheduleReceivers` will return (r1 -> host1, r2 -> host2, r3 -> host3, r4 -> host4, r5 -> host1). Let's assume r1 starts at first on `host1` as `scheduleReceivers` suggested, and try to register with ReceiverTracker. But the previous `ReceiverSchedulingPolicy.rescheduleReceiver` will return (host2, host3, host4) according to the current executor weights (host1 -> 1.0, host2 -> 0.5, host3 -> 0.5, host4 -> 0.5), so ReceiverTracker will reject `r1`. This is unexpected since r1 is starting exactly where `scheduleReceivers` suggested. This case can be fixed by ignoring the information of the receiver that is rescheduling in `receiverTrackingInfoMap`. 2) Assume there are 3 executors (host1, host2, host3) and each executors has 3 cores, and 3 receivers: r1, r2, r3. Assume r1 is running on host1. Now r2 is restarting, the previous `ReceiverSchedulingPolicy.rescheduleReceiver` will always return (host1, host2, host3). So it's possible that r2 will be scheduled to host1 by TaskScheduler. r3 is similar. Then at last, it's possible that there are 3 receivers running on host1, while host2 and host3 are idle. This issue can be fixed by returning only executors that have the minimum wight rather than returning at least 3 executors. Author: zsxwing Closes #8340 from zsxwing/fix-receiver-scheduling. --- .../scheduler/ReceiverSchedulingPolicy.scala | 58 +++++++--- .../streaming/scheduler/ReceiverTracker.scala | 106 ++++++++++++------ .../ReceiverSchedulingPolicySuite.scala | 13 ++- 3 files changed, 120 insertions(+), 57 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala index ef5b687b5831a..10b5a7f57a802 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicy.scala @@ -22,6 +22,36 @@ import scala.collection.mutable import org.apache.spark.streaming.receiver.Receiver +/** + * A class that tries to schedule receivers with evenly distributed. There are two phases for + * scheduling receivers. + * + * - The first phase is global scheduling when ReceiverTracker is starting and we need to schedule + * all receivers at the same time. ReceiverTracker will call `scheduleReceivers` at this phase. + * It will try to schedule receivers with evenly distributed. ReceiverTracker should update its + * receiverTrackingInfoMap according to the results of `scheduleReceivers`. + * `ReceiverTrackingInfo.scheduledExecutors` for each receiver will set to an executor list that + * contains the scheduled locations. Then when a receiver is starting, it will send a register + * request and `ReceiverTracker.registerReceiver` will be called. In + * `ReceiverTracker.registerReceiver`, if a receiver's scheduled executors is set, it should check + * if the location of this receiver is one of the scheduled executors, if not, the register will + * be rejected. + * - The second phase is local scheduling when a receiver is restarting. There are two cases of + * receiver restarting: + * - If a receiver is restarting because it's rejected due to the real location and the scheduled + * executors mismatching, in other words, it fails to start in one of the locations that + * `scheduleReceivers` suggested, `ReceiverTracker` should firstly choose the executors that are + * still alive in the list of scheduled executors, then use them to launch the receiver job. + * - If a receiver is restarting without a scheduled executors list, or the executors in the list + * are dead, `ReceiverTracker` should call `rescheduleReceiver`. If so, `ReceiverTracker` should + * not set `ReceiverTrackingInfo.scheduledExecutors` for this executor, instead, it should clear + * it. Then when this receiver is registering, we can know this is a local scheduling, and + * `ReceiverTrackingInfo` should call `rescheduleReceiver` again to check if the launching + * location is matching. + * + * In conclusion, we should make a global schedule, try to achieve that exactly as long as possible, + * otherwise do local scheduling. + */ private[streaming] class ReceiverSchedulingPolicy { /** @@ -102,8 +132,7 @@ private[streaming] class ReceiverSchedulingPolicy { /** * Return a list of candidate executors to run the receiver. If the list is empty, the caller can - * run this receiver in arbitrary executor. The caller can use `preferredNumExecutors` to require - * returning `preferredNumExecutors` executors if possible. + * run this receiver in arbitrary executor. * * This method tries to balance executors' load. Here is the approach to schedule executors * for a receiver. @@ -122,9 +151,8 @@ private[streaming] class ReceiverSchedulingPolicy { * If a receiver is scheduled to an executor but has not yet run, it contributes * `1.0 / #candidate_executors_of_this_receiver` to the executor's weight. * - * At last, if there are more than `preferredNumExecutors` idle executors (weight = 0), - * returns all idle executors. Otherwise, we only return `preferredNumExecutors` best options - * according to the weights. + * At last, if there are any idle executors (weight = 0), returns all idle executors. + * Otherwise, returns the executors that have the minimum weight. * * * @@ -134,8 +162,7 @@ private[streaming] class ReceiverSchedulingPolicy { receiverId: Int, preferredLocation: Option[String], receiverTrackingInfoMap: Map[Int, ReceiverTrackingInfo], - executors: Seq[String], - preferredNumExecutors: Int = 3): Seq[String] = { + executors: Seq[String]): Seq[String] = { if (executors.isEmpty) { return Seq.empty } @@ -156,15 +183,18 @@ private[streaming] class ReceiverSchedulingPolicy { } }.groupBy(_._1).mapValues(_.map(_._2).sum) // Sum weights for each executor - val idleExecutors = (executors.toSet -- executorWeights.keys).toSeq - if (idleExecutors.size >= preferredNumExecutors) { - // If there are more than `preferredNumExecutors` idle executors, return all of them + val idleExecutors = executors.toSet -- executorWeights.keys + if (idleExecutors.nonEmpty) { scheduledExecutors ++= idleExecutors } else { - // If there are less than `preferredNumExecutors` idle executors, return 3 best options - scheduledExecutors ++= idleExecutors - val sortedExecutors = executorWeights.toSeq.sortBy(_._2).map(_._1) - scheduledExecutors ++= (idleExecutors ++ sortedExecutors).take(preferredNumExecutors) + // There is no idle executor. So select all executors that have the minimum weight. + val sortedExecutors = executorWeights.toSeq.sortBy(_._2) + if (sortedExecutors.nonEmpty) { + val minWeight = sortedExecutors(0)._2 + scheduledExecutors ++= sortedExecutors.takeWhile(_._2 == minWeight).map(_._1) + } else { + // This should not happen since "executors" is not empty + } } scheduledExecutors.toSeq } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 30d25a64e307a..3d532a675db02 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -244,8 +244,21 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } if (isTrackerStopping || isTrackerStopped) { - false - } else if (!scheduleReceiver(streamId).contains(hostPort)) { + return false + } + + val scheduledExecutors = receiverTrackingInfos(streamId).scheduledExecutors + val accetableExecutors = if (scheduledExecutors.nonEmpty) { + // This receiver is registering and it's scheduled by + // ReceiverSchedulingPolicy.scheduleReceivers. So use "scheduledExecutors" to check it. + scheduledExecutors.get + } else { + // This receiver is scheduled by "ReceiverSchedulingPolicy.rescheduleReceiver", so calling + // "ReceiverSchedulingPolicy.rescheduleReceiver" again to check it. + scheduleReceiver(streamId) + } + + if (!accetableExecutors.contains(hostPort)) { // Refuse it since it's scheduled to a wrong executor false } else { @@ -426,12 +439,25 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false startReceiver(receiver, executors) } case RestartReceiver(receiver) => - val scheduledExecutors = schedulingPolicy.rescheduleReceiver( - receiver.streamId, - receiver.preferredLocation, - receiverTrackingInfos, - getExecutors) - updateReceiverScheduledExecutors(receiver.streamId, scheduledExecutors) + // Old scheduled executors minus the ones that are not active any more + val oldScheduledExecutors = getStoredScheduledExecutors(receiver.streamId) + val scheduledExecutors = if (oldScheduledExecutors.nonEmpty) { + // Try global scheduling again + oldScheduledExecutors + } else { + val oldReceiverInfo = receiverTrackingInfos(receiver.streamId) + // Clear "scheduledExecutors" to indicate we are going to do local scheduling + val newReceiverInfo = oldReceiverInfo.copy( + state = ReceiverState.INACTIVE, scheduledExecutors = None) + receiverTrackingInfos(receiver.streamId) = newReceiverInfo + schedulingPolicy.rescheduleReceiver( + receiver.streamId, + receiver.preferredLocation, + receiverTrackingInfos, + getExecutors) + } + // Assume there is one receiver restarting at one time, so we don't need to update + // receiverTrackingInfos startReceiver(receiver, scheduledExecutors) case c: CleanupOldBlocks => receiverTrackingInfos.values.flatMap(_.endpoint).foreach(_.send(c)) @@ -464,6 +490,24 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(true) } + /** + * Return the stored scheduled executors that are still alive. + */ + private def getStoredScheduledExecutors(receiverId: Int): Seq[String] = { + if (receiverTrackingInfos.contains(receiverId)) { + val scheduledExecutors = receiverTrackingInfos(receiverId).scheduledExecutors + if (scheduledExecutors.nonEmpty) { + val executors = getExecutors.toSet + // Only return the alive executors + scheduledExecutors.get.filter(executors) + } else { + Nil + } + } else { + Nil + } + } + /** * Start a receiver along with its scheduled executors */ @@ -484,7 +528,23 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false new SerializableConfiguration(ssc.sparkContext.hadoopConfiguration) // Function to start the receiver on the worker node - val startReceiverFunc = new StartReceiverFunc(checkpointDirOption, serializableHadoopConf) + val startReceiverFunc: Iterator[Receiver[_]] => Unit = + (iterator: Iterator[Receiver[_]]) => { + if (!iterator.hasNext) { + throw new SparkException( + "Could not start receiver as object not found.") + } + if (TaskContext.get().attemptNumber() == 0) { + val receiver = iterator.next() + assert(iterator.hasNext == false) + val supervisor = new ReceiverSupervisorImpl( + receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) + supervisor.start() + supervisor.awaitTermination() + } else { + // It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. + } + } // Create the RDD using the scheduledExecutors to run the receiver in a Spark job val receiverRDD: RDD[Receiver[_]] = @@ -541,31 +601,3 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } - -/** - * Function to start the receiver on the worker node. Use a class instead of closure to avoid - * the serialization issue. - */ -private[streaming] class StartReceiverFunc( - checkpointDirOption: Option[String], - serializableHadoopConf: SerializableConfiguration) - extends (Iterator[Receiver[_]] => Unit) with Serializable { - - override def apply(iterator: Iterator[Receiver[_]]): Unit = { - if (!iterator.hasNext) { - throw new SparkException( - "Could not start receiver as object not found.") - } - if (TaskContext.get().attemptNumber() == 0) { - val receiver = iterator.next() - assert(iterator.hasNext == false) - val supervisor = new ReceiverSupervisorImpl( - receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) - supervisor.start() - supervisor.awaitTermination() - } else { - // It's restarted by TaskScheduler, but we want to reschedule it again. So exit it. - } - } - -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala index 0418d776ecc9a..b2a51d72bac2b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverSchedulingPolicySuite.scala @@ -39,7 +39,7 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(scheduledExecutors.toSet === Set("host1", "host2")) } - test("rescheduleReceiver: return all idle executors if more than 3 idle executors") { + test("rescheduleReceiver: return all idle executors if there are any idle executors") { val executors = Seq("host1", "host2", "host3", "host4", "host5") // host3 is idle val receiverTrackingInfoMap = Map( @@ -49,16 +49,16 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(scheduledExecutors.toSet === Set("host2", "host3", "host4", "host5")) } - test("rescheduleReceiver: return 3 best options if less than 3 idle executors") { + test("rescheduleReceiver: return all executors that have minimum weight if no idle executors") { val executors = Seq("host1", "host2", "host3", "host4", "host5") - // Weights: host1 = 1.5, host2 = 0.5, host3 = 1.0 - // host4 and host5 are idle + // Weights: host1 = 1.5, host2 = 0.5, host3 = 1.0, host4 = 0.5, host5 = 0.5 val receiverTrackingInfoMap = Map( 0 -> ReceiverTrackingInfo(0, ReceiverState.ACTIVE, None, Some("host1")), 1 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host2", "host3")), None), - 2 -> ReceiverTrackingInfo(1, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None)) + 2 -> ReceiverTrackingInfo(2, ReceiverState.SCHEDULED, Some(Seq("host1", "host3")), None), + 3 -> ReceiverTrackingInfo(4, ReceiverState.SCHEDULED, Some(Seq("host4", "host5")), None)) val scheduledExecutors = receiverSchedulingPolicy.rescheduleReceiver( - 3, None, receiverTrackingInfoMap, executors) + 4, None, receiverTrackingInfoMap, executors) assert(scheduledExecutors.toSet === Set("host2", "host4", "host5")) } @@ -127,4 +127,5 @@ class ReceiverSchedulingPolicySuite extends SparkFunSuite { assert(executors.isEmpty) } } + } From df7041d02d3fd44b08a859f5d77bf6fb726895f0 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 24 Aug 2015 23:38:32 -0700 Subject: [PATCH 069/232] [SPARK-10196] [SQL] Correctly saving decimals in internal rows to JSON. https://issues.apache.org/jira/browse/SPARK-10196 Author: Yin Huai Closes #8408 from yhuai/DecimalJsonSPARK-10196. --- .../datasources/json/JacksonGenerator.scala | 2 +- .../sources/JsonHadoopFsRelationSuite.scala | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala index 99ac7730bd1c9..330ba907b2ef9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala @@ -95,7 +95,7 @@ private[sql] object JacksonGenerator { case (FloatType, v: Float) => gen.writeNumber(v) case (DoubleType, v: Double) => gen.writeNumber(v) case (LongType, v: Long) => gen.writeNumber(v) - case (DecimalType(), v: java.math.BigDecimal) => gen.writeNumber(v) + case (DecimalType(), v: Decimal) => gen.writeNumber(v.toJavaBigDecimal) case (ByteType, v: Byte) => gen.writeNumber(v.toInt) case (BinaryType, v: Array[Byte]) => gen.writeBinary(v) case (BooleanType, v: Boolean) => gen.writeBoolean(v) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala index ed6d512ab36fe..8ca3a17085194 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/JsonHadoopFsRelationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources +import java.math.BigDecimal + import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil @@ -75,4 +77,29 @@ class JsonHadoopFsRelationSuite extends HadoopFsRelationTest { ) } } + + test("SPARK-10196: save decimal type to JSON") { + withTempDir { file => + file.delete() + + val schema = + new StructType() + .add("decimal", DecimalType(7, 2)) + + val data = + Row(new BigDecimal("10.02")) :: + Row(new BigDecimal("20000.99")) :: + Row(new BigDecimal("10000")) :: Nil + val df = createDataFrame(sparkContext.parallelize(data), schema) + + // Write the data out. + df.write.format(dataSourceName).save(file.getCanonicalPath) + + // Read it back and check the result. + checkAnswer( + read.format(dataSourceName).schema(schema).load(file.getCanonicalPath), + df + ) + } + } } From bf03fe68d62f33dda70dff45c3bda1f57b032dfc Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 25 Aug 2015 14:58:42 +0800 Subject: [PATCH 070/232] [SPARK-10136] [SQL] A more robust fix for SPARK-10136 PR #8341 is a valid fix for SPARK-10136, but it didn't catch the real root cause. The real problem can be rather tricky to explain, and requires audiences to be pretty familiar with parquet-format spec, especially details of `LIST` backwards-compatibility rules. Let me have a try to give an explanation here. The structure of the problematic Parquet schema generated by parquet-avro is something like this: ``` message m { group f (LIST) { // Level 1 repeated group array (LIST) { // Level 2 repeated array; // Level 3 } } } ``` (The schema generated by parquet-thrift is structurally similar, just replace the `array` at level 2 with `f_tuple`, and the other one at level 3 with `f_tuple_tuple`.) This structure consists of two nested legacy 2-level `LIST`-like structures: 1. The repeated group type at level 2 is the element type of the outer array defined at level 1 This group should map to an `CatalystArrayConverter.ElementConverter` when building converters. 2. The repeated primitive type at level 3 is the element type of the inner array defined at level 2 This group should also map to an `CatalystArrayConverter.ElementConverter`. The root cause of SPARK-10136 is that, the group at level 2 isn't properly recognized as the element type of level 1. Thus, according to parquet-format spec, the repeated primitive at level 3 is left as a so called "unannotated repeated primitive type", and is recognized as a required list of required primitive type, thus a `RepeatedPrimitiveConverter` instead of a `CatalystArrayConverter.ElementConverter` is created for it. According to parquet-format spec, unannotated repeated type shouldn't appear in a `LIST`- or `MAP`-annotated group. PR #8341 fixed this issue by allowing such unannotated repeated type appear in `LIST`-annotated groups, which is a non-standard, hacky, but valid fix. (I didn't realize this when authoring #8341 though.) As for the reason why level 2 isn't recognized as a list element type, it's because of the following `LIST` backwards-compatibility rule defined in the parquet-format spec: > If the repeated field is a group with one field and is named either `array` or uses the `LIST`-annotated group's name with `_tuple` appended then the repeated type is the element type and elements are required. (The `array` part is for parquet-avro compatibility, while the `_tuple` part is for parquet-thrift.) This rule is implemented in [`CatalystSchemaConverter.isElementType`] [1], but neglected in [`CatalystRowConverter.isElementType`] [2]. This PR delivers a more robust fix by adding this rule in the latter method. Note that parquet-avro 1.7.0 also suffers from this issue. Details can be found at [PARQUET-364] [3]. [1]: https://github.com/apache/spark/blob/85f9a61357994da5023b08b0a8a2eb09388ce7f8/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala#L259-L305 [2]: https://github.com/apache/spark/blob/85f9a61357994da5023b08b0a8a2eb09388ce7f8/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala#L456-L463 [3]: https://issues.apache.org/jira/browse/PARQUET-364 Author: Cheng Lian Closes #8361 from liancheng/spark-10136/proper-version. --- .../parquet/CatalystRowConverter.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index d2c2db51769ba..cbf0704c4a9a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -415,8 +415,9 @@ private[parquet] class CatalystRowConverter( private val elementConverter: Converter = { val repeatedType = parquetSchema.getType(0) val elementType = catalystSchema.elementType + val parentName = parquetSchema.getName - if (isElementType(repeatedType, elementType)) { + if (isElementType(repeatedType, elementType, parentName)) { newConverter(repeatedType, elementType, new ParentContainerUpdater { override def set(value: Any): Unit = currentArray += value }) @@ -453,10 +454,13 @@ private[parquet] class CatalystRowConverter( * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules */ // scalastyle:on - private def isElementType(parquetRepeatedType: Type, catalystElementType: DataType): Boolean = { + private def isElementType( + parquetRepeatedType: Type, catalystElementType: DataType, parentName: String): Boolean = { (parquetRepeatedType, catalystElementType) match { case (t: PrimitiveType, _) => true case (t: GroupType, _) if t.getFieldCount > 1 => true + case (t: GroupType, _) if t.getFieldCount == 1 && t.getName == "array" => true + case (t: GroupType, _) if t.getFieldCount == 1 && t.getName == parentName + "_tuple" => true case (t: GroupType, StructType(Array(f))) if f.name == t.getFieldName(0) => true case _ => false } @@ -474,15 +478,9 @@ private[parquet] class CatalystRowConverter( override def getConverter(fieldIndex: Int): Converter = converter - override def end(): Unit = { - converter.updater.end() - currentArray += currentElement - } + override def end(): Unit = currentArray += currentElement - override def start(): Unit = { - converter.updater.start() - currentElement = null - } + override def start(): Unit = currentElement = null } } From 82268f07abfa658869df2354ae72f8d6ddd119e8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 25 Aug 2015 00:04:10 -0700 Subject: [PATCH 071/232] [SPARK-9293] [SPARK-9813] Analysis should check that set operations are only performed on tables with equal numbers of columns This patch adds an analyzer rule to ensure that set operations (union, intersect, and except) are only applied to tables with the same number of columns. Without this rule, there are scenarios where invalid queries can return incorrect results instead of failing with error messages; SPARK-9813 provides one example of this problem. In other cases, the invalid query can crash at runtime with extremely confusing exceptions. I also performed a bit of cleanup to refactor some of those logical operators' code into a common `SetOperation` base class. Author: Josh Rosen Closes #7631 from JoshRosen/SPARK-9293. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 +++ .../catalyst/analysis/HiveTypeCoercion.scala | 14 +++---- .../plans/logical/basicOperators.scala | 38 +++++++++---------- .../analysis/AnalysisErrorSuite.scala | 18 +++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 2 +- 6 files changed, 48 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 39f554c137c98..7701fd0451041 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -137,6 +137,12 @@ trait CheckAnalysis { } } + case s @ SetOperation(left, right) if left.output.length != right.output.length => + failAnalysis( + s"${s.nodeName} can only be performed on tables with the same number of columns, " + + s"but the left table has ${left.output.length} columns and the right has " + + s"${right.output.length}") + case _ => // Fallbacks to the following checks } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 2cb067f4aac91..a1aa2a2b2c680 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -203,6 +203,7 @@ object HiveTypeCoercion { planName: String, left: LogicalPlan, right: LogicalPlan): (LogicalPlan, LogicalPlan) = { + require(left.output.length == right.output.length) val castedTypes = left.output.zip(right.output).map { case (lhs, rhs) if lhs.dataType != rhs.dataType => @@ -229,15 +230,10 @@ object HiveTypeCoercion { def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case p if p.analyzed => p - case u @ Union(left, right) if u.childrenResolved && !u.resolved => - val (newLeft, newRight) = widenOutputTypes(u.nodeName, left, right) - Union(newLeft, newRight) - case e @ Except(left, right) if e.childrenResolved && !e.resolved => - val (newLeft, newRight) = widenOutputTypes(e.nodeName, left, right) - Except(newLeft, newRight) - case i @ Intersect(left, right) if i.childrenResolved && !i.resolved => - val (newLeft, newRight) = widenOutputTypes(i.nodeName, left, right) - Intersect(newLeft, newRight) + case s @ SetOperation(left, right) if s.childrenResolved + && left.output.length == right.output.length && !s.resolved => + val (newLeft, newRight) = widenOutputTypes(s.nodeName, left, right) + s.makeCopy(Array(newLeft, newRight)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 73b8261260acb..722f69cdca827 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -89,13 +89,21 @@ case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output } -case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { +abstract class SetOperation(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { // TODO: These aren't really the same attributes as nullability etc might change. - override def output: Seq[Attribute] = left.output + final override def output: Seq[Attribute] = left.output - override lazy val resolved: Boolean = + final override lazy val resolved: Boolean = childrenResolved && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } + left.output.length == right.output.length && + left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } +} + +private[sql] object SetOperation { + def unapply(p: SetOperation): Option[(LogicalPlan, LogicalPlan)] = Some((p.left, p.right)) +} + +case class Union(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) { override def statistics: Statistics = { val sizeInBytes = left.statistics.sizeInBytes + right.statistics.sizeInBytes @@ -103,6 +111,10 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { } } +case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) + +case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(left, right) + case class Join( left: LogicalPlan, right: LogicalPlan, @@ -142,15 +154,6 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { override def output: Seq[Attribute] = child.output } - -case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - override def output: Seq[Attribute] = left.output - - override lazy val resolved: Boolean = - childrenResolved && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } -} - case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], @@ -160,7 +163,7 @@ case class InsertIntoTable( extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil - override def output: Seq[Attribute] = child.output + override def output: Seq[Attribute] = Seq.empty assert(overwrite || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { @@ -440,10 +443,3 @@ case object OneRowRelation extends LeafNode { override def statistics: Statistics = Statistics(sizeInBytes = 1) } -case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { - override def output: Seq[Attribute] = left.output - - override lazy val resolved: Boolean = - childrenResolved && - left.output.zip(right.output).forall { case (l, r) => l.dataType == r.dataType } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 7065adce04bf8..fbdd3a7776f50 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -145,6 +145,24 @@ class AnalysisErrorSuite extends AnalysisTest { UnresolvedTestPlan(), "unresolved" :: Nil) + errorTest( + "union with unequal number of columns", + testRelation.unionAll(testRelation2), + "union" :: "number of columns" :: testRelation2.output.length.toString :: + testRelation.output.length.toString :: Nil) + + errorTest( + "intersect with unequal number of columns", + testRelation.intersect(testRelation2), + "intersect" :: "number of columns" :: testRelation2.output.length.toString :: + testRelation.output.length.toString :: Nil) + + errorTest( + "except with unequal number of columns", + testRelation.except(testRelation2), + "except" :: "number of columns" :: testRelation2.output.length.toString :: + testRelation.output.length.toString :: Nil) + errorTest( "SPARK-9955: correct error message for aggregate", // When parse SQL string, we will wrap aggregate expressions with UnresolvedAlias. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bbe8c1911bf86..98d21aa76d64e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -751,7 +751,7 @@ private[hive] case class InsertIntoHiveTable( extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil - override def output: Seq[Attribute] = child.output + override def output: Seq[Attribute] = Seq.empty val numDynamicPartitions = partition.values.count(_.isEmpty) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 12c667e6e92da..62efda613a176 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -61,7 +61,7 @@ case class InsertIntoHiveTable( serializer } - def output: Seq[Attribute] = child.output + def output: Seq[Attribute] = Seq.empty def saveAsHiveFile( rdd: RDD[InternalRow], From d4549fe58fa0d781e0e891bceff893420cb1d598 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Tue, 25 Aug 2015 00:28:51 -0700 Subject: [PATCH 072/232] [SPARK-10214] [SPARKR] [DOCS] Improve SparkR Column, DataFrame API docs cc: shivaram ## Summary - Add name tags to each methods in DataFrame.R and column.R - Replace `rdname column` with `rdname {each_func}`. i.e. alias method : `rdname column` => `rdname alias` ## Generated PDF File https://drive.google.com/file/d/0B9biIZIU47lLNHN2aFpnQXlSeGs/view?usp=sharing ## JIRA [[SPARK-10214] Improve SparkR Column, DataFrame API docs - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10214) Author: Yu ISHIKAWA Closes #8414 from yu-iskw/SPARK-10214. --- R/pkg/R/DataFrame.R | 101 +++++++++++++++++++++++++++++++++++--------- R/pkg/R/column.R | 40 ++++++++++++------ R/pkg/R/generics.R | 2 +- 3 files changed, 109 insertions(+), 34 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 895603235011e..10f3c4ea59864 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -27,9 +27,10 @@ setOldClass("jobj") #' \code{jsonFile}, \code{table} etc. #' @rdname DataFrame #' @seealso jsonFile, table +#' @docType class #' -#' @param env An R environment that stores bookkeeping states of the DataFrame -#' @param sdf A Java object reference to the backing Scala DataFrame +#' @slot env An R environment that stores bookkeeping states of the DataFrame +#' @slot sdf A Java object reference to the backing Scala DataFrame #' @export setClass("DataFrame", slots = list(env = "environment", @@ -61,6 +62,7 @@ dataFrame <- function(sdf, isCached = FALSE) { #' @param x A SparkSQL DataFrame #' #' @rdname printSchema +#' @name printSchema #' @export #' @examples #'\dontrun{ @@ -84,6 +86,7 @@ setMethod("printSchema", #' @param x A SparkSQL DataFrame #' #' @rdname schema +#' @name schema #' @export #' @examples #'\dontrun{ @@ -106,6 +109,7 @@ setMethod("schema", #' @param x A SparkSQL DataFrame #' @param extended Logical. If extended is False, explain() only prints the physical plan. #' @rdname explain +#' @name explain #' @export #' @examples #'\dontrun{ @@ -135,6 +139,7 @@ setMethod("explain", #' @param x A SparkSQL DataFrame #' #' @rdname isLocal +#' @name isLocal #' @export #' @examples #'\dontrun{ @@ -158,6 +163,7 @@ setMethod("isLocal", #' @param numRows The number of rows to print. Defaults to 20. #' #' @rdname showDF +#' @name showDF #' @export #' @examples #'\dontrun{ @@ -181,6 +187,7 @@ setMethod("showDF", #' @param x A SparkSQL DataFrame #' #' @rdname show +#' @name show #' @export #' @examples #'\dontrun{ @@ -206,6 +213,7 @@ setMethod("show", "DataFrame", #' @param x A SparkSQL DataFrame #' #' @rdname dtypes +#' @name dtypes #' @export #' @examples #'\dontrun{ @@ -230,6 +238,8 @@ setMethod("dtypes", #' @param x A SparkSQL DataFrame #' #' @rdname columns +#' @name columns +#' @aliases names #' @export #' @examples #'\dontrun{ @@ -248,7 +258,7 @@ setMethod("columns", }) #' @rdname columns -#' @aliases names,DataFrame,function-method +#' @name names setMethod("names", signature(x = "DataFrame"), function(x) { @@ -256,6 +266,7 @@ setMethod("names", }) #' @rdname columns +#' @name names<- setMethod("names<-", signature(x = "DataFrame"), function(x, value) { @@ -273,6 +284,7 @@ setMethod("names<-", #' @param tableName A character vector containing the name of the table #' #' @rdname registerTempTable +#' @name registerTempTable #' @export #' @examples #'\dontrun{ @@ -299,6 +311,7 @@ setMethod("registerTempTable", #' the existing rows in the table. #' #' @rdname insertInto +#' @name insertInto #' @export #' @examples #'\dontrun{ @@ -321,7 +334,8 @@ setMethod("insertInto", #' #' @param x A SparkSQL DataFrame #' -#' @rdname cache-methods +#' @rdname cache +#' @name cache #' @export #' @examples #'\dontrun{ @@ -347,6 +361,7 @@ setMethod("cache", #' #' @param x The DataFrame to persist #' @rdname persist +#' @name persist #' @export #' @examples #'\dontrun{ @@ -372,6 +387,7 @@ setMethod("persist", #' @param x The DataFrame to unpersist #' @param blocking Whether to block until all blocks are deleted #' @rdname unpersist-methods +#' @name unpersist #' @export #' @examples #'\dontrun{ @@ -397,6 +413,7 @@ setMethod("unpersist", #' @param x A SparkSQL DataFrame #' @param numPartitions The number of partitions to use. #' @rdname repartition +#' @name repartition #' @export #' @examples #'\dontrun{ @@ -446,6 +463,7 @@ setMethod("toJSON", #' @param x A SparkSQL DataFrame #' @param path The directory where the file is saved #' @rdname saveAsParquetFile +#' @name saveAsParquetFile #' @export #' @examples #'\dontrun{ @@ -467,6 +485,7 @@ setMethod("saveAsParquetFile", #' #' @param x A SparkSQL DataFrame #' @rdname distinct +#' @name distinct #' @export #' @examples #'\dontrun{ @@ -488,7 +507,8 @@ setMethod("distinct", #' @description Returns a new DataFrame containing distinct rows in this DataFrame #' #' @rdname unique -#' @aliases unique +#' @name unique +#' @aliases distinct setMethod("unique", signature(x = "DataFrame"), function(x) { @@ -526,7 +546,7 @@ setMethod("sample", }) #' @rdname sample -#' @aliases sample +#' @name sample_frac setMethod("sample_frac", signature(x = "DataFrame", withReplacement = "logical", fraction = "numeric"), @@ -541,6 +561,8 @@ setMethod("sample_frac", #' @param x A SparkSQL DataFrame #' #' @rdname count +#' @name count +#' @aliases nrow #' @export #' @examples #'\dontrun{ @@ -574,6 +596,7 @@ setMethod("nrow", #' @param x a SparkSQL DataFrame #' #' @rdname ncol +#' @name ncol #' @export #' @examples #'\dontrun{ @@ -593,6 +616,7 @@ setMethod("ncol", #' @param x a SparkSQL DataFrame #' #' @rdname dim +#' @name dim #' @export #' @examples #'\dontrun{ @@ -613,8 +637,8 @@ setMethod("dim", #' @param x A SparkSQL DataFrame #' @param stringsAsFactors (Optional) A logical indicating whether or not string columns #' should be converted to factors. FALSE by default. - -#' @rdname collect-methods +#' @rdname collect +#' @name collect #' @export #' @examples #'\dontrun{ @@ -650,6 +674,7 @@ setMethod("collect", #' @return A new DataFrame containing the number of rows specified. #' #' @rdname limit +#' @name limit #' @export #' @examples #' \dontrun{ @@ -669,6 +694,7 @@ setMethod("limit", #' Take the first NUM rows of a DataFrame and return a the results as a data.frame #' #' @rdname take +#' @name take #' @export #' @examples #'\dontrun{ @@ -696,6 +722,7 @@ setMethod("take", #' @return A data.frame #' #' @rdname head +#' @name head #' @export #' @examples #'\dontrun{ @@ -717,6 +744,7 @@ setMethod("head", #' @param x A SparkSQL DataFrame #' #' @rdname first +#' @name first #' @export #' @examples #'\dontrun{ @@ -732,7 +760,7 @@ setMethod("first", take(x, 1) }) -# toRDD() +# toRDD # # Converts a Spark DataFrame to an RDD while preserving column names. # @@ -769,6 +797,7 @@ setMethod("toRDD", #' @seealso GroupedData #' @aliases group_by #' @rdname groupBy +#' @name groupBy #' @export #' @examples #' \dontrun{ @@ -792,7 +821,7 @@ setMethod("groupBy", }) #' @rdname groupBy -#' @aliases group_by +#' @name group_by setMethod("group_by", signature(x = "DataFrame"), function(x, ...) { @@ -804,7 +833,8 @@ setMethod("group_by", #' Compute aggregates by specifying a list of columns #' #' @param x a DataFrame -#' @rdname DataFrame +#' @rdname agg +#' @name agg #' @aliases summarize #' @export setMethod("agg", @@ -813,8 +843,8 @@ setMethod("agg", agg(groupBy(x), ...) }) -#' @rdname DataFrame -#' @aliases agg +#' @rdname agg +#' @name summarize setMethod("summarize", signature(x = "DataFrame"), function(x, ...) { @@ -890,12 +920,14 @@ getColumn <- function(x, c) { } #' @rdname select +#' @name $ setMethod("$", signature(x = "DataFrame"), function(x, name) { getColumn(x, name) }) #' @rdname select +#' @name $<- setMethod("$<-", signature(x = "DataFrame"), function(x, name, value) { stopifnot(class(value) == "Column" || is.null(value)) @@ -923,6 +955,7 @@ setMethod("$<-", signature(x = "DataFrame"), }) #' @rdname select +#' @name [[ setMethod("[[", signature(x = "DataFrame"), function(x, i) { if (is.numeric(i)) { @@ -933,6 +966,7 @@ setMethod("[[", signature(x = "DataFrame"), }) #' @rdname select +#' @name [ setMethod("[", signature(x = "DataFrame", i = "missing"), function(x, i, j, ...) { if (is.numeric(j)) { @@ -1008,6 +1042,7 @@ setMethod("select", #' @param ... Additional expressions #' @return A DataFrame #' @rdname selectExpr +#' @name selectExpr #' @export #' @examples #'\dontrun{ @@ -1034,6 +1069,8 @@ setMethod("selectExpr", #' @param col A Column expression. #' @return A DataFrame with the new column added. #' @rdname withColumn +#' @name withColumn +#' @aliases mutate #' @export #' @examples #'\dontrun{ @@ -1057,7 +1094,7 @@ setMethod("withColumn", #' @param col a named argument of the form name = col #' @return A new DataFrame with the new columns added. #' @rdname withColumn -#' @aliases withColumn +#' @name mutate #' @export #' @examples #'\dontrun{ @@ -1094,6 +1131,7 @@ setMethod("mutate", #' @param newCol The new column name. #' @return A DataFrame with the column name changed. #' @rdname withColumnRenamed +#' @name withColumnRenamed #' @export #' @examples #'\dontrun{ @@ -1124,6 +1162,7 @@ setMethod("withColumnRenamed", #' @param newCol A named pair of the form new_column_name = existing_column #' @return A DataFrame with the column name changed. #' @rdname withColumnRenamed +#' @name rename #' @aliases withColumnRenamed #' @export #' @examples @@ -1165,6 +1204,8 @@ setClassUnion("characterOrColumn", c("character", "Column")) #' @param ... Additional sorting fields #' @return A DataFrame where all elements are sorted. #' @rdname arrange +#' @name arrange +#' @aliases orderby #' @export #' @examples #'\dontrun{ @@ -1191,7 +1232,7 @@ setMethod("arrange", }) #' @rdname arrange -#' @aliases orderBy,DataFrame,function-method +#' @name orderby setMethod("orderBy", signature(x = "DataFrame", col = "characterOrColumn"), function(x, col) { @@ -1207,6 +1248,7 @@ setMethod("orderBy", #' or a string containing a SQL statement #' @return A DataFrame containing only the rows that meet the condition. #' @rdname filter +#' @name filter #' @export #' @examples #'\dontrun{ @@ -1228,7 +1270,7 @@ setMethod("filter", }) #' @rdname filter -#' @aliases where,DataFrame,function-method +#' @name where setMethod("where", signature(x = "DataFrame", condition = "characterOrColumn"), function(x, condition) { @@ -1247,6 +1289,7 @@ setMethod("where", #' 'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'. The default joinType is "inner". #' @return A DataFrame containing the result of the join operation. #' @rdname join +#' @name join #' @export #' @examples #'\dontrun{ @@ -1279,8 +1322,9 @@ setMethod("join", dataFrame(sdf) }) -#' rdname merge -#' aliases join +#' @rdname merge +#' @name merge +#' @aliases join setMethod("merge", signature(x = "DataFrame", y = "DataFrame"), function(x, y, joinExpr = NULL, joinType = NULL, ...) { @@ -1298,6 +1342,7 @@ setMethod("merge", #' @param y A Spark DataFrame #' @return A DataFrame containing the result of the union. #' @rdname unionAll +#' @name unionAll #' @export #' @examples #'\dontrun{ @@ -1319,6 +1364,7 @@ setMethod("unionAll", #' @description Returns a new DataFrame containing rows of all parameters. # #' @rdname rbind +#' @name rbind #' @aliases unionAll setMethod("rbind", signature(... = "DataFrame"), @@ -1339,6 +1385,7 @@ setMethod("rbind", #' @param y A Spark DataFrame #' @return A DataFrame containing the result of the intersect. #' @rdname intersect +#' @name intersect #' @export #' @examples #'\dontrun{ @@ -1364,6 +1411,7 @@ setMethod("intersect", #' @param y A Spark DataFrame #' @return A DataFrame containing the result of the except operation. #' @rdname except +#' @name except #' @export #' @examples #'\dontrun{ @@ -1403,6 +1451,8 @@ setMethod("except", #' @param mode One of 'append', 'overwrite', 'error', 'ignore' #' #' @rdname write.df +#' @name write.df +#' @aliases saveDF #' @export #' @examples #'\dontrun{ @@ -1435,7 +1485,7 @@ setMethod("write.df", }) #' @rdname write.df -#' @aliases saveDF +#' @name saveDF #' @export setMethod("saveDF", signature(df = "DataFrame", path = "character"), @@ -1466,6 +1516,7 @@ setMethod("saveDF", #' @param mode One of 'append', 'overwrite', 'error', 'ignore' #' #' @rdname saveAsTable +#' @name saveAsTable #' @export #' @examples #'\dontrun{ @@ -1505,6 +1556,8 @@ setMethod("saveAsTable", #' @param ... Additional expressions #' @return A DataFrame #' @rdname describe +#' @name describe +#' @aliases summary #' @export #' @examples #'\dontrun{ @@ -1525,6 +1578,7 @@ setMethod("describe", }) #' @rdname describe +#' @name describe setMethod("describe", signature(x = "DataFrame"), function(x) { @@ -1538,7 +1592,7 @@ setMethod("describe", #' @description Computes statistics for numeric columns of the DataFrame #' #' @rdname summary -#' @aliases describe +#' @name summary setMethod("summary", signature(x = "DataFrame"), function(x) { @@ -1562,6 +1616,8 @@ setMethod("summary", #' @return A DataFrame #' #' @rdname nafunctions +#' @name dropna +#' @aliases na.omit #' @export #' @examples #'\dontrun{ @@ -1588,7 +1644,8 @@ setMethod("dropna", dataFrame(sdf) }) -#' @aliases dropna +#' @rdname nafunctions +#' @name na.omit #' @export setMethod("na.omit", signature(x = "DataFrame"), @@ -1615,6 +1672,7 @@ setMethod("na.omit", #' @return A DataFrame #' #' @rdname nafunctions +#' @name fillna #' @export #' @examples #'\dontrun{ @@ -1685,6 +1743,7 @@ setMethod("fillna", #' occurrences will have zero as their counts. #' #' @rdname statfunctions +#' @name crosstab #' @export #' @examples #' \dontrun{ diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index a1f50c383367c..4805096f3f9c5 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -24,10 +24,9 @@ setOldClass("jobj") #' @title S4 class that represents a DataFrame column #' @description The column class supports unary, binary operations on DataFrame columns - #' @rdname column #' -#' @param jc reference to JVM DataFrame column +#' @slot jc reference to JVM DataFrame column #' @export setClass("Column", slots = list(jc = "jobj")) @@ -46,6 +45,7 @@ col <- function(x) { } #' @rdname show +#' @name show setMethod("show", "Column", function(object) { cat("Column", callJMethod(object@jc, "toString"), "\n") @@ -122,8 +122,11 @@ createMethods() #' alias #' #' Set a new name for a column - -#' @rdname column +#' +#' @rdname alias +#' @name alias +#' @family colum_func +#' @export setMethod("alias", signature(object = "Column"), function(object, data) { @@ -138,7 +141,9 @@ setMethod("alias", #' #' An expression that returns a substring. #' -#' @rdname column +#' @rdname substr +#' @name substr +#' @family colum_func #' #' @param start starting position #' @param stop ending position @@ -152,7 +157,9 @@ setMethod("substr", signature(x = "Column"), #' #' Test if the column is between the lower bound and upper bound, inclusive. #' -#' @rdname column +#' @rdname between +#' @name between +#' @family colum_func #' #' @param bounds lower and upper bounds setMethod("between", signature(x = "Column"), @@ -167,7 +174,9 @@ setMethod("between", signature(x = "Column"), #' Casts the column to a different data type. #' -#' @rdname column +#' @rdname cast +#' @name cast +#' @family colum_func #' #' @examples \dontrun{ #' cast(df$age, "string") @@ -189,11 +198,15 @@ setMethod("cast", #' Match a column with given values. #' -#' @rdname column +#' @rdname match +#' @name %in% +#' @aliases %in% #' @return a matched values as a result of comparing with given values. -#' @examples \dontrun{ -#' filter(df, "age in (10, 30)") -#' where(df, df$age %in% c(10, 30)) +#' @export +#' @examples +#' \dontrun{ +#' filter(df, "age in (10, 30)") +#' where(df, df$age %in% c(10, 30)) #' } setMethod("%in%", signature(x = "Column"), @@ -208,7 +221,10 @@ setMethod("%in%", #' If values in the specified column are null, returns the value. #' Can be used in conjunction with `when` to specify a default value for expressions. #' -#' @rdname column +#' @rdname otherwise +#' @name otherwise +#' @family colum_func +#' @export setMethod("otherwise", signature(x = "Column", value = "ANY"), function(x, value) { diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 610a8c31223cd..a829d46c1894c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -441,7 +441,7 @@ setGeneric("filter", function(x, condition) { standardGeneric("filter") }) #' @export setGeneric("group_by", function(x, ...) { standardGeneric("group_by") }) -#' @rdname DataFrame +#' @rdname groupBy #' @export setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") }) From 57b960bf3706728513f9e089455a533f0244312e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 25 Aug 2015 08:32:20 +0100 Subject: [PATCH 073/232] [SPARK-6196] [BUILD] Remove MapR profiles in favor of hadoop-provided Follow up to https://github.com/apache/spark/pull/7047 pwendell mentioned that MapR should use `hadoop-provided` now, and indeed the new build script does not produce `mapr3`/`mapr4` artifacts anymore. Hence the action seems to be to remove the profiles, which are now not used. CC trystanleftwich Author: Sean Owen Closes #8338 from srowen/SPARK-6196. --- pom.xml | 38 -------------------------------------- 1 file changed, 38 deletions(-) diff --git a/pom.xml b/pom.xml index d5945f2546d38..0716016523ee1 100644 --- a/pom.xml +++ b/pom.xml @@ -2386,44 +2386,6 @@ - - mapr3 - - 1.0.3-mapr-3.0.3 - 2.4.1-mapr-1408 - 0.98.4-mapr-1408 - 3.4.5-mapr-1406 - - - - - mapr4 - - 2.4.1-mapr-1408 - 2.4.1-mapr-1408 - 0.98.4-mapr-1408 - 3.4.5-mapr-1406 - - - - org.apache.curator - curator-recipes - ${curator.version} - - - org.apache.zookeeper - zookeeper - - - - - org.apache.zookeeper - zookeeper - 3.4.5-mapr-1406 - - - - hive-thriftserver From 1fc37581a52530bac5d555dbf14927a5780c3b75 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Aug 2015 00:35:51 -0700 Subject: [PATCH 074/232] [SPARK-10210] [STREAMING] Filter out non-existent blocks before creating BlockRDD When write ahead log is not enabled, a recovered streaming driver still tries to run jobs using pre-failure block ids, and fails as the block do not exists in-memory any more (and cannot be recovered as receiver WAL is not enabled). This occurs because the driver-side WAL of ReceivedBlockTracker is recovers that past block information, and ReceiveInputDStream creates BlockRDDs even if those blocks do not exist. The solution in this PR is to filter out block ids that do not exist before creating the BlockRDD. In addition, it adds unit tests to verify other logic in ReceiverInputDStream. Author: Tathagata Das Closes #8405 from tdas/SPARK-10210. --- .../dstream/ReceiverInputDStream.scala | 10 +- .../rdd/WriteAheadLogBackedBlockRDD.scala | 2 +- .../streaming/ReceiverInputDStreamSuite.scala | 156 ++++++++++++++++++ 3 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index a15800917c6f4..6c139f32da31d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -116,7 +116,15 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont logWarning("Some blocks have Write Ahead Log information; this is unexpected") } } - new BlockRDD[T](ssc.sc, blockIds) + val validBlockIds = blockIds.filter { id => + ssc.sparkContext.env.blockManager.master.contains(id) + } + if (validBlockIds.size != blockIds.size) { + logWarning("Some blocks could not be recovered as they were not found in memory. " + + "To prevent such data loss, enabled Write Ahead Log (see programming guide " + + "for more details.") + } + new BlockRDD[T](ssc.sc, validBlockIds) } } else { // If no block is ready now, creating WriteAheadLogBackedBlockRDD or BlockRDD diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 620b8a36a2baf..e081ffe46f502 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -75,7 +75,7 @@ private[streaming] class WriteAheadLogBackedBlockRDD[T: ClassTag]( @transient sc: SparkContext, @transient blockIds: Array[BlockId], - @transient walRecordHandles: Array[WriteAheadLogRecordHandle], + @transient val walRecordHandles: Array[WriteAheadLogRecordHandle], @transient isBlockIdValid: Array[Boolean] = Array.empty, storeInBlockManager: Boolean = false, storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala new file mode 100644 index 0000000000000..6d388d9624d92 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala @@ -0,0 +1,156 @@ +/* + * 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 + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.rdd.BlockRDD +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD +import org.apache.spark.streaming.receiver.{BlockManagerBasedStoreResult, Receiver, WriteAheadLogBasedStoreResult} +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo +import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} +import org.apache.spark.{SparkConf, SparkEnv} + +class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { + + override def afterAll(): Unit = { + StreamingContext.getActive().map { _.stop() } + } + + testWithoutWAL("createBlockRDD creates empty BlockRDD when no block info") { receiverStream => + val rdd = receiverStream.createBlockRDD(Time(0), Seq.empty) + assert(rdd.isInstanceOf[BlockRDD[_]]) + assert(!rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) + assert(rdd.isEmpty()) + } + + testWithoutWAL("createBlockRDD creates correct BlockRDD with block info") { receiverStream => + val blockInfos = Seq.fill(5) { createBlockInfo(withWALInfo = false) } + val blockIds = blockInfos.map(_.blockId) + + // Verify that there are some blocks that are present, and some that are not + require(blockIds.forall(blockId => SparkEnv.get.blockManager.master.contains(blockId))) + + val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) + assert(rdd.isInstanceOf[BlockRDD[_]]) + assert(!rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) + val blockRDD = rdd.asInstanceOf[BlockRDD[_]] + assert(blockRDD.blockIds.toSeq === blockIds) + } + + testWithoutWAL("createBlockRDD filters non-existent blocks before creating BlockRDD") { + receiverStream => + val presentBlockInfos = Seq.fill(2)(createBlockInfo(withWALInfo = false, createBlock = true)) + val absentBlockInfos = Seq.fill(3)(createBlockInfo(withWALInfo = false, createBlock = false)) + val blockInfos = presentBlockInfos ++ absentBlockInfos + val blockIds = blockInfos.map(_.blockId) + + // Verify that there are some blocks that are present, and some that are not + require(blockIds.exists(blockId => SparkEnv.get.blockManager.master.contains(blockId))) + require(blockIds.exists(blockId => !SparkEnv.get.blockManager.master.contains(blockId))) + + val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) + assert(rdd.isInstanceOf[BlockRDD[_]]) + val blockRDD = rdd.asInstanceOf[BlockRDD[_]] + assert(blockRDD.blockIds.toSeq === presentBlockInfos.map { _.blockId}) + } + + testWithWAL("createBlockRDD creates empty WALBackedBlockRDD when no block info") { + receiverStream => + val rdd = receiverStream.createBlockRDD(Time(0), Seq.empty) + assert(rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) + assert(rdd.isEmpty()) + } + + testWithWAL( + "createBlockRDD creates correct WALBackedBlockRDD with all block info having WAL info") { + receiverStream => + val blockInfos = Seq.fill(5) { createBlockInfo(withWALInfo = true) } + val blockIds = blockInfos.map(_.blockId) + val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) + assert(rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) + val blockRDD = rdd.asInstanceOf[WriteAheadLogBackedBlockRDD[_]] + assert(blockRDD.blockIds.toSeq === blockIds) + assert(blockRDD.walRecordHandles.toSeq === blockInfos.map { _.walRecordHandleOption.get }) + } + + testWithWAL("createBlockRDD creates BlockRDD when some block info dont have WAL info") { + receiverStream => + val blockInfos1 = Seq.fill(2) { createBlockInfo(withWALInfo = true) } + val blockInfos2 = Seq.fill(3) { createBlockInfo(withWALInfo = false) } + val blockInfos = blockInfos1 ++ blockInfos2 + val blockIds = blockInfos.map(_.blockId) + val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) + assert(rdd.isInstanceOf[BlockRDD[_]]) + val blockRDD = rdd.asInstanceOf[BlockRDD[_]] + assert(blockRDD.blockIds.toSeq === blockIds) + } + + + private def testWithoutWAL(msg: String)(body: ReceiverInputDStream[_] => Unit): Unit = { + test(s"Without WAL enabled: $msg") { + runTest(enableWAL = false, body) + } + } + + private def testWithWAL(msg: String)(body: ReceiverInputDStream[_] => Unit): Unit = { + test(s"With WAL enabled: $msg") { + runTest(enableWAL = true, body) + } + } + + private def runTest(enableWAL: Boolean, body: ReceiverInputDStream[_] => Unit): Unit = { + val conf = new SparkConf() + conf.setMaster("local[4]").setAppName("ReceiverInputDStreamSuite") + conf.set(WriteAheadLogUtils.RECEIVER_WAL_ENABLE_CONF_KEY, enableWAL.toString) + require(WriteAheadLogUtils.enableReceiverLog(conf) === enableWAL) + val ssc = new StreamingContext(conf, Seconds(1)) + val receiverStream = new ReceiverInputDStream[Int](ssc) { + override def getReceiver(): Receiver[Int] = null + } + withStreamingContext(ssc) { ssc => + body(receiverStream) + } + } + + /** + * Create a block info for input to the ReceiverInputDStream.createBlockRDD + * @param withWALInfo Create block with WAL info in it + * @param createBlock Actually create the block in the BlockManager + * @return + */ + private def createBlockInfo( + withWALInfo: Boolean, + createBlock: Boolean = true): ReceivedBlockInfo = { + val blockId = new StreamBlockId(0, Random.nextLong()) + if (createBlock) { + SparkEnv.get.blockManager.putSingle(blockId, 1, StorageLevel.MEMORY_ONLY, tellMaster = true) + require(SparkEnv.get.blockManager.master.contains(blockId)) + } + val storeResult = if (withWALInfo) { + new WriteAheadLogBasedStoreResult(blockId, None, new WriteAheadLogRecordHandle { }) + } else { + new BlockManagerBasedStoreResult(blockId, None) + } + new ReceivedBlockInfo(0, None, None, storeResult) + } +} From 2f493f7e3924b769160a16f73cccbebf21973b91 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 25 Aug 2015 16:00:44 +0800 Subject: [PATCH 075/232] [SPARK-10177] [SQL] fix reading Timestamp in parquet from Hive We misunderstood the Julian days and nanoseconds of the day in parquet (as TimestampType) from Hive/Impala, they are overlapped, so can't be added together directly. In order to avoid the confusing rounding when do the converting, we use `2440588` as the Julian Day of epoch of unix timestamp (which should be 2440587.5). Author: Davies Liu Author: Cheng Lian Closes #8400 from davies/timestamp_parquet. --- .../spark/sql/catalyst/util/DateTimeUtils.scala | 7 ++++--- .../sql/catalyst/util/DateTimeUtilsSuite.scala | 13 +++++++++---- .../sql/hive/ParquetHiveCompatibilitySuite.scala | 2 +- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 672620460c3c5..d652fce3fd9b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -37,7 +37,8 @@ object DateTimeUtils { type SQLTimestamp = Long // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian - final val JULIAN_DAY_OF_EPOCH = 2440587 // and .5 + // it's 2440587.5, rounding up to compatible with Hive + final val JULIAN_DAY_OF_EPOCH = 2440588 final val SECONDS_PER_DAY = 60 * 60 * 24L final val MICROS_PER_SECOND = 1000L * 1000L final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L @@ -183,7 +184,7 @@ object DateTimeUtils { */ def fromJulianDay(day: Int, nanoseconds: Long): SQLTimestamp = { // use Long to avoid rounding errors - val seconds = (day - JULIAN_DAY_OF_EPOCH).toLong * SECONDS_PER_DAY - SECONDS_PER_DAY / 2 + val seconds = (day - JULIAN_DAY_OF_EPOCH).toLong * SECONDS_PER_DAY seconds * MICROS_PER_SECOND + nanoseconds / 1000L } @@ -191,7 +192,7 @@ object DateTimeUtils { * Returns Julian day and nanoseconds in a day from the number of microseconds */ def toJulianDay(us: SQLTimestamp): (Int, Long) = { - val seconds = us / MICROS_PER_SECOND + SECONDS_PER_DAY / 2 + val seconds = us / MICROS_PER_SECOND val day = seconds / SECONDS_PER_DAY + JULIAN_DAY_OF_EPOCH val secondsInDay = seconds % SECONDS_PER_DAY val nanos = (us % MICROS_PER_SECOND) * 1000L diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index d18fa4df13355..1596bb79fa94b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -49,13 +49,18 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("us and julian day") { val (d, ns) = toJulianDay(0) assert(d === JULIAN_DAY_OF_EPOCH) - assert(ns === SECONDS_PER_DAY / 2 * NANOS_PER_SECOND) + assert(ns === 0) assert(fromJulianDay(d, ns) == 0L) - val t = new Timestamp(61394778610000L) // (2015, 6, 11, 10, 10, 10, 100) + val t = Timestamp.valueOf("2015-06-11 10:10:10.100") val (d1, ns1) = toJulianDay(fromJavaTimestamp(t)) - val t2 = toJavaTimestamp(fromJulianDay(d1, ns1)) - assert(t.equals(t2)) + val t1 = toJavaTimestamp(fromJulianDay(d1, ns1)) + assert(t.equals(t1)) + + val t2 = Timestamp.valueOf("2015-06-11 20:10:10.100") + val (d2, ns2) = toJulianDay(fromJavaTimestamp(t2)) + val t22 = toJavaTimestamp(fromJulianDay(d2, ns2)) + assert(t2.equals(t22)) } test("SPARK-6785: java date conversion before and after epoch") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala index bc30180cf0917..91d7a48208e8d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -113,7 +113,7 @@ class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest with Before "BOOLEAN", "TINYINT", "SMALLINT", "INT", "BIGINT", "FLOAT", "DOUBLE", "STRING") } - ignore("SPARK-10177 timestamp") { + test("SPARK-10177 timestamp") { testParquetHiveCompatibility(Row(Timestamp.valueOf("2015-08-24 00:31:00")), "TIMESTAMP") } From 7bc9a8c6249300ded31ea931c463d0a8f798e193 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 25 Aug 2015 01:06:36 -0700 Subject: [PATCH 076/232] [SPARK-10195] [SQL] Data sources Filter should not expose internal types Spark SQL's data sources API exposes Catalyst's internal types through its Filter interfaces. This is a problem because types like UTF8String are not stable developer APIs and should not be exposed to third-parties. This issue caused incompatibilities when upgrading our `spark-redshift` library to work against Spark 1.5.0. To avoid these issues in the future we should only expose public types through these Filter objects. This patch accomplishes this by using CatalystTypeConverters to add the appropriate conversions. Author: Josh Rosen Closes #8403 from JoshRosen/datasources-internal-vs-external-types. --- .../datasources/DataSourceStrategy.scala | 67 ++++++++++--------- .../execution/datasources/jdbc/JDBCRDD.scala | 2 +- .../datasources/parquet/ParquetFilters.scala | 19 +++--- .../spark/sql/sources/FilteredScanSuite.scala | 7 ++ 4 files changed, 54 insertions(+), 41 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 2a4c40db8bb66..6c1ef6a6df887 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.{Logging, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{MapPartitionsRDD, RDD, UnionRDD} -import org.apache.spark.sql.catalyst.{InternalRow, expressions} +import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, expressions} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical @@ -344,45 +345,47 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { */ protected[sql] def selectFilters(filters: Seq[Expression]) = { def translate(predicate: Expression): Option[Filter] = predicate match { - case expressions.EqualTo(a: Attribute, Literal(v, _)) => - Some(sources.EqualTo(a.name, v)) - case expressions.EqualTo(Literal(v, _), a: Attribute) => - Some(sources.EqualTo(a.name, v)) - - case expressions.EqualNullSafe(a: Attribute, Literal(v, _)) => - Some(sources.EqualNullSafe(a.name, v)) - case expressions.EqualNullSafe(Literal(v, _), a: Attribute) => - Some(sources.EqualNullSafe(a.name, v)) - - case expressions.GreaterThan(a: Attribute, Literal(v, _)) => - Some(sources.GreaterThan(a.name, v)) - case expressions.GreaterThan(Literal(v, _), a: Attribute) => - Some(sources.LessThan(a.name, v)) - - case expressions.LessThan(a: Attribute, Literal(v, _)) => - Some(sources.LessThan(a.name, v)) - case expressions.LessThan(Literal(v, _), a: Attribute) => - Some(sources.GreaterThan(a.name, v)) - - case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, _)) => - Some(sources.GreaterThanOrEqual(a.name, v)) - case expressions.GreaterThanOrEqual(Literal(v, _), a: Attribute) => - Some(sources.LessThanOrEqual(a.name, v)) - - case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) => - Some(sources.LessThanOrEqual(a.name, v)) - case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) => - Some(sources.GreaterThanOrEqual(a.name, v)) + case expressions.EqualTo(a: Attribute, Literal(v, t)) => + Some(sources.EqualTo(a.name, convertToScala(v, t))) + case expressions.EqualTo(Literal(v, t), a: Attribute) => + Some(sources.EqualTo(a.name, convertToScala(v, t))) + + case expressions.EqualNullSafe(a: Attribute, Literal(v, t)) => + Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) + case expressions.EqualNullSafe(Literal(v, t), a: Attribute) => + Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) + + case expressions.GreaterThan(a: Attribute, Literal(v, t)) => + Some(sources.GreaterThan(a.name, convertToScala(v, t))) + case expressions.GreaterThan(Literal(v, t), a: Attribute) => + Some(sources.LessThan(a.name, convertToScala(v, t))) + + case expressions.LessThan(a: Attribute, Literal(v, t)) => + Some(sources.LessThan(a.name, convertToScala(v, t))) + case expressions.LessThan(Literal(v, t), a: Attribute) => + Some(sources.GreaterThan(a.name, convertToScala(v, t))) + + case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, t)) => + Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) + case expressions.GreaterThanOrEqual(Literal(v, t), a: Attribute) => + Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) + + case expressions.LessThanOrEqual(a: Attribute, Literal(v, t)) => + Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) + case expressions.LessThanOrEqual(Literal(v, t), a: Attribute) => + Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) case expressions.InSet(a: Attribute, set) => - Some(sources.In(a.name, set.toArray)) + val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) + Some(sources.In(a.name, set.toArray.map(toScala))) // Because we only convert In to InSet in Optimizer when there are more than certain // items. So it is possible we still get an In expression here that needs to be pushed // down. case expressions.In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) => val hSet = list.map(e => e.eval(EmptyRow)) - Some(sources.In(a.name, hSet.toArray)) + val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) + Some(sources.In(a.name, hSet.toArray.map(toScala))) case expressions.IsNull(a: Attribute) => Some(sources.IsNull(a.name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala index e537d631f4559..730d88b024cb1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala @@ -262,7 +262,7 @@ private[sql] class JDBCRDD( * Converts value to SQL expression. */ private def compileValue(value: Any): Any = value match { - case stringValue: UTF8String => s"'${escapeSql(stringValue.toString)}'" + case stringValue: String => s"'${escapeSql(stringValue)}'" case _ => value } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index c74c8388632f5..c6b3fe7900da8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -32,7 +32,6 @@ import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.sources import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" @@ -65,7 +64,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), @@ -86,7 +85,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), @@ -104,7 +103,8 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) + FilterApi.lt(binaryColumn(n), + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -121,7 +121,8 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) + FilterApi.ltEq(binaryColumn(n), + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -138,7 +139,8 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) + FilterApi.gt(binaryColumn(n), + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -155,7 +157,8 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) + FilterApi.gtEq(binaryColumn(n), + Binary.fromByteArray(v.asInstanceOf[String].getBytes("utf-8"))) case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -177,7 +180,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), - SetInFilter(v.map(e => Binary.fromByteArray(e.asInstanceOf[UTF8String].getBytes)))) + SetInFilter(v.map(s => Binary.fromByteArray(s.asInstanceOf[String].getBytes("utf-8"))))) case BinaryType => (n: String, v: Set[Any]) => FilterApi.userDefined(binaryColumn(n), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index c81c3d3982805..68ce37c00077e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.sources import scala.language.existentials import org.apache.spark.rdd.RDD +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.sql._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -78,6 +79,9 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL case StringStartsWith("c", v) => _.startsWith(v) case StringEndsWith("c", v) => _.endsWith(v) case StringContains("c", v) => _.contains(v) + case EqualTo("c", v: String) => _.equals(v) + case EqualTo("c", v: UTF8String) => sys.error("UTF8String should not appear in filters") + case In("c", values) => (s: String) => values.map(_.asInstanceOf[String]).toSet.contains(s) case _ => (c: String) => true } @@ -237,6 +241,9 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext { testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%eE%'", 1) testPushDown("SELECT a, b, c FROM oneToTenFiltered WHERE c like '%Ee%'", 0) + testPushDown("SELECT c FROM oneToTenFiltered WHERE c = 'aaaaaAAAAA'", 1) + testPushDown("SELECT c FROM oneToTenFiltered WHERE c IN ('aaaaaAAAAA', 'foo')", 1) + def testPushDown(sqlString: String, expectedCount: Int): Unit = { test(s"PushDown Returns $expectedCount: $sqlString") { val queryExecution = sql(sqlString).queryExecution From 0e6368ffaec1965d0c7f89420e04a974675c7f6e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 25 Aug 2015 16:19:34 +0800 Subject: [PATCH 077/232] [SPARK-10197] [SQL] Add null check in wrapperFor (inside HiveInspectors). https://issues.apache.org/jira/browse/SPARK-10197 Author: Yin Huai Closes #8407 from yhuai/ORCSPARK-10197. --- .../spark/sql/hive/HiveInspectors.scala | 29 +++++++++++++++---- .../spark/sql/hive/orc/OrcSourceSuite.scala | 29 +++++++++++++++++++ 2 files changed, 53 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 9824dad239596..64fffdbf9b020 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -370,17 +370,36 @@ private[hive] trait HiveInspectors { protected def wrapperFor(oi: ObjectInspector, dataType: DataType): Any => Any = oi match { case _: JavaHiveVarcharObjectInspector => (o: Any) => - val s = o.asInstanceOf[UTF8String].toString - new HiveVarchar(s, s.size) + if (o != null) { + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.size) + } else { + null + } case _: JavaHiveDecimalObjectInspector => - (o: Any) => HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal) + (o: Any) => + if (o != null) { + HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal) + } else { + null + } case _: JavaDateObjectInspector => - (o: Any) => DateTimeUtils.toJavaDate(o.asInstanceOf[Int]) + (o: Any) => + if (o != null) { + DateTimeUtils.toJavaDate(o.asInstanceOf[Int]) + } else { + null + } case _: JavaTimestampObjectInspector => - (o: Any) => DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long]) + (o: Any) => + if (o != null) { + DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long]) + } else { + null + } case soi: StandardStructObjectInspector => val schema = dataType.asInstanceOf[StructType] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala index 82e08caf46457..80c38084f293d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala @@ -121,6 +121,35 @@ abstract class OrcSuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM normal_orc_as_source"), (6 to 10).map(i => Row(i, s"part-$i"))) } + + test("write null values") { + sql("DROP TABLE IF EXISTS orcNullValues") + + val df = sql( + """ + |SELECT + | CAST(null as TINYINT), + | CAST(null as SMALLINT), + | CAST(null as INT), + | CAST(null as BIGINT), + | CAST(null as FLOAT), + | CAST(null as DOUBLE), + | CAST(null as DECIMAL(7,2)), + | CAST(null as TIMESTAMP), + | CAST(null as DATE), + | CAST(null as STRING), + | CAST(null as VARCHAR(10)) + |FROM orc_temp_table limit 1 + """.stripMargin) + + df.write.format("orc").saveAsTable("orcNullValues") + + checkAnswer( + sql("SELECT * FROM orcNullValues"), + Row.fromSeq(Seq.fill(11)(null))) + + sql("DROP TABLE IF EXISTS orcNullValues") + } } class OrcSourceSuite extends OrcSuite { From 5c14890159a5711072bf395f662b2433a389edf9 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 25 Aug 2015 11:48:55 +0100 Subject: [PATCH 078/232] [DOC] add missing parameters in SparkContext.scala for scala doc Author: Zhang, Liye Closes #8412 from liyezhang556520/minorDoc. --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1ddaca8a5ba8c..9849aff85d72e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -114,6 +114,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * :: DeveloperApi :: * Alternative constructor for setting preferred locations where Spark will create executors. * + * @param config a [[org.apache.spark.SparkConf]] object specifying other Spark parameters * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. * Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] * from a list of input files or InputFormats for the application. @@ -145,6 +146,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @param jars Collection of JARs to send to the cluster. These can be paths on the local file * system or HDFS, HTTP, HTTPS, or FTP URLs. * @param environment Environment variables to set on worker nodes. + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. + * Can be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. */ def this( master: String, @@ -841,6 +845,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @note Small files are preferred, large file is also allowable, but may cause bad performance. * @note On some filesystems, `.../path/*` can be a more efficient way to read all files * in a directory rather than `.../path/` or `.../path` + * + * @param path Directory to the input data files, the path can be comma separated paths as the + * list of inputs. * @param minPartitions A suggestion value of the minimal splitting number for input data. */ def wholeTextFiles( @@ -889,6 +896,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @note Small files are preferred; very large files may cause bad performance. * @note On some filesystems, `.../path/*` can be a more efficient way to read all files * in a directory rather than `.../path/` or `.../path` + * + * @param path Directory to the input data files, the path can be comma separated paths as the + * list of inputs. * @param minPartitions A suggestion value of the minimal splitting number for input data. */ @Experimental @@ -918,8 +928,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * '''Note:''' We ensure that the byte array for each record in the resulting RDD * has the provided record length. * - * @param path Directory to the input data files + * @param path Directory to the input data files, the path can be comma separated paths as the + * list of inputs. * @param recordLength The length at which to split the records + * @param conf Configuration for setting up the dataset. + * * @return An RDD of data with values, represented as byte arrays */ @Experimental From 7f1e507bf7e82bff323c5dec3c1ee044687c4173 Mon Sep 17 00:00:00 2001 From: ehnalis Date: Tue, 25 Aug 2015 12:30:06 +0100 Subject: [PATCH 079/232] Fixed a typo in DAGScheduler. Author: ehnalis Closes #8308 from ehnalis/master. --- .../apache/spark/scheduler/DAGScheduler.scala | 27 ++++++++++++++----- 1 file changed, 20 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 684db6646765f..daf9b0f95273e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -152,17 +152,24 @@ class DAGScheduler( // may lead to more delay in scheduling if those locations are busy. private[scheduler] val REDUCER_PREF_LOCS_FRACTION = 0.2 - // Called by TaskScheduler to report task's starting. + /** + * Called by the TaskSetManager to report task's starting. + */ def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessLoop.post(BeginEvent(task, taskInfo)) } - // Called to report that a task has completed and results are being fetched remotely. + /** + * Called by the TaskSetManager to report that a task has completed + * and results are being fetched remotely. + */ def taskGettingResult(taskInfo: TaskInfo) { eventProcessLoop.post(GettingResultEvent(taskInfo)) } - // Called by TaskScheduler to report task completions or failures. + /** + * Called by the TaskSetManager to report task completions or failures. + */ def taskEnded( task: Task[_], reason: TaskEndReason, @@ -188,18 +195,24 @@ class DAGScheduler( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } - // Called by TaskScheduler when an executor fails. + /** + * Called by TaskScheduler implementation when an executor fails. + */ def executorLost(execId: String): Unit = { eventProcessLoop.post(ExecutorLost(execId)) } - // Called by TaskScheduler when a host is added + /** + * Called by TaskScheduler implementation when a host is added. + */ def executorAdded(execId: String, host: String): Unit = { eventProcessLoop.post(ExecutorAdded(execId, host)) } - // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or - // cancellation of the job itself. + /** + * Called by the TaskSetManager to cancel an entire TaskSet due to either repeated failures or + * cancellation of the job itself. + */ def taskSetFailed(taskSet: TaskSet, reason: String, exception: Option[Throwable]): Unit = { eventProcessLoop.post(TaskSetFailed(taskSet, reason, exception)) } From 69c9c177160e32a2fbc9b36ecc52156077fca6fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 25 Aug 2015 12:33:13 +0100 Subject: [PATCH 080/232] [SPARK-9613] [CORE] Ban use of JavaConversions and migrate all existing uses to JavaConverters Replace `JavaConversions` implicits with `JavaConverters` Most occurrences I've seen so far are necessary conversions; a few have been avoidable. None are in critical code as far as I see, yet. Author: Sean Owen Closes #8033 from srowen/SPARK-9613. --- .../shuffle/unsafe/UnsafeShuffleWriter.java | 4 +- .../org/apache/spark/MapOutputTracker.scala | 4 +- .../scala/org/apache/spark/SSLOptions.scala | 11 +- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../scala/org/apache/spark/TestUtils.scala | 9 +- .../apache/spark/api/java/JavaHadoopRDD.scala | 4 +- .../spark/api/java/JavaNewHadoopRDD.scala | 4 +- .../apache/spark/api/java/JavaPairRDD.scala | 19 ++- .../apache/spark/api/java/JavaRDDLike.scala | 75 +++++------- .../spark/api/java/JavaSparkContext.scala | 20 ++-- .../spark/api/python/PythonHadoopUtil.scala | 28 ++--- .../apache/spark/api/python/PythonRDD.scala | 26 ++--- .../apache/spark/api/python/PythonUtils.scala | 15 ++- .../api/python/PythonWorkerFactory.scala | 11 +- .../apache/spark/api/python/SerDeUtil.scala | 3 +- .../WriteInputFormatTestDataGenerator.scala | 8 +- .../scala/org/apache/spark/api/r/RRDD.scala | 13 ++- .../scala/org/apache/spark/api/r/RUtils.scala | 5 +- .../scala/org/apache/spark/api/r/SerDe.scala | 4 +- .../spark/broadcast/TorrentBroadcast.scala | 4 +- .../spark/deploy/ExternalShuffleService.scala | 8 +- .../apache/spark/deploy/PythonRunner.scala | 4 +- .../apache/spark/deploy/RPackageUtils.scala | 4 +- .../org/apache/spark/deploy/RRunner.scala | 4 +- .../spark/deploy/SparkCuratorUtil.scala | 4 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 19 +-- .../spark/deploy/SparkSubmitArguments.scala | 6 +- .../master/ZooKeeperPersistenceEngine.scala | 6 +- .../spark/deploy/worker/CommandUtils.scala | 5 +- .../spark/deploy/worker/DriverRunner.scala | 8 +- .../spark/deploy/worker/ExecutorRunner.scala | 7 +- .../apache/spark/deploy/worker/Worker.scala | 1 - .../org/apache/spark/executor/Executor.scala | 6 +- .../spark/executor/ExecutorSource.scala | 4 +- .../spark/executor/MesosExecutorBackend.scala | 6 +- .../spark/input/PortableDataStream.scala | 11 +- .../input/WholeTextFileInputFormat.scala | 8 +- .../spark/launcher/WorkerCommandBuilder.scala | 4 +- .../apache/spark/metrics/MetricsConfig.scala | 22 ++-- .../network/netty/NettyBlockRpcServer.scala | 4 +- .../netty/NettyBlockTransferService.scala | 6 +- .../apache/spark/network/nio/Connection.scala | 4 +- .../spark/partial/GroupedCountEvaluator.scala | 10 +- .../spark/partial/GroupedMeanEvaluator.scala | 10 +- .../spark/partial/GroupedSumEvaluator.scala | 10 +- .../apache/spark/rdd/PairRDDFunctions.scala | 6 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 6 +- .../org/apache/spark/rdd/SubtractedRDD.scala | 4 +- .../spark/scheduler/InputFormatInfo.scala | 4 +- .../org/apache/spark/scheduler/Pool.scala | 10 +- .../mesos/CoarseMesosSchedulerBackend.scala | 20 ++-- .../mesos/MesosClusterPersistenceEngine.scala | 4 +- .../cluster/mesos/MesosClusterScheduler.scala | 14 +-- .../cluster/mesos/MesosSchedulerBackend.scala | 22 ++-- .../cluster/mesos/MesosSchedulerUtils.scala | 25 ++-- .../spark/serializer/KryoSerializer.scala | 10 +- .../shuffle/FileShuffleBlockResolver.scala | 8 +- .../storage/BlockManagerMasterEndpoint.scala | 8 +- .../org/apache/spark/util/AkkaUtils.scala | 4 +- .../org/apache/spark/util/ListenerBus.scala | 7 +- .../spark/util/MutableURLClassLoader.scala | 2 - .../spark/util/TimeStampedHashMap.scala | 10 +- .../spark/util/TimeStampedHashSet.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 20 ++-- .../apache/spark/util/collection/Utils.scala | 4 +- .../java/org/apache/spark/JavaAPISuite.java | 6 +- .../org/apache/spark/SparkConfSuite.scala | 7 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 1 - .../spark/deploy/RPackageUtilsSuite.scala | 8 +- .../deploy/worker/ExecutorRunnerTest.scala | 5 +- .../spark/scheduler/SparkListenerSuite.scala | 9 +- .../mesos/MesosSchedulerBackendSuite.scala | 15 +-- .../serializer/KryoSerializerSuite.scala | 3 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 21 ++-- .../spark/examples/CassandraCQLTest.scala | 15 +-- .../apache/spark/examples/CassandraTest.scala | 6 +- .../spark/examples/DriverSubmissionTest.scala | 6 +- .../pythonconverters/AvroConverters.scala | 16 +-- .../CassandraConverters.scala | 14 ++- .../pythonconverters/HBaseConverters.scala | 5 +- .../streaming/flume/sink/SparkSinkSuite.scala | 4 +- .../streaming/flume/EventTransformer.scala | 4 +- .../streaming/flume/FlumeBatchFetcher.scala | 3 +- .../streaming/flume/FlumeInputDStream.scala | 7 +- .../flume/FlumePollingInputDStream.scala | 6 +- .../streaming/flume/FlumeTestUtils.scala | 10 +- .../spark/streaming/flume/FlumeUtils.scala | 8 +- .../flume/PollingFlumeTestUtils.scala | 16 ++- .../flume/FlumePollingStreamSuite.scala | 8 +- .../streaming/flume/FlumeStreamSuite.scala | 2 +- .../streaming/kafka/KafkaTestUtils.scala | 4 +- .../spark/streaming/kafka/KafkaUtils.scala | 35 +++--- .../spark/streaming/zeromq/ZeroMQUtils.scala | 15 ++- .../kinesis/KinesisBackedBlockRDD.scala | 4 +- .../streaming/kinesis/KinesisReceiver.scala | 4 +- .../streaming/kinesis/KinesisTestUtils.scala | 3 +- .../kinesis/KinesisReceiverSuite.scala | 12 +- .../mllib/util/LinearDataGenerator.scala | 4 +- .../ml/classification/JavaOneVsRestSuite.java | 7 +- .../LogisticRegressionSuite.scala | 4 +- .../spark/mllib/classification/SVMSuite.scala | 4 +- .../optimization/GradientDescentSuite.scala | 4 +- .../spark/mllib/recommendation/ALSSuite.scala | 4 +- project/SparkBuild.scala | 8 +- python/pyspark/sql/column.py | 12 ++ python/pyspark/sql/dataframe.py | 4 +- scalastyle-config.xml | 7 ++ .../main/scala/org/apache/spark/sql/Row.scala | 12 +- .../spark/sql/catalyst/analysis/Catalog.scala | 4 +- .../spark/sql/DataFrameNaFunctions.scala | 8 +- .../apache/spark/sql/DataFrameReader.scala | 4 +- .../apache/spark/sql/DataFrameWriter.scala | 4 +- .../org/apache/spark/sql/GroupedData.scala | 4 +- .../scala/org/apache/spark/sql/SQLConf.scala | 13 ++- .../org/apache/spark/sql/SQLContext.scala | 8 +- .../datasources/ResolvedDataSource.scala | 4 +- .../parquet/CatalystReadSupport.scala | 8 +- .../parquet/CatalystRowConverter.scala | 4 +- .../parquet/CatalystSchemaConverter.scala | 4 +- .../datasources/parquet/ParquetRelation.scala | 13 ++- .../parquet/ParquetTypesConverter.scala | 4 +- .../joins/ShuffledHashOuterJoin.scala | 6 +- .../spark/sql/execution/pythonUDFs.scala | 11 +- .../apache/spark/sql/JavaDataFrameSuite.java | 8 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 6 +- .../org/apache/spark/sql/QueryTest.scala | 4 +- .../ParquetAvroCompatibilitySuite.scala | 3 +- .../parquet/ParquetCompatibilityTest.scala | 7 +- .../datasources/parquet/ParquetIOSuite.scala | 25 ++-- .../SparkExecuteStatementOperation.scala | 10 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 16 +-- .../thriftserver/SparkSQLCLIService.scala | 6 +- .../hive/thriftserver/SparkSQLDriver.scala | 14 +-- .../sql/hive/thriftserver/SparkSQLEnv.scala | 4 +- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 40 +++---- .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 +- .../org/apache/spark/sql/hive/HiveQl.scala | 110 ++++++++++-------- .../org/apache/spark/sql/hive/HiveShim.scala | 5 +- .../spark/sql/hive/client/ClientWrapper.scala | 27 ++--- .../spark/sql/hive/client/HiveShim.scala | 14 +-- .../execution/DescribeHiveTableCommand.scala | 8 +- .../sql/hive/execution/HiveTableScan.scala | 9 +- .../hive/execution/InsertIntoHiveTable.scala | 12 +- .../hive/execution/ScriptTransformation.scala | 12 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 9 +- .../spark/sql/hive/orc/OrcRelation.scala | 8 +- .../apache/spark/sql/hive/test/TestHive.scala | 11 +- .../spark/sql/hive/client/FiltersSuite.scala | 4 +- .../sql/hive/execution/HiveUDFSuite.scala | 29 +++-- .../sql/hive/execution/PruningSuite.scala | 7 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../sql/sources/hadoopFsRelationSuites.scala | 8 +- .../streaming/api/java/JavaDStreamLike.scala | 12 +- .../streaming/api/java/JavaPairDStream.scala | 28 ++--- .../api/java/JavaStreamingContext.scala | 32 ++--- .../streaming/api/python/PythonDStream.scala | 5 +- .../spark/streaming/receiver/Receiver.scala | 6 +- .../streaming/scheduler/JobScheduler.scala | 4 +- .../scheduler/ReceivedBlockTracker.scala | 4 +- .../util/FileBasedWriteAheadLog.scala | 4 +- .../spark/streaming/JavaTestUtils.scala | 24 ++-- .../streaming/util/WriteAheadLogSuite.scala | 4 +- .../spark/tools/GenerateMIMAIgnore.scala | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 13 ++- .../spark/deploy/yarn/ExecutorRunnable.scala | 24 ++-- .../spark/deploy/yarn/YarnAllocator.scala | 19 ++- .../spark/deploy/yarn/YarnRMClient.scala | 8 +- .../deploy/yarn/BaseYarnClusterSuite.scala | 6 +- .../spark/deploy/yarn/ClientSuite.scala | 8 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 5 +- 171 files changed, 863 insertions(+), 880 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java index 2389c28b28395..fdb309e365f69 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java @@ -24,7 +24,7 @@ import scala.Option; import scala.Product2; -import scala.collection.JavaConversions; +import scala.collection.JavaConverters; import scala.collection.immutable.Map; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; @@ -160,7 +160,7 @@ public long getPeakMemoryUsedBytes() { */ @VisibleForTesting public void write(Iterator> records) throws IOException { - write(JavaConversions.asScalaIterator(records)); + write(JavaConverters.asScalaIteratorConverter(records).asScala()); } @Override diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 92218832d256f..a387592783850 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -21,8 +21,8 @@ import java.io._ import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} -import scala.collection.JavaConversions._ import scala.reflect.ClassTag import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, RpcEndpoint} @@ -398,7 +398,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { protected val mapStatuses: Map[Int, Array[MapStatus]] = - new ConcurrentHashMap[Int, Array[MapStatus]] + new ConcurrentHashMap[Int, Array[MapStatus]]().asScala } private[spark] object MapOutputTracker extends Logging { diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 32df42d57dbd6..3b9c885bf97a7 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -17,9 +17,11 @@ package org.apache.spark -import java.io.{File, FileInputStream} -import java.security.{KeyStore, NoSuchAlgorithmException} -import javax.net.ssl.{KeyManager, KeyManagerFactory, SSLContext, TrustManager, TrustManagerFactory} +import java.io.File +import java.security.NoSuchAlgorithmException +import javax.net.ssl.SSLContext + +import scala.collection.JavaConverters._ import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} import org.eclipse.jetty.util.ssl.SslContextFactory @@ -79,7 +81,6 @@ private[spark] case class SSLOptions( * object. It can be used then to compose the ultimate Akka configuration. */ def createAkkaConfig: Option[Config] = { - import scala.collection.JavaConversions._ if (enabled) { Some(ConfigFactory.empty() .withValue("akka.remote.netty.tcp.security.key-store", @@ -97,7 +98,7 @@ private[spark] case class SSLOptions( .withValue("akka.remote.netty.tcp.security.protocol", ConfigValueFactory.fromAnyRef(protocol.getOrElse(""))) .withValue("akka.remote.netty.tcp.security.enabled-algorithms", - ConfigValueFactory.fromIterable(supportedAlgorithms.toSeq)) + ConfigValueFactory.fromIterable(supportedAlgorithms.asJava)) .withValue("akka.remote.netty.tcp.enable-ssl", ConfigValueFactory.fromAnyRef(true))) } else { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9849aff85d72e..f3da04a7f55d0 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,8 +26,8 @@ import java.util.{Arrays, Properties, UUID} import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean, AtomicInteger} import java.util.UUID.randomUUID +import scala.collection.JavaConverters._ import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} @@ -1546,7 +1546,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def getAllPools: Seq[Schedulable] = { assertNotStopped() // TODO(xiajunluan): We should take nested pools into account - taskScheduler.rootPool.schedulableQueue.toSeq + taskScheduler.rootPool.schedulableQueue.asScala.toSeq } /** diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index a1ebbecf93b7b..888763a3e8ebf 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -19,11 +19,12 @@ package org.apache.spark import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} import java.net.{URI, URL} +import java.nio.charset.StandardCharsets +import java.util.Arrays import java.util.jar.{JarEntry, JarOutputStream} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.{ByteStreams, Files} import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} @@ -71,7 +72,7 @@ private[spark] object TestUtils { files.foreach { case (k, v) => val entry = new JarEntry(k) jarStream.putNextEntry(entry) - ByteStreams.copy(new ByteArrayInputStream(v.getBytes(UTF_8)), jarStream) + ByteStreams.copy(new ByteArrayInputStream(v.getBytes(StandardCharsets.UTF_8)), jarStream) } jarStream.close() jarFile.toURI.toURL @@ -125,7 +126,7 @@ private[spark] object TestUtils { } else { Seq() } - compiler.getTask(null, null, null, options, null, Seq(sourceFile)).call() + compiler.getTask(null, null, null, options.asJava, null, Arrays.asList(sourceFile)).call() val fileName = className + ".class" val result = new File(fileName) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala index 0ae0b4ec042e2..891bcddeac286 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.hadoop.mapred.InputSplit @@ -37,7 +37,7 @@ class JavaHadoopRDD[K, V](rdd: HadoopRDD[K, V]) def mapPartitionsWithInputSplit[R]( f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = { - new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, b.asJava).asScala, preservesPartitioning)(fakeClassTag))(fakeClassTag) } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala index ec4f3964d75e0..0f49279f3e647 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.hadoop.mapreduce.InputSplit @@ -37,7 +37,7 @@ class JavaNewHadoopRDD[K, V](rdd: NewHadoopRDD[K, V]) def mapPartitionsWithInputSplit[R]( f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = { - new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, b.asJava).asScala, preservesPartitioning)(fakeClassTag))(fakeClassTag) } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 8441bb3a3047e..fb787979c1820 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList, Map => JMap} import java.lang.{Iterable => JIterable} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -142,7 +142,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], seed: Long): JavaPairRDD[K, V] = - new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, seed)) + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions.asScala, seed)) /** * Return a subset of this RDD sampled by key (via stratified sampling). @@ -173,7 +173,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double], seed: Long): JavaPairRDD[K, V] = - new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions, seed)) + new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions.asScala, seed)) /** * ::Experimental:: @@ -768,7 +768,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return the list of values in the RDD for key `key`. This operation is done efficiently if the * RDD has a known partitioner by only searching the partition that the key maps to. */ - def lookup(key: K): JList[V] = seqAsJavaList(rdd.lookup(key)) + def lookup(key: K): JList[V] = rdd.lookup(key).asJava /** Output the RDD to any Hadoop-supported file system. */ def saveAsHadoopFile[F <: OutputFormat[_, _]]( @@ -987,30 +987,27 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { - rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) + rddToPairRDDFunctions(rdd).mapValues(_.asJava) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) + rddToPairRDDFunctions(rdd).mapValues(x => (x._1.asJava, x._2.asJava)) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { - rddToPairRDDFunctions(rdd) - .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) + rddToPairRDDFunctions(rdd).mapValues(x => (x._1.asJava, x._2.asJava, x._3.asJava)) } private[spark] def cogroupResult3ToJava[K: ClassTag, V, W1, W2, W3]( rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))]) : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2], JIterable[W3]))] = { - rddToPairRDDFunctions(rdd) - .mapValues(x => - (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3), asJavaIterable(x._4))) + rddToPairRDDFunctions(rdd).mapValues(x => (x._1.asJava, x._2.asJava, x._3.asJava, x._4.asJava)) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index c582488f16fe7..fc817cdd6a3f8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -21,7 +21,6 @@ import java.{lang => jl} import java.lang.{Iterable => JIterable, Long => JLong} import java.util.{Comparator, List => JList, Iterator => JIterator} -import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -59,10 +58,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def rdd: RDD[T] @deprecated("Use partitions() instead.", "1.1.0") - def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) + def splits: JList[Partition] = rdd.partitions.toSeq.asJava /** Set of partitions in this RDD. */ - def partitions: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) + def partitions: JList[Partition] = rdd.partitions.toSeq.asJava /** The partitioner of this RDD. */ def partitioner: Optional[Partitioner] = JavaUtils.optionToOptional(rdd.partitioner) @@ -82,7 +81,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * subclasses of RDD. */ def iterator(split: Partition, taskContext: TaskContext): java.util.Iterator[T] = - asJavaIterator(rdd.iterator(split, taskContext)) + rdd.iterator(split, taskContext).asJava // Transformations (return a new RDD) @@ -99,7 +98,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitionsWithIndex[R]( f: JFunction2[jl.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = - new JavaRDD(rdd.mapPartitionsWithIndex(((a, b) => f(a, asJavaIterator(b))), + new JavaRDD(rdd.mapPartitionsWithIndex((a, b) => f.call(a, b.asJava).asScala, preservesPartitioning)(fakeClassTag))(fakeClassTag) /** @@ -153,7 +152,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaRDD[U] = { def fn: (Iterator[T]) => Iterator[U] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } JavaRDD.fromRDD(rdd.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -164,7 +163,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { def fn: (Iterator[T]) => Iterator[U] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } JavaRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning)(fakeClassTag[U]))(fakeClassTag[U]) @@ -175,7 +174,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = { def fn: (Iterator[T]) => Iterator[jl.Double] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: jl.Double) => x.doubleValue())) } @@ -186,7 +185,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): JavaPairRDD[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -197,7 +196,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]], preservesPartitioning: Boolean): JavaDoubleRDD = { def fn: (Iterator[T]) => Iterator[jl.Double] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) .map(x => x.doubleValue())) @@ -209,7 +208,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } JavaPairRDD.fromRDD( rdd.mapPartitions(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) @@ -219,14 +218,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Applies a function f to each partition of this RDD. */ def foreachPartition(f: VoidFunction[java.util.Iterator[T]]) { - rdd.foreachPartition((x => f.call(asJavaIterator(x)))) + rdd.foreachPartition((x => f.call(x.asJava))) } /** * Return an RDD created by coalescing all elements within each partition into an array. */ def glom(): JavaRDD[JList[T]] = - new JavaRDD(rdd.glom().map(x => new java.util.ArrayList[T](x.toSeq))) + new JavaRDD(rdd.glom().map(_.toSeq.asJava)) /** * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of @@ -266,13 +265,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD created by piping elements to a forked external process. */ def pipe(command: JList[String]): JavaRDD[String] = - rdd.pipe(asScalaBuffer(command)) + rdd.pipe(command.asScala) /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: JList[String], env: java.util.Map[String, String]): JavaRDD[String] = - rdd.pipe(asScalaBuffer(command), mapAsScalaMap(env)) + rdd.pipe(command.asScala, env.asScala) /** * Zips this RDD with another one, returning key-value pairs with the first element in each RDD, @@ -294,8 +293,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { other: JavaRDDLike[U, _], f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V]): JavaRDD[V] = { def fn: (Iterator[T], Iterator[U]) => Iterator[V] = { - (x: Iterator[T], y: Iterator[U]) => asScalaIterator( - f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) + (x: Iterator[T], y: Iterator[U]) => f.call(x.asJava, y.asJava).iterator().asScala } JavaRDD.fromRDD( rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) @@ -333,22 +331,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in this RDD. */ - def collect(): JList[T] = { - import scala.collection.JavaConversions._ - val arr: java.util.Collection[T] = rdd.collect().toSeq - new java.util.ArrayList(arr) - } + def collect(): JList[T] = + rdd.collect().toSeq.asJava /** * Return an iterator that contains all of the elements in this RDD. * * The iterator will consume as much memory as the largest partition in this RDD. */ - def toLocalIterator(): JIterator[T] = { - import scala.collection.JavaConversions._ - rdd.toLocalIterator - } - + def toLocalIterator(): JIterator[T] = + asJavaIteratorConverter(rdd.toLocalIterator).asJava /** * Return an array that contains all of the elements in this RDD. @@ -363,9 +355,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = { // This is useful for implementing `take` from other language frontends // like Python where the data is serialized. - import scala.collection.JavaConversions._ val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds) - res.map(x => new java.util.ArrayList(x.toSeq)).toArray + res.map(_.toSeq.asJava) } /** @@ -489,20 +480,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * it will be slow if a lot of partitions are required. In that case, use collect() to get the * whole RDD instead. */ - def take(num: Int): JList[T] = { - import scala.collection.JavaConversions._ - val arr: java.util.Collection[T] = rdd.take(num).toSeq - new java.util.ArrayList(arr) - } + def take(num: Int): JList[T] = + rdd.take(num).toSeq.asJava def takeSample(withReplacement: Boolean, num: Int): JList[T] = takeSample(withReplacement, num, Utils.random.nextLong) - def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = { - import scala.collection.JavaConversions._ - val arr: java.util.Collection[T] = rdd.takeSample(withReplacement, num, seed).toSeq - new java.util.ArrayList(arr) - } + def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = + rdd.takeSample(withReplacement, num, seed).toSeq.asJava /** * Return the first element in this RDD. @@ -582,10 +567,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * @return an array of top elements */ def top(num: Int, comp: Comparator[T]): JList[T] = { - import scala.collection.JavaConversions._ - val topElems = rdd.top(num)(Ordering.comparatorToOrdering(comp)) - val arr: java.util.Collection[T] = topElems.toSeq - new java.util.ArrayList(arr) + rdd.top(num)(Ordering.comparatorToOrdering(comp)).toSeq.asJava } /** @@ -607,10 +589,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * @return an array of top elements */ def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = { - import scala.collection.JavaConversions._ - val topElems = rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)) - val arr: java.util.Collection[T] = topElems.toSeq - new java.util.ArrayList(arr) + rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)).toSeq.asJava } /** @@ -696,7 +675,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * applies a function f to each partition of this RDD. */ def foreachPartitionAsync(f: VoidFunction[java.util.Iterator[T]]): JavaFutureAction[Void] = { - new JavaFutureActionWrapper[Unit, Void](rdd.foreachPartitionAsync(x => f.call(x)), + new JavaFutureActionWrapper[Unit, Void](rdd.foreachPartitionAsync(x => f.call(x.asJava)), { x => null.asInstanceOf[Void] }) } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 02e49a853c5f7..609496ccdfef1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -21,8 +21,7 @@ import java.io.Closeable import java.util import java.util.{Map => JMap} -import scala.collection.JavaConversions -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -104,7 +103,7 @@ class JavaSparkContext(val sc: SparkContext) */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment.asScala, Map())) private[spark] val env = sc.env @@ -118,7 +117,7 @@ class JavaSparkContext(val sc: SparkContext) def appName: String = sc.appName - def jars: util.List[String] = sc.jars + def jars: util.List[String] = sc.jars.asJava def startTime: java.lang.Long = sc.startTime @@ -142,7 +141,7 @@ class JavaSparkContext(val sc: SparkContext) /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag - sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) + sc.parallelize(list.asScala, numSlices) } /** Get an RDD that has no partitions or elements. */ @@ -161,7 +160,7 @@ class JavaSparkContext(val sc: SparkContext) : JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[V] = fakeClassTag - JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) + JavaPairRDD.fromRDD(sc.parallelize(list.asScala, numSlices)) } /** Distribute a local Scala collection to form an RDD. */ @@ -170,8 +169,7 @@ class JavaSparkContext(val sc: SparkContext) /** Distribute a local Scala collection to form an RDD. */ def parallelizeDoubles(list: java.util.List[java.lang.Double], numSlices: Int): JavaDoubleRDD = - JavaDoubleRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list).map(_.doubleValue()), - numSlices)) + JavaDoubleRDD.fromRDD(sc.parallelize(list.asScala.map(_.doubleValue()), numSlices)) /** Distribute a local Scala collection to form an RDD. */ def parallelizeDoubles(list: java.util.List[java.lang.Double]): JavaDoubleRDD = @@ -519,7 +517,7 @@ class JavaSparkContext(val sc: SparkContext) /** Build the union of two or more RDDs. */ override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = { - val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) + val rdds: Seq[RDD[T]] = (Seq(first) ++ rest.asScala).map(_.rdd) implicit val ctag: ClassTag[T] = first.classTag sc.union(rdds) } @@ -527,7 +525,7 @@ class JavaSparkContext(val sc: SparkContext) /** Build the union of two or more RDDs. */ override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]]) : JavaPairRDD[K, V] = { - val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd) + val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ rest.asScala).map(_.rdd) implicit val ctag: ClassTag[(K, V)] = first.classTag implicit val ctagK: ClassTag[K] = first.kClassTag implicit val ctagV: ClassTag[V] = first.vClassTag @@ -536,7 +534,7 @@ class JavaSparkContext(val sc: SparkContext) /** Build the union of two or more RDDs. */ override def union(first: JavaDoubleRDD, rest: java.util.List[JavaDoubleRDD]): JavaDoubleRDD = { - val rdds: Seq[RDD[Double]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.srdd) + val rdds: Seq[RDD[Double]] = (Seq(first) ++ rest.asScala).map(_.srdd) new JavaDoubleRDD(sc.union(rdds)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index b959b683d1674..a7dfa1d257cf2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -17,15 +17,17 @@ package org.apache.spark.api.python -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD -import org.apache.spark.util.{SerializableConfiguration, Utils} -import org.apache.spark.{Logging, SparkException} +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io._ -import scala.util.{Failure, Success, Try} -import org.apache.spark.annotation.Experimental +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.annotation.Experimental +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.util.{SerializableConfiguration, Utils} /** * :: Experimental :: @@ -68,7 +70,6 @@ private[python] class WritableToJavaConverter( * object representation */ private def convertWritable(writable: Writable): Any = { - import collection.JavaConversions._ writable match { case iw: IntWritable => iw.get() case dw: DoubleWritable => dw.get() @@ -89,9 +90,7 @@ private[python] class WritableToJavaConverter( aw.get().map(convertWritable(_)) case mw: MapWritable => val map = new java.util.HashMap[Any, Any]() - mw.foreach { case (k, v) => - map.put(convertWritable(k), convertWritable(v)) - } + mw.asScala.foreach { case (k, v) => map.put(convertWritable(k), convertWritable(v)) } map case w: Writable => WritableUtils.clone(w, conf.value.value) case other => other @@ -122,7 +121,6 @@ private[python] class JavaToWritableConverter extends Converter[Any, Writable] { * supported out-of-the-box. */ private def convertToWritable(obj: Any): Writable = { - import collection.JavaConversions._ obj match { case i: java.lang.Integer => new IntWritable(i) case d: java.lang.Double => new DoubleWritable(d) @@ -134,7 +132,7 @@ private[python] class JavaToWritableConverter extends Converter[Any, Writable] { case null => NullWritable.get() case map: java.util.Map[_, _] => val mapWritable = new MapWritable() - map.foreach { case (k, v) => + map.asScala.foreach { case (k, v) => mapWritable.put(convertToWritable(k), convertToWritable(v)) } mapWritable @@ -161,9 +159,8 @@ private[python] object PythonHadoopUtil { * Convert a [[java.util.Map]] of properties to a [[org.apache.hadoop.conf.Configuration]] */ def mapToConf(map: java.util.Map[String, String]): Configuration = { - import collection.JavaConversions._ val conf = new Configuration() - map.foreach{ case (k, v) => conf.set(k, v) } + map.asScala.foreach { case (k, v) => conf.set(k, v) } conf } @@ -172,9 +169,8 @@ private[python] object PythonHadoopUtil { * any matching keys in left */ def mergeConfs(left: Configuration, right: Configuration): Configuration = { - import collection.JavaConversions._ val copy = new Configuration(left) - right.iterator().foreach(entry => copy.set(entry.getKey, entry.getValue)) + right.asScala.foreach(entry => copy.set(entry.getKey, entry.getValue)) copy } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 2a56bf28d7027..b4d152b336602 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -21,7 +21,7 @@ import java.io._ import java.net._ import java.util.{Collections, ArrayList => JArrayList, List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials @@ -66,11 +66,11 @@ private[spark] class PythonRDD( val env = SparkEnv.get val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") - envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread + envVars.put("SPARK_LOCAL_DIRS", localdir) // it's also used in monitor thread if (reuse_worker) { - envVars += ("SPARK_REUSE_WORKER" -> "1") + envVars.put("SPARK_REUSE_WORKER", "1") } - val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) + val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap) // Whether is the worker released into idle pool @volatile var released = false @@ -150,7 +150,7 @@ private[spark] class PythonRDD( // Check whether the worker is ready to be re-used. if (stream.readInt() == SpecialLengths.END_OF_STREAM) { if (reuse_worker) { - env.releasePythonWorker(pythonExec, envVars.toMap, worker) + env.releasePythonWorker(pythonExec, envVars.asScala.toMap, worker) released = true } } @@ -217,13 +217,13 @@ private[spark] class PythonRDD( // sparkFilesDir PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { + dataOut.writeInt(pythonIncludes.size()) + for (include <- pythonIncludes.asScala) { PythonRDD.writeUTF(include, dataOut) } // Broadcast variables val oldBids = PythonRDD.getWorkerBroadcasts(worker) - val newBids = broadcastVars.map(_.id).toSet + val newBids = broadcastVars.asScala.map(_.id).toSet // number of different broadcasts val toRemove = oldBids.diff(newBids) val cnt = toRemove.size + newBids.diff(oldBids).size @@ -233,7 +233,7 @@ private[spark] class PythonRDD( dataOut.writeLong(- bid - 1) // bid >= 0 oldBids.remove(bid) } - for (broadcast <- broadcastVars) { + for (broadcast <- broadcastVars.asScala) { if (!oldBids.contains(broadcast.id)) { // send new broadcast dataOut.writeLong(broadcast.id) @@ -287,7 +287,7 @@ private[spark] class PythonRDD( if (!context.isCompleted) { try { logWarning("Incomplete task interrupted: Attempting to kill Python Worker") - env.destroyPythonWorker(pythonExec, envVars.toMap, worker) + env.destroyPythonWorker(pythonExec, envVars.asScala.toMap, worker) } catch { case e: Exception => logError("Exception when trying to kill worker", e) @@ -358,10 +358,10 @@ private[spark] object PythonRDD extends Logging { type ByteArray = Array[Byte] type UnrolledPartition = Array[ByteArray] val allPartitions: Array[UnrolledPartition] = - sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions) + sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions.asScala) val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*) serveIterator(flattenedPartition.iterator, - s"serve RDD ${rdd.id} with partitions ${partitions.mkString(",")}") + s"serve RDD ${rdd.id} with partitions ${partitions.asScala.mkString(",")}") } /** @@ -819,7 +819,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: val in = socket.getInputStream val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) - for (array <- val2) { + for (array <- val2.asScala) { out.writeInt(array.length) out.write(array) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 90dacaeb93429..31e534f160eeb 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -17,10 +17,10 @@ package org.apache.spark.api.python -import java.io.{File} +import java.io.File import java.util.{List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext @@ -51,7 +51,14 @@ private[spark] object PythonUtils { * Convert list of T into seq of T (for calling API with varargs) */ def toSeq[T](vs: JList[T]): Seq[T] = { - vs.toList.toSeq + vs.asScala + } + + /** + * Convert list of T into a (Scala) List of T + */ + def toList[T](vs: JList[T]): List[T] = { + vs.asScala.toList } /** @@ -65,6 +72,6 @@ private[spark] object PythonUtils { * Convert java map of K, V into Map of K, V (for calling API with varargs) */ def toScalaMap[K, V](jm: java.util.Map[K, V]): Map[K, V] = { - jm.toMap + jm.asScala.toMap } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index e314408c067e9..7039b734d2e40 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -19,9 +19,10 @@ package org.apache.spark.api.python import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} +import java.util.Arrays import scala.collection.mutable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark._ import org.apache.spark.util.{RedirectThread, Utils} @@ -108,9 +109,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.worker")) + val pb = new ProcessBuilder(Arrays.asList(pythonExec, "-m", "pyspark.worker")) val workerEnv = pb.environment() - workerEnv.putAll(envVars) + workerEnv.putAll(envVars.asJava) workerEnv.put("PYTHONPATH", pythonPath) // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: workerEnv.put("PYTHONUNBUFFERED", "YES") @@ -151,9 +152,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.daemon")) + val pb = new ProcessBuilder(Arrays.asList(pythonExec, "-m", "pyspark.daemon")) val workerEnv = pb.environment() - workerEnv.putAll(envVars) + workerEnv.putAll(envVars.asJava) workerEnv.put("PYTHONPATH", pythonPath) // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: workerEnv.put("PYTHONUNBUFFERED", "YES") diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 1f1debcf84ad4..fd27276e70bfe 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -22,7 +22,6 @@ import java.util.{ArrayList => JArrayList} import org.apache.spark.api.java.JavaRDD -import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Failure @@ -214,7 +213,7 @@ private[spark] object SerDeUtil extends Logging { new AutoBatchedPickler(cleaned) } else { val pickle = new Pickler - cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched))) + cleaned.grouped(batchSize).map(batched => pickle.dumps(batched.asJava)) } } } diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index 8f30ff9202c83..ee1fb056f0d96 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -20,6 +20,8 @@ package org.apache.spark.api.python import java.io.{DataOutput, DataInput} import java.{util => ju} +import scala.collection.JavaConverters._ + import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ @@ -62,10 +64,9 @@ private[python] class TestInputKeyConverter extends Converter[Any, Any] { } private[python] class TestInputValueConverter extends Converter[Any, Any] { - import collection.JavaConversions._ override def convert(obj: Any): ju.List[Double] = { val m = obj.asInstanceOf[MapWritable] - seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq) + m.keySet.asScala.map(_.asInstanceOf[DoubleWritable].get()).toSeq.asJava } } @@ -76,9 +77,8 @@ private[python] class TestOutputKeyConverter extends Converter[Any, Any] { } private[python] class TestOutputValueConverter extends Converter[Any, Any] { - import collection.JavaConversions._ override def convert(obj: Any): DoubleWritable = { - new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head) + new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().iterator().next()) } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 1cf2824f862ee..9d5bbb5d609f3 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -19,9 +19,10 @@ package org.apache.spark.api.r import java.io._ import java.net.{InetAddress, ServerSocket} +import java.util.Arrays import java.util.{Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.io.Source import scala.reflect.ClassTag import scala.util.Try @@ -365,11 +366,11 @@ private[r] object RRDD { sparkConf.setIfMissing("spark.master", "local") } - for ((name, value) <- sparkEnvirMap) { - sparkConf.set(name.asInstanceOf[String], value.asInstanceOf[String]) + for ((name, value) <- sparkEnvirMap.asScala) { + sparkConf.set(name.toString, value.toString) } - for ((name, value) <- sparkExecutorEnvMap) { - sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String]) + for ((name, value) <- sparkExecutorEnvMap.asScala) { + sparkConf.setExecutorEnv(name.toString, value.toString) } val jsc = new JavaSparkContext(sparkConf) @@ -395,7 +396,7 @@ private[r] object RRDD { val rOptions = "--vanilla" val rLibDir = RUtils.sparkRPackagePath(isDriver = false) val rExecScript = rLibDir + "/SparkR/worker/" + script - val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript)) + val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript)) // Unset the R_TESTS environment variable for workers. // This is set by R CMD check as startup.Rs // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala index 427b2bc7cbcbb..9e807cc52f18c 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -18,8 +18,7 @@ package org.apache.spark.api.r import java.io.File - -import scala.collection.JavaConversions._ +import java.util.Arrays import org.apache.spark.{SparkEnv, SparkException} @@ -68,7 +67,7 @@ private[spark] object RUtils { /** Check if R is installed before running tests that use R commands. */ def isRInstalled: Boolean = { try { - val builder = new ProcessBuilder(Seq("R", "--version")) + val builder = new ProcessBuilder(Arrays.asList("R", "--version")) builder.start().waitFor() == 0 } catch { case e: Exception => false diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index 3c89f24473744..dbbbcf40c1e96 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -20,7 +20,7 @@ package org.apache.spark.api.r import java.io.{DataInputStream, DataOutputStream} import java.sql.{Timestamp, Date, Time} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ /** * Utility functions to serialize, deserialize objects to / from R @@ -165,7 +165,7 @@ private[spark] object SerDe { val valueType = readObjectType(in) readTypedObject(in, valueType) }) - mapAsJavaMap(keys.zip(values).toMap) + keys.zip(values).toMap.asJava } else { new java.util.HashMap[Object, Object]() } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index a0c9b5e63c744..7e3764d802fe1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -20,7 +20,7 @@ package org.apache.spark.broadcast import java.io._ import java.nio.ByteBuffer -import scala.collection.JavaConversions.asJavaEnumeration +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.Random @@ -210,7 +210,7 @@ private object TorrentBroadcast extends Logging { compressionCodec: Option[CompressionCodec]): T = { require(blocks.nonEmpty, "Cannot unblockify an empty array of blocks") val is = new SequenceInputStream( - asJavaEnumeration(blocks.iterator.map(block => new ByteBufferInputStream(block)))) + blocks.iterator.map(new ByteBufferInputStream(_)).asJavaEnumeration) val in: InputStream = compressionCodec.map(c => c.compressedInputStream(is)).getOrElse(is) val ser = serializer.newInstance() val serIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index 22ef701d833b2..6840a3ae831f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -19,13 +19,13 @@ package org.apache.spark.deploy import java.util.concurrent.CountDownLatch -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.sasl.SaslServerBootstrap -import org.apache.spark.network.server.TransportServer +import org.apache.spark.network.server.{TransportServerBootstrap, TransportServer} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler import org.apache.spark.network.util.TransportConf import org.apache.spark.util.Utils @@ -67,13 +67,13 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana def start() { require(server == null, "Shuffle server already started") logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl") - val bootstraps = + val bootstraps: Seq[TransportServerBootstrap] = if (useSasl) { Seq(new SaslServerBootstrap(transportConf, securityManager)) } else { Nil } - server = transportContext.createServer(port, bootstraps) + server = transportContext.createServer(port, bootstraps.asJava) } /** Clean up all shuffle files associated with an application that has exited. */ diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 23d01e9cbb9f9..d85327603f64d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -21,7 +21,7 @@ import java.net.URI import java.io.File import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Try import org.apache.spark.SparkUserAppException @@ -71,7 +71,7 @@ object PythonRunner { val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) // Launch Python process - val builder = new ProcessBuilder(Seq(pythonExec, formattedPythonFile) ++ otherArgs) + val builder = new ProcessBuilder((Seq(pythonExec, formattedPythonFile) ++ otherArgs).asJava) val env = builder.environment() env.put("PYTHONPATH", pythonPath) // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: diff --git a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala index ed1e972955679..4b28866dcaa7c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala @@ -22,7 +22,7 @@ import java.util.jar.JarFile import java.util.logging.Level import java.util.zip.{ZipEntry, ZipOutputStream} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.io.{ByteStreams, Files} @@ -110,7 +110,7 @@ private[deploy] object RPackageUtils extends Logging { print(s"Building R package with the command: $installCmd", printStream) } try { - val builder = new ProcessBuilder(installCmd) + val builder = new ProcessBuilder(installCmd.asJava) builder.redirectErrorStream(true) val env = builder.environment() env.clear() diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index c0cab22fa8252..05b954ce36998 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy import java.io._ import java.util.concurrent.{Semaphore, TimeUnit} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path @@ -68,7 +68,7 @@ object RRunner { if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { // Launch R val returnCode = try { - val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) + val builder = new ProcessBuilder((Seq(rCommand, rFileNormalized) ++ otherArgs).asJava) val env = builder.environment() env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) val rPackageDir = RUtils.sparkRPackagePath(isDriver = true) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index b8d3993540220..8d5e716e6aea4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry @@ -57,7 +57,7 @@ private[spark] object SparkCuratorUtil extends Logging { def deleteRecursive(zk: CuratorFramework, path: String) { if (zk.checkExists().forPath(path) != null) { - for (child <- zk.getChildren.forPath(path)) { + for (child <- zk.getChildren.forPath(path).asScala) { zk.delete().forPath(path + "/" + child) } zk.delete().forPath(path) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index dda4216c7efe2..f7723ef5bde4c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -22,7 +22,7 @@ import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import java.util.{Arrays, Comparator} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.control.NonFatal @@ -71,7 +71,7 @@ class SparkHadoopUtil extends Logging { } def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { - for (token <- source.getTokens()) { + for (token <- source.getTokens.asScala) { dest.addToken(token) } } @@ -175,8 +175,8 @@ class SparkHadoopUtil extends Logging { } private def getFileSystemThreadStatistics(): Seq[AnyRef] = { - val stats = FileSystem.getAllStatistics() - stats.map(Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) + FileSystem.getAllStatistics.asScala.map( + Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) } private def getFileSystemThreadStatisticsMethod(methodName: String): Method = { @@ -306,12 +306,13 @@ class SparkHadoopUtil extends Logging { val renewalInterval = sparkConf.getLong("spark.yarn.token.renewal.interval", (24 hours).toMillis) - credentials.getAllTokens.filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + credentials.getAllTokens.asScala + .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .map { t => - val identifier = new DelegationTokenIdentifier() - identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - (identifier.getIssueDate + fraction * renewalInterval).toLong - now - }.foldLeft(0L)(math.max) + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + (identifier.getIssueDate + fraction * renewalInterval).toLong - now + }.foldLeft(0L)(math.max) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3f3c6627c21fb..18a1c52ae53fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -23,7 +23,7 @@ import java.net.URI import java.util.{List => JList} import java.util.jar.JarFile -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.io.Source @@ -94,7 +94,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S // Set parameters from command line arguments try { - parse(args.toList) + parse(args.asJava) } catch { case e: IllegalArgumentException => SparkSubmit.printErrorAndExit(e.getMessage()) @@ -458,7 +458,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } override protected def handleExtraArgs(extra: JList[String]): Unit = { - childArgs ++= extra + childArgs ++= extra.asScala } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 563831cc6b8dd..540e802420ce0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import java.nio.ByteBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework @@ -49,8 +49,8 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer } override def read[T: ClassTag](prefix: String): Seq[T] = { - val file = zk.getChildren.forPath(WORKING_DIR).filter(_.startsWith(prefix)) - file.map(deserializeFromFile[T]).flatten + zk.getChildren.forPath(WORKING_DIR).asScala + .filter(_.startsWith(prefix)).map(deserializeFromFile[T]).flatten } override def close() { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 45a3f43045437..ce02ee203a4bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -18,9 +18,8 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} -import java.lang.System._ -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.Map import org.apache.spark.Logging @@ -62,7 +61,7 @@ object CommandUtils extends Logging { // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows val cmd = new WorkerCommandBuilder(sparkHome, memory, command).buildCommand() - cmd.toSeq ++ Seq(command.mainClass) ++ command.arguments + cmd.asScala ++ Seq(command.mainClass) ++ command.arguments } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index ec51c3d935d8e..89159ff5e2b3c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import java.io._ -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -172,8 +172,8 @@ private[deploy] class DriverRunner( CommandUtils.redirectStream(process.getInputStream, stdout) val stderr = new File(baseDir, "stderr") - val header = "Launch Command: %s\n%s\n\n".format( - builder.command.mkString("\"", "\" \"", "\""), "=" * 40) + val formattedCommand = builder.command.asScala.mkString("\"", "\" \"", "\"") + val header = "Launch Command: %s\n%s\n\n".format(formattedCommand, "=" * 40) Files.append(header, stderr, UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } @@ -229,6 +229,6 @@ private[deploy] trait ProcessBuilderLike { private[deploy] object ProcessBuilderLike { def apply(processBuilder: ProcessBuilder): ProcessBuilderLike = new ProcessBuilderLike { override def start(): Process = processBuilder.start() - override def command: Seq[String] = processBuilder.command() + override def command: Seq[String] = processBuilder.command().asScala } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index ab3fea475c2a5..3aef0515cbf6e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import java.io._ -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -129,7 +129,8 @@ private[deploy] class ExecutorRunner( val builder = CommandUtils.buildProcessBuilder(appDesc.command, new SecurityManager(conf), memory, sparkHome.getAbsolutePath, substituteVariables) val command = builder.command() - logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) + val formattedCommand = command.asScala.mkString("\"", "\" \"", "\"") + logInfo(s"Launch command: $formattedCommand") builder.directory(executorDir) builder.environment.put("SPARK_EXECUTOR_DIRS", appLocalDirs.mkString(File.pathSeparator)) @@ -145,7 +146,7 @@ private[deploy] class ExecutorRunner( process = builder.start() val header = "Spark Executor Command: %s\n%s\n\n".format( - command.mkString("\"", "\" \"", "\""), "=" * 40) + formattedCommand, "=" * 40) // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 79b1536d94016..770927c80f7a4 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -24,7 +24,6 @@ import java.util.{UUID, Date} import java.util.concurrent._ import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture} -import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap} import scala.concurrent.ExecutionContext import scala.util.Random diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 42a85e42ea2b6..c3491bb8b1cf3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,7 +23,7 @@ import java.net.URL import java.nio.ByteBuffer import java.util.concurrent.{ConcurrentHashMap, TimeUnit} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal @@ -147,7 +147,7 @@ private[spark] class Executor( /** Returns the total amount of time this JVM process has spent in garbage collection. */ private def computeTotalGcTime(): Long = { - ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + ManagementFactory.getGarbageCollectorMXBeans.asScala.map(_.getCollectionTime).sum } class TaskRunner( @@ -425,7 +425,7 @@ private[spark] class Executor( val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() val curGCTime = computeTotalGcTime() - for (taskRunner <- runningTasks.values()) { + for (taskRunner <- runningTasks.values().asScala) { if (taskRunner.task != null) { taskRunner.task.metrics.foreach { metrics => metrics.updateShuffleReadMetrics() diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 293c512f8b70c..d16f4a1fc4e3b 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.util.concurrent.ThreadPoolExecutor -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.hadoop.fs.FileSystem @@ -30,7 +30,7 @@ private[spark] class ExecutorSource(threadPool: ThreadPoolExecutor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = - FileSystem.getAllStatistics().find(s => s.getScheme.equals(scheme)) + FileSystem.getAllStatistics.asScala.find(s => s.getScheme.equals(scheme)) private def registerFileSystemStat[T]( scheme: String, name: String, f: FileSystem.Statistics => T, defaultValue: T) = { diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index cfd672e1d8a97..0474fd2ccc12e 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver} @@ -28,7 +28,7 @@ import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler.cluster.mesos.{MesosTaskLaunchData} +import org.apache.spark.scheduler.cluster.mesos.MesosTaskLaunchData import org.apache.spark.util.{SignalLogger, Utils} private[spark] class MesosExecutorBackend @@ -55,7 +55,7 @@ private[spark] class MesosExecutorBackend slaveInfo: SlaveInfo) { // Get num cores for this task from ExecutorInfo, created in MesosSchedulerBackend. - val cpusPerTask = executorInfo.getResourcesList + val cpusPerTask = executorInfo.getResourcesList.asScala .find(_.getName == "cpus") .map(_.getScalar.getValue.toInt) .getOrElse(0) diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 6cda7772f77bc..a5ad47293f1c2 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -19,7 +19,7 @@ package org.apache.spark.input import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.io.ByteStreams import org.apache.hadoop.conf.Configuration @@ -44,12 +44,9 @@ private[spark] abstract class StreamFileInputFormat[T] * which is set through setMaxSplitSize */ def setMinPartitions(context: JobContext, minPartitions: Int) { - val files = listStatus(context) - val totalLen = files.map { file => - if (file.isDir) 0L else file.getLen - }.sum - - val maxSplitSize = Math.ceil(totalLen * 1.0 / files.length).toLong + val files = listStatus(context).asScala + val totalLen = files.map(file => if (file.isDir) 0L else file.getLen).sum + val maxSplitSize = Math.ceil(totalLen * 1.0 / files.size).toLong super.setMaxSplitSize(maxSplitSize) } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index aaef7c74eea33..1ba34a11414a2 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -17,7 +17,7 @@ package org.apache.spark.input -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit @@ -52,10 +52,8 @@ private[spark] class WholeTextFileInputFormat * which is set through setMaxSplitSize */ def setMinPartitions(context: JobContext, minPartitions: Int) { - val files = listStatus(context) - val totalLen = files.map { file => - if (file.isDir) 0L else file.getLen - }.sum + val files = listStatus(context).asScala + val totalLen = files.map(file => if (file.isDir) 0L else file.getLen).sum val maxSplitSize = Math.ceil(totalLen * 1.0 / (if (minPartitions == 0) 1 else minPartitions)).toLong super.setMaxSplitSize(maxSplitSize) diff --git a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala index 9be98723aed14..0c096656f9236 100644 --- a/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala +++ b/core/src/main/scala/org/apache/spark/launcher/WorkerCommandBuilder.scala @@ -20,7 +20,7 @@ package org.apache.spark.launcher import java.io.File import java.util.{HashMap => JHashMap, List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.deploy.Command @@ -32,7 +32,7 @@ import org.apache.spark.deploy.Command private[spark] class WorkerCommandBuilder(sparkHome: String, memoryMb: Int, command: Command) extends AbstractCommandBuilder { - childEnv.putAll(command.environment) + childEnv.putAll(command.environment.asJava) childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sparkHome) override def buildCommand(env: JMap[String, String]): JList[String] = { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index d7495551ad233..dd2d325d87034 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -20,6 +20,7 @@ package org.apache.spark.metrics import java.io.{FileInputStream, InputStream} import java.util.Properties +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.matching.Regex @@ -58,25 +59,20 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { propertyCategories = subProperties(properties, INSTANCE_REGEX) if (propertyCategories.contains(DEFAULT_PREFIX)) { - import scala.collection.JavaConversions._ - - val defaultProperty = propertyCategories(DEFAULT_PREFIX) - for { (inst, prop) <- propertyCategories - if (inst != DEFAULT_PREFIX) - (k, v) <- defaultProperty - if (prop.getProperty(k) == null) } { - prop.setProperty(k, v) + val defaultProperty = propertyCategories(DEFAULT_PREFIX).asScala + for((inst, prop) <- propertyCategories if (inst != DEFAULT_PREFIX); + (k, v) <- defaultProperty if (prop.get(k) == null)) { + prop.put(k, v) } } } def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { val subProperties = new mutable.HashMap[String, Properties] - import scala.collection.JavaConversions._ - prop.foreach { kv => - if (regex.findPrefixOf(kv._1).isDefined) { - val regex(prefix, suffix) = kv._1 - subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) + prop.asScala.foreach { kv => + if (regex.findPrefixOf(kv._1.toString).isDefined) { + val regex(prefix, suffix) = kv._1.toString + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2.toString) } } subProperties diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index b089da8596e2b..7c170a742fb64 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -19,7 +19,7 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.Logging import org.apache.spark.network.BlockDataManager @@ -55,7 +55,7 @@ class NettyBlockRpcServer( case openBlocks: OpenBlocks => val blocks: Seq[ManagedBuffer] = openBlocks.blockIds.map(BlockId.apply).map(blockManager.getBlockData) - val streamId = streamManager.registerStream(blocks.iterator) + val streamId = streamManager.registerStream(blocks.iterator.asJava) logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") responseContext.onSuccess(new StreamHandle(streamId, blocks.size).toByteArray) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index d650d5fe73087..ff8aae9ebe9f0 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,7 +17,7 @@ package org.apache.spark.network.netty -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.{Future, Promise} import org.apache.spark.{SecurityManager, SparkConf} @@ -58,7 +58,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage securityManager.isSaslEncryptionEnabled())) } transportContext = new TransportContext(transportConf, rpcHandler) - clientFactory = transportContext.createClientFactory(clientBootstrap.toList) + clientFactory = transportContext.createClientFactory(clientBootstrap.toSeq.asJava) server = createServer(serverBootstrap.toList) appId = conf.getAppId logInfo("Server created on " + server.getPort) @@ -67,7 +67,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage /** Creates and binds the TransportServer, possibly trying multiple ports. */ private def createServer(bootstraps: List[TransportServerBootstrap]): TransportServer = { def startService(port: Int): (TransportServer, Int) = { - val server = transportContext.createServer(port, bootstraps) + val server = transportContext.createServer(port, bootstraps.asJava) (server, server.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 1499da07bb83b..8d9ebadaf79d4 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -23,7 +23,7 @@ import java.nio.channels._ import java.util.concurrent.ConcurrentLinkedQueue import java.util.LinkedList -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal @@ -145,7 +145,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, } def callOnExceptionCallbacks(e: Throwable) { - onExceptionCallbacks foreach { + onExceptionCallbacks.asScala.foreach { callback => try { callback(this, e) diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 91b07ce3af1b6..5afce75680f94 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -19,7 +19,7 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.HashMap import scala.reflect.ClassTag @@ -48,9 +48,9 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf if (outputsMerged == totalOutputs) { val result = new JHashMap[T, BoundedDouble](sums.size) sums.foreach { case (key, sum) => - result(key) = new BoundedDouble(sum, 1.0, sum, sum) + result.put(key, new BoundedDouble(sum, 1.0, sum, sum)) } - result + result.asScala } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { @@ -64,9 +64,9 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf val stdev = math.sqrt(variance) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(key) = new BoundedDouble(mean, confidence, low, high) + result.put(key, new BoundedDouble(mean, confidence, low, high)) } - result + result.asScala } } } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index af26c3d59ac02..a164040684803 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -19,7 +19,7 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.HashMap @@ -55,9 +55,9 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub while (iter.hasNext) { val entry = iter.next() val mean = entry.getValue.mean - result(entry.getKey) = new BoundedDouble(mean, 1.0, mean, mean) + result.put(entry.getKey, new BoundedDouble(mean, 1.0, mean, mean)) } - result + result.asScala } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { @@ -72,9 +72,9 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub val confFactor = studentTCacher.get(counter.count) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(entry.getKey) = new BoundedDouble(mean, confidence, low, high) + result.put(entry.getKey, new BoundedDouble(mean, confidence, low, high)) } - result + result.asScala } } } diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala index 442fb86227d86..54a1beab3514b 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala @@ -19,7 +19,7 @@ package org.apache.spark.partial import java.util.{HashMap => JHashMap} -import scala.collection.JavaConversions.mapAsScalaMap +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.HashMap @@ -55,9 +55,9 @@ private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Doubl while (iter.hasNext) { val entry = iter.next() val sum = entry.getValue.sum - result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum) + result.put(entry.getKey, new BoundedDouble(sum, 1.0, sum, sum)) } - result + result.asScala } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { @@ -80,9 +80,9 @@ private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Doubl val confFactor = studentTCacher.get(counter.count) val low = sumEstimate - confFactor * sumStdev val high = sumEstimate + confFactor * sumStdev - result(entry.getKey) = new BoundedDouble(sumEstimate, confidence, low, high) + result.put(entry.getKey, new BoundedDouble(sumEstimate, confidence, low, high)) } - result + result.asScala } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 326fafb230a40..4e5f2e8a5d467 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -22,7 +22,7 @@ import java.text.SimpleDateFormat import java.util.{Date, HashMap => JHashMap} import scala.collection.{Map, mutable} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import scala.util.DynamicVariable @@ -312,14 +312,14 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } : Iterator[JHashMap[K, V]] val mergeMaps = (m1: JHashMap[K, V], m2: JHashMap[K, V]) => { - m2.foreach { pair => + m2.asScala.foreach { pair => val old = m1.get(pair._1) m1.put(pair._1, if (old == null) pair._2 else cleanedF(old, pair._2)) } m1 } : JHashMap[K, V] - self.mapPartitions(reducePartition).reduce(mergeMaps) + self.mapPartitions(reducePartition).reduce(mergeMaps).asScala } /** Alias for reduceByKeyLocally */ diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 3bb9998e1db44..afbe566b76566 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -23,7 +23,7 @@ import java.io.IOException import java.io.PrintWriter import java.util.StringTokenizer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source @@ -72,7 +72,7 @@ private[spark] class PipedRDD[T: ClassTag]( } override def compute(split: Partition, context: TaskContext): Iterator[String] = { - val pb = new ProcessBuilder(command) + val pb = new ProcessBuilder(command.asJava) // Add the environmental variables to the process. val currentEnvVars = pb.environment() envVars.foreach { case (variable, value) => currentEnvVars.put(variable, value) } @@ -81,7 +81,7 @@ private[spark] class PipedRDD[T: ClassTag]( // so the user code can access the input filename if (split.isInstanceOf[HadoopPartition]) { val hadoopSplit = split.asInstanceOf[HadoopPartition] - currentEnvVars.putAll(hadoopSplit.getPipeEnvVars()) + currentEnvVars.putAll(hadoopSplit.getPipeEnvVars().asJava) } // When spark.worker.separated.working.directory option is turned on, each diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index f7cb1791d4ac6..9a4fa301b06e3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.util.{HashMap => JHashMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag @@ -125,7 +125,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( integrate(0, t => getSeq(t._1) += t._2) // the second dep is rdd2; remove all of its keys integrate(1, t => map.remove(t._1)) - map.iterator.map { t => t._2.iterator.map { (t._1, _) } }.flatten + map.asScala.iterator.map(t => t._2.iterator.map((t._1, _))).flatten } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index bac37bfdaa23f..0e438ab4366d9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.immutable.Set import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -107,7 +107,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl val retval = new ArrayBuffer[SplitInfo]() val list = instance.getSplits(job) - for (split <- list) { + for (split <- list.asScala) { retval ++= SplitInfo.toSplitInfo(inputFormatClazz, path, split) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 174b73221afc0..5821afea98982 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging @@ -74,7 +74,7 @@ private[spark] class Pool( if (schedulableNameToSchedulable.containsKey(schedulableName)) { return schedulableNameToSchedulable.get(schedulableName) } - for (schedulable <- schedulableQueue) { + for (schedulable <- schedulableQueue.asScala) { val sched = schedulable.getSchedulableByName(schedulableName) if (sched != null) { return sched @@ -84,12 +84,12 @@ private[spark] class Pool( } override def executorLost(executorId: String, host: String) { - schedulableQueue.foreach(_.executorLost(executorId, host)) + schedulableQueue.asScala.foreach(_.executorLost(executorId, host)) } override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (schedulable <- schedulableQueue) { + for (schedulable <- schedulableQueue.asScala) { shouldRevive |= schedulable.checkSpeculatableTasks() } shouldRevive @@ -98,7 +98,7 @@ private[spark] class Pool( override def getSortedTaskSetQueue: ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] val sortedSchedulableQueue = - schedulableQueue.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) + schedulableQueue.asScala.toSeq.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d6e1e9e5bebc2..452c32d5411cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -21,7 +21,7 @@ import java.io.File import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, HashSet} import com.google.common.collect.HashBiMap @@ -233,7 +233,7 @@ private[spark] class CoarseMesosSchedulerBackend( override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { stateLock.synchronized { val filters = Filters.newBuilder().setRefuseSeconds(5).build() - for (offer <- offers) { + for (offer <- offers.asScala) { val offerAttributes = toAttributeMap(offer.getAttributesList) val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) val slaveId = offer.getSlaveId.getValue @@ -251,21 +251,21 @@ private[spark] class CoarseMesosSchedulerBackend( val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) totalCoresAcquired += cpusToUse val taskId = newMesosTaskId() - taskIdToSlaveId(taskId) = slaveId + taskIdToSlaveId.put(taskId, slaveId) slaveIdsWithExecutors += slaveId coresByTaskId(taskId) = cpusToUse // Gather cpu resources from the available resources and use them in the task. val (remainingResources, cpuResourcesToUse) = partitionResources(offer.getResourcesList, "cpus", cpusToUse) val (_, memResourcesToUse) = - partitionResources(remainingResources, "mem", calculateTotalMemory(sc)) + partitionResources(remainingResources.asJava, "mem", calculateTotalMemory(sc)) val taskBuilder = MesosTaskInfo.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) .setSlaveId(offer.getSlaveId) .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave, taskId)) .setName("Task " + taskId) - .addAllResources(cpuResourcesToUse) - .addAllResources(memResourcesToUse) + .addAllResources(cpuResourcesToUse.asJava) + .addAllResources(memResourcesToUse.asJava) sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => MesosSchedulerBackendUtil @@ -314,9 +314,9 @@ private[spark] class CoarseMesosSchedulerBackend( } if (TaskState.isFinished(TaskState.fromMesos(state))) { - val slaveId = taskIdToSlaveId(taskId) + val slaveId = taskIdToSlaveId.get(taskId) slaveIdsWithExecutors -= slaveId - taskIdToSlaveId -= taskId + taskIdToSlaveId.remove(taskId) // Remove the cores we have remembered for this task, if it's in the hashmap for (cores <- coresByTaskId.get(taskId)) { totalCoresAcquired -= cores @@ -361,7 +361,7 @@ private[spark] class CoarseMesosSchedulerBackend( stateLock.synchronized { if (slaveIdsWithExecutors.contains(slaveId)) { val slaveIdToTaskId = taskIdToSlaveId.inverse() - if (slaveIdToTaskId.contains(slaveId)) { + if (slaveIdToTaskId.containsKey(slaveId)) { val taskId: Int = slaveIdToTaskId.get(slaveId) taskIdToSlaveId.remove(taskId) removeExecutor(sparkExecutorId(slaveId, taskId.toString), reason) @@ -411,7 +411,7 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveIdToTaskId = taskIdToSlaveId.inverse() for (executorId <- executorIds) { val slaveId = executorId.split("/")(0) - if (slaveIdToTaskId.contains(slaveId)) { + if (slaveIdToTaskId.containsKey(slaveId)) { mesosDriver.killTask( TaskID.newBuilder().setValue(slaveIdToTaskId.get(slaveId).toString).build()) pendingRemovedSlaveIds += slaveId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 3efc536f1456c..e0c547dce6d07 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster.mesos -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode @@ -129,6 +129,6 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( } override def fetchAll[T](): Iterable[T] = { - zk.getChildren.forPath(WORKING_DIR).map(fetch[T]).flatten + zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T]) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1206f184fbc82..07da9242b9922 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -21,7 +21,7 @@ import java.io.File import java.util.concurrent.locks.ReentrantLock import java.util.{Collections, Date, List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -350,7 +350,7 @@ private[spark] class MesosClusterScheduler( } // TODO: Page the status updates to avoid trying to reconcile // a large amount of tasks at once. - driver.reconcileTasks(statuses) + driver.reconcileTasks(statuses.toSeq.asJava) } } } @@ -493,10 +493,10 @@ private[spark] class MesosClusterScheduler( } override def resourceOffers(driver: SchedulerDriver, offers: JList[Offer]): Unit = { - val currentOffers = offers.map { o => + val currentOffers = offers.asScala.map(o => new ResourceOffer( o, getResource(o.getResourcesList, "cpus"), getResource(o.getResourcesList, "mem")) - }.toList + ).toList logTrace(s"Received offers from Mesos: \n${currentOffers.mkString("\n")}") val tasks = new mutable.HashMap[OfferID, ArrayBuffer[TaskInfo]]() val currentTime = new Date() @@ -521,10 +521,10 @@ private[spark] class MesosClusterScheduler( currentOffers, tasks) } - tasks.foreach { case (offerId, tasks) => - driver.launchTasks(Collections.singleton(offerId), tasks) + tasks.foreach { case (offerId, taskInfos) => + driver.launchTasks(Collections.singleton(offerId), taskInfos.asJava) } - offers + offers.asScala .filter(o => !tasks.keySet.contains(o.getId)) .foreach(o => driver.declineOffer(o.getId)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 5c20606d58715..2e424054be785 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File import java.util.{ArrayList => JArrayList, Collections, List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, HashSet} import org.apache.mesos.{Scheduler => MScheduler, _} @@ -129,14 +129,12 @@ private[spark] class MesosSchedulerBackend( val (resourcesAfterCpu, usedCpuResources) = partitionResources(availableResources, "cpus", scheduler.CPUS_PER_TASK) val (resourcesAfterMem, usedMemResources) = - partitionResources(resourcesAfterCpu, "mem", calculateTotalMemory(sc)) + partitionResources(resourcesAfterCpu.asJava, "mem", calculateTotalMemory(sc)) - builder.addAllResources(usedCpuResources) - builder.addAllResources(usedMemResources) + builder.addAllResources(usedCpuResources.asJava) + builder.addAllResources(usedMemResources.asJava) - sc.conf.getOption("spark.mesos.uris").map { uris => - setupUris(uris, command) - } + sc.conf.getOption("spark.mesos.uris").foreach(setupUris(_, command)) val executorInfo = builder .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -148,7 +146,7 @@ private[spark] class MesosSchedulerBackend( .setupContainerBuilderDockerInfo(image, sc.conf, executorInfo.getContainerBuilder()) } - (executorInfo.build(), resourcesAfterMem) + (executorInfo.build(), resourcesAfterMem.asJava) } /** @@ -193,7 +191,7 @@ private[spark] class MesosSchedulerBackend( private def getTasksSummary(tasks: JArrayList[MesosTaskInfo]): String = { val builder = new StringBuilder - tasks.foreach { t => + tasks.asScala.foreach { t => builder.append("Task id: ").append(t.getTaskId.getValue).append("\n") .append("Slave id: ").append(t.getSlaveId.getValue).append("\n") .append("Task resources: ").append(t.getResourcesList).append("\n") @@ -211,7 +209,7 @@ private[spark] class MesosSchedulerBackend( override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { // Fail-fast on offers we know will be rejected - val (usableOffers, unUsableOffers) = offers.partition { o => + val (usableOffers, unUsableOffers) = offers.asScala.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue @@ -323,10 +321,10 @@ private[spark] class MesosSchedulerBackend( .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) .setExecutor(executorInfo) .setName(task.name) - .addAllResources(cpuResources) + .addAllResources(cpuResources.asJava) .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString) .build() - (taskInfo, finalResources) + (taskInfo, finalResources.asJava) } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 5b854aa5c2754..860c8e097b3b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.util.{List => JList} import java.util.concurrent.CountDownLatch -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -137,7 +137,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { protected def getResource(res: JList[Resource], name: String): Double = { // A resource can have multiple values in the offer since it can either be from // a specific role or wildcard. - res.filter(_.getName == name).map(_.getScalar.getValue).sum + res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum } protected def markRegistered(): Unit = { @@ -169,7 +169,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { amountToUse: Double): (List[Resource], List[Resource]) = { var remain = amountToUse var requestedResources = new ArrayBuffer[Resource] - val remainingResources = resources.map { + val remainingResources = resources.asScala.map { case r => { if (remain > 0 && r.getType == Value.Type.SCALAR && @@ -214,7 +214,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { * @return */ protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { - offerAttributes.map(attr => { + offerAttributes.asScala.map(attr => { val attrValue = attr.getType match { case Value.Type.SCALAR => attr.getScalar case Value.Type.RANGES => attr.getRanges @@ -253,7 +253,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { requiredValues.map(_.toLong).exists(offerRange.contains(_)) case Some(offeredValue: Value.Set) => // check if the specified required values is a subset of offered set - requiredValues.subsetOf(offeredValue.getItemList.toSet) + requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet) case Some(textValue: Value.Text) => // check if the specified value is equal, if multiple values are specified // we succeed if any of them match. @@ -299,14 +299,13 @@ private[mesos] trait MesosSchedulerUtils extends Logging { Map() } else { try { - Map() ++ mapAsScalaMap(splitter.split(constraintsVal)).map { - case (k, v) => - if (v == null || v.isEmpty) { - (k, Set[String]()) - } else { - (k, v.split(',').toSet) - } - } + splitter.split(constraintsVal).asScala.toMap.mapValues(v => + if (v == null || v.isEmpty) { + Set[String]() + } else { + v.split(',').toSet + } + ) } catch { case NonFatal(e) => throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 0ff7562e912ca..048a938507277 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -21,6 +21,7 @@ import java.io.{EOFException, IOException, InputStream, OutputStream} import java.nio.ByteBuffer import javax.annotation.Nullable +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException} @@ -373,16 +374,15 @@ private class JavaIterableWrapperSerializer override def read(kryo: Kryo, in: KryoInput, clz: Class[java.lang.Iterable[_]]) : java.lang.Iterable[_] = { kryo.readClassAndObject(in) match { - case scalaIterable: Iterable[_] => - scala.collection.JavaConversions.asJavaIterable(scalaIterable) - case javaIterable: java.lang.Iterable[_] => - javaIterable + case scalaIterable: Iterable[_] => scalaIterable.asJava + case javaIterable: java.lang.Iterable[_] => javaIterable } } } private object JavaIterableWrapperSerializer extends Logging { - // The class returned by asJavaIterable (scala.collection.convert.Wrappers$IterableWrapper). + // The class returned by JavaConverters.asJava + // (scala.collection.convert.Wrappers$IterableWrapper). val wrapperClass = scala.collection.convert.WrapAsJava.asJavaIterable(Seq(1)).getClass diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index f6a96d81e7aa9..c057de9b3f4df 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -21,7 +21,7 @@ import java.io.File import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicInteger -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics @@ -210,11 +210,13 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) shuffleStates.get(shuffleId) match { case Some(state) => if (consolidateShuffleFiles) { - for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { + for (fileGroup <- state.allFileGroups.asScala; + file <- fileGroup.files) { file.delete() } } else { - for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { + for (mapId <- state.completedMapTasks.asScala; + reduceId <- 0 until state.numBuckets) { val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) blockManager.diskBlockManager.getFile(blockId).delete() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 6fec5240707a6..7db6035553ae6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -21,7 +21,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.immutable.HashSet import scala.collection.mutable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcCallContext, ThreadSafeRpcEndpoint} @@ -133,7 +133,7 @@ class BlockManagerMasterEndpoint( // Find all blocks for the given RDD, remove the block from both blockLocations and // the blockManagerInfo that is tracking the blocks. - val blocks = blockLocations.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val blocks = blockLocations.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) blocks.foreach { blockId => val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId) bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) @@ -242,7 +242,7 @@ class BlockManagerMasterEndpoint( private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case (blockManagerId, info) => - new StorageStatus(blockManagerId, info.maxMem, info.blocks) + new StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala) }.toArray } @@ -292,7 +292,7 @@ class BlockManagerMasterEndpoint( if (askSlaves) { info.slaveEndpoint.ask[Seq[BlockId]](getMatchingBlockIds) } else { - Future { info.blocks.keys.filter(filter).toSeq } + Future { info.blocks.asScala.keys.filter(filter).toSeq } } future } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 78e7ddc27d1c7..1738258a0c794 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import scala.collection.JavaConversions.mapAsJavaMap +import scala.collection.JavaConverters._ import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -92,7 +92,7 @@ private[spark] object AkkaUtils extends Logging { val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig .getOrElse(ConfigFactory.empty()) - val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]) + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap.asJava) .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( s""" |akka.daemonic = on diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index a725767d08cc2..13cb516b583e9 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -19,12 +19,11 @@ package org.apache.spark.util import java.util.concurrent.CopyOnWriteArrayList -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.Logging -import org.apache.spark.scheduler.SparkListener /** * An event bus which posts events to its listeners. @@ -46,7 +45,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { * `postToAll` in the same thread for all events. */ final def postToAll(event: E): Unit = { - // JavaConversions will create a JIterableWrapper if we use some Scala collection functions. + // JavaConverters can create a JIterableWrapper if we use asScala. // However, this method will be called frequently. To avoid the wrapper cost, here ewe use // Java Iterator directly. val iter = listeners.iterator @@ -69,7 +68,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { val c = implicitly[ClassTag[T]].runtimeClass - listeners.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq + listeners.asScala.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq } } diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index 169489df6c1ea..a1c33212cdb2b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -21,8 +21,6 @@ import java.net.{URLClassLoader, URL} import java.util.Enumeration import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions._ - /** * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. */ diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 8de75ba9a9c92..d7e5143c30953 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -21,7 +21,8 @@ import java.util.Set import java.util.Map.Entry import java.util.concurrent.ConcurrentHashMap -import scala.collection.{JavaConversions, mutable} +import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.Logging @@ -50,8 +51,7 @@ private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = fa } def iterator: Iterator[(A, B)] = { - val jIterator = getEntrySet.iterator - JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue.value)) + getEntrySet.iterator.asScala.map(kv => (kv.getKey, kv.getValue.value)) } def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet @@ -90,9 +90,7 @@ private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = fa } override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = { - JavaConversions.mapAsScalaConcurrentMap(internalMap) - .map { case (k, TimeStampedValue(v, t)) => (k, v) } - .filter(p) + internalMap.asScala.map { case (k, TimeStampedValue(v, t)) => (k, v) }.filter(p) } override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]() diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 7cd8f28b12dd6..65efeb1f4c19c 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions +import scala.collection.JavaConverters._ import scala.collection.mutable.Set private[spark] class TimeStampedHashSet[A] extends Set[A] { @@ -31,7 +31,7 @@ private[spark] class TimeStampedHashSet[A] extends Set[A] { def iterator: Iterator[A] = { val jIterator = internalMap.entrySet().iterator() - JavaConversions.asScalaIterator(jIterator).map(_.getKey) + jIterator.asScala.map(_.getKey) } override def + (elem: A): Set[A] = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8313312226713..2bab4af2e73ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -25,7 +25,7 @@ import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import scala.io.Source @@ -748,12 +748,12 @@ private[spark] object Utils extends Logging { // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order // on unix-like system. On windows, it returns in index order. // It's more proper to pick ip address following system output order. - val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.toList + val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.asScala.toSeq val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse for (ni <- reOrderedNetworkIFs) { - val addresses = ni.getInetAddresses.toList - .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress) + val addresses = ni.getInetAddresses.asScala + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress).toSeq if (addresses.nonEmpty) { val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) // because of Inet6Address.toHostName may add interface at the end if it knows about it @@ -1498,10 +1498,8 @@ private[spark] object Utils extends Logging { * properties which have been set explicitly, as well as those for which only a default value * has been defined. */ def getSystemProperties: Map[String, String] = { - val sysProps = for (key <- System.getProperties.stringPropertyNames()) yield - (key, System.getProperty(key)) - - sysProps.toMap + System.getProperties.stringPropertyNames().asScala + .map(key => (key, System.getProperty(key))).toMap } /** @@ -1812,7 +1810,8 @@ private[spark] object Utils extends Logging { try { val properties = new Properties() properties.load(inReader) - properties.stringPropertyNames().map(k => (k, properties(k).trim)).toMap + properties.stringPropertyNames().asScala.map( + k => (k, properties.getProperty(k).trim)).toMap } catch { case e: IOException => throw new SparkException(s"Failed when loading Spark properties from $filename", e) @@ -1941,7 +1940,8 @@ private[spark] object Utils extends Logging { return true } isBindCollision(e.getCause) - case e: MultiException => e.getThrowables.exists(isBindCollision) + case e: MultiException => + e.getThrowables.asScala.exists(isBindCollision) case e: Exception => isBindCollision(e.getCause) case _ => false } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala index bdbca00a00622..4939b600dbfbd 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import scala.collection.JavaConversions.{collectionAsScalaIterable, asJavaIterator} +import scala.collection.JavaConverters._ import com.google.common.collect.{Ordering => GuavaOrdering} @@ -34,6 +34,6 @@ private[spark] object Utils { val ordering = new GuavaOrdering[T] { override def compare(l: T, r: T): Int = ord.compare(l, r) } - collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator + ordering.leastOf(input.asJava, num).iterator.asScala } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index ffe4b4baffb2a..ebd3d61ae7324 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -24,10 +24,10 @@ import java.util.*; import java.util.concurrent.*; -import scala.collection.JavaConversions; import scala.Tuple2; import scala.Tuple3; import scala.Tuple4; +import scala.collection.JavaConverters; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -1473,7 +1473,9 @@ public Integer call(Integer v1, Integer v2) throws Exception { Assert.assertEquals(expected, results); Partitioner defaultPartitioner = Partitioner.defaultPartitioner( - combinedRDD.rdd(), JavaConversions.asScalaBuffer(Lists.>newArrayList())); + combinedRDD.rdd(), + JavaConverters.collectionAsScalaIterableConverter( + Collections.>emptyList()).asScala().toSeq()); combinedRDD = originalRDD.keyBy(keyFunction) .combineByKey( createCombinerFunction, diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 90cb7da94e88a..ff9a92cc0a421 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.util.concurrent.{TimeUnit, Executors} +import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.{Try, Random} @@ -148,7 +149,6 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("Thread safeness - SPARK-5425") { - import scala.collection.JavaConversions._ val executor = Executors.newSingleThreadScheduledExecutor() val sf = executor.scheduleAtFixedRate(new Runnable { override def run(): Unit = @@ -163,8 +163,9 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } } finally { executor.shutdownNow() - for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425.")) - System.getProperties.remove(key) + val sysProps = System.getProperties + for (key <- sysProps.stringPropertyNames().asScala if key.startsWith("spark.5425.")) + sysProps.remove(key) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index cbd2aee10c0e2..86eb41dd7e5d7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy import java.net.URL -import scala.collection.JavaConversions._ import scala.collection.mutable import scala.io.Source diff --git a/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala index 47a64081e297e..1ed4bae3ca21e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/RPackageUtilsSuite.scala @@ -21,14 +21,14 @@ import java.io.{PrintStream, OutputStream, File} import java.net.URI import java.util.jar.Attributes.Name import java.util.jar.{JarFile, Manifest} -import java.util.zip.{ZipEntry, ZipFile} +import java.util.zip.ZipFile -import org.scalatest.BeforeAndAfterEach -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.apache.commons.io.FileUtils +import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.api.r.RUtils @@ -142,7 +142,7 @@ class RPackageUtilsSuite extends SparkFunSuite with BeforeAndAfterEach { IvyTestUtils.writeFile(fakePackageDir, "DESCRIPTION", "abc") val finalZip = RPackageUtils.zipRLibraries(tempDir, "sparkr.zip") assert(finalZip.exists()) - val entries = new ZipFile(finalZip).entries().toSeq.map(_.getName) + val entries = new ZipFile(finalZip).entries().asScala.map(_.getName).toSeq assert(entries.contains("/test.R")) assert(entries.contains("/SparkR/abc.R")) assert(entries.contains("/SparkR/DESCRIPTION")) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index bed6f3ea61241..98664dc1101e6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,8 +19,6 @@ package org.apache.spark.deploy.worker import java.io.File -import scala.collection.JavaConversions._ - import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} @@ -36,6 +34,7 @@ class ExecutorRunnerTest extends SparkFunSuite { ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder( appDesc.command, new SecurityManager(conf), 512, sparkHome, er.substituteVariables) - assert(builder.command().last === appId) + val builderCommand = builder.command() + assert(builderCommand.get(builderCommand.size() - 1) === appId) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 730535ece7878..a9652d7e7d0b0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import java.util.concurrent.Semaphore import scala.collection.mutable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.scalatest.Matchers @@ -365,10 +365,9 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," + classOf[BasicJobCounter].getName) sc = new SparkContext(conf) - sc.listenerBus.listeners.collect { case x: BasicJobCounter => x}.size should be (1) - sc.listenerBus.listeners.collect { - case x: ListenerThatAcceptsSparkConf => x - }.size should be (1) + sc.listenerBus.listeners.asScala.count(_.isInstanceOf[BasicJobCounter]) should be (1) + sc.listenerBus.listeners.asScala + .count(_.isInstanceOf[ListenerThatAcceptsSparkConf]) should be (1) } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 5ed30f64d705f..319b3173e7a6e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.scheduler.cluster.mesos import java.nio.ByteBuffer -import java.util +import java.util.Arrays +import java.util.Collection import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -61,7 +62,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val resources = List( + val resources = Arrays.asList( mesosSchedulerBackend.createResource("cpus", 4), mesosSchedulerBackend.createResource("mem", 1024)) // uri is null. @@ -98,7 +99,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") val (execInfo, _) = backend.createExecutorInfo( - List(backend.createResource("cpus", 4)), "mockExecutor") + Arrays.asList(backend.createResource("cpus", 4)), "mockExecutor") assert(execInfo.getContainer.getDocker.getImage.equals("spark/mock")) val portmaps = execInfo.getContainer.getDocker.getPortMappingsList assert(portmaps.get(0).getHostPort.equals(80)) @@ -179,7 +180,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) - val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), @@ -279,7 +280,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(1) - val capture = ArgumentCaptor.forClass(classOf[util.Collection[TaskInfo]]) + val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), @@ -304,7 +305,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi assert(cpusDev.getName.equals("cpus")) assert(cpusDev.getScalar.getValue.equals(1.0)) assert(cpusDev.getRole.equals("dev")) - val executorResources = taskInfo.getExecutor.getResourcesList + val executorResources = taskInfo.getExecutor.getResourcesList.asScala assert(executorResources.exists { r => r.getName.equals("mem") && r.getScalar.getValue.equals(484.0) && r.getRole.equals("prod") }) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 23a1fdb0f5009..8d1c9d17e977e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.serializer import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.reflect.ClassTag @@ -173,7 +174,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("asJavaIterable") { // Serialize a collection wrapped by asJavaIterable val ser = new KryoSerializer(conf).newInstance() - val a = ser.serialize(scala.collection.convert.WrapAsJava.asJavaIterable(Seq(12345))) + val a = ser.serialize(Seq(12345).asJava) val b = ser.deserialize[java.lang.Iterable[Int]](a) assert(b.iterator().next() === 12345) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 69888b2694bae..22e30ecaf0533 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -21,7 +21,6 @@ import java.net.{HttpURLConnection, URL} import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.io.Source -import scala.collection.JavaConversions._ import scala.xml.Node import com.gargoylesoftware.htmlunit.DefaultCssErrorHandler @@ -341,15 +340,15 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B // The completed jobs table should have two rows. The first row will be the most recent job: val firstRow = find(cssSelector("tbody tr")).get.underlying val firstRowColumns = firstRow.findElements(By.tagName("td")) - firstRowColumns(0).getText should be ("1") - firstRowColumns(4).getText should be ("1/1 (2 skipped)") - firstRowColumns(5).getText should be ("8/8 (16 skipped)") + firstRowColumns.get(0).getText should be ("1") + firstRowColumns.get(4).getText should be ("1/1 (2 skipped)") + firstRowColumns.get(5).getText should be ("8/8 (16 skipped)") // The second row is the first run of the job, where nothing was skipped: val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying val secondRowColumns = secondRow.findElements(By.tagName("td")) - secondRowColumns(0).getText should be ("0") - secondRowColumns(4).getText should be ("3/3") - secondRowColumns(5).getText should be ("24/24") + secondRowColumns.get(0).getText should be ("0") + secondRowColumns.get(4).getText should be ("3/3") + secondRowColumns.get(5).getText should be ("24/24") } } } @@ -502,8 +501,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B for { (row, idx) <- rows.zipWithIndex columns = row.findElements(By.tagName("td")) - id = columns(0).getText() - description = columns(1).getText() + id = columns.get(0).getText() + description = columns.get(1).getText() } { id should be (expJobInfo(idx)._1) description should include (expJobInfo(idx)._2) @@ -547,8 +546,8 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B for { (row, idx) <- rows.zipWithIndex columns = row.findElements(By.tagName("td")) - id = columns(0).getText() - description = columns(1).getText() + id = columns.get(0).getText() + description = columns.get(1).getText() } { id should be (expStageInfo(idx)._1) description should include (expStageInfo(idx)._2) diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 36832f51d2ad4..fa07c1e5017cd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -19,10 +19,7 @@ package org.apache.spark.examples import java.nio.ByteBuffer - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ListBuffer -import scala.collection.immutable.Map +import java.util.Collections import org.apache.cassandra.hadoop.ConfigHelper import org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat @@ -32,7 +29,6 @@ import org.apache.cassandra.utils.ByteBufferUtil import org.apache.hadoop.mapreduce.Job import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ /* @@ -121,12 +117,9 @@ object CassandraCQLTest { val casoutputCF = aggregatedRDD.map { case (productId, saleCount) => { - val outColFamKey = Map("prod_id" -> ByteBufferUtil.bytes(productId)) - val outKey: java.util.Map[String, ByteBuffer] = outColFamKey - var outColFamVal = new ListBuffer[ByteBuffer] - outColFamVal += ByteBufferUtil.bytes(saleCount) - val outVal: java.util.List[ByteBuffer] = outColFamVal - (outKey, outVal) + val outKey = Collections.singletonMap("prod_id", ByteBufferUtil.bytes(productId)) + val outVal = Collections.singletonList(ByteBufferUtil.bytes(saleCount)) + (outKey, outVal) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index 96ef3e198e380..2e56d24c60c33 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -19,10 +19,9 @@ package org.apache.spark.examples import java.nio.ByteBuffer +import java.util.Arrays import java.util.SortedMap -import scala.collection.JavaConversions._ - import org.apache.cassandra.db.IColumn import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat import org.apache.cassandra.hadoop.ConfigHelper @@ -32,7 +31,6 @@ import org.apache.cassandra.utils.ByteBufferUtil import org.apache.hadoop.mapreduce.Job import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ /* * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra @@ -118,7 +116,7 @@ object CassandraTest { val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) - val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil + val mutations = Arrays.asList(new Mutation(), new Mutation()) mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) mutations.get(0).column_or_supercolumn.setColumn(colWord) mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index c42df2b8845d2..bec61f3cd4296 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -18,7 +18,7 @@ // scalastyle:off println package org.apache.spark.examples -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.util.Utils @@ -36,10 +36,10 @@ object DriverSubmissionTest { val properties = Utils.getSystemProperties println("Environment variables containing SPARK_TEST:") - env.filter{case (k, v) => k.contains("SPARK_TEST")}.foreach(println) + env.asScala.filter { case (k, _) => k.contains("SPARK_TEST")}.foreach(println) println("System properties containing spark.test:") - properties.filter{case (k, v) => k.toString.contains("spark.test")}.foreach(println) + properties.filter { case (k, _) => k.toString.contains("spark.test") }.foreach(println) for (i <- 1 until numSecondsToSleep) { println(s"Alive for $i out of $numSecondsToSleep seconds") diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala index 3ebb112fc069e..805184e740f06 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.pythonconverters import java.util.{Collection => JCollection, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.avro.generic.{GenericFixed, IndexedRecord} import org.apache.avro.mapred.AvroWrapper @@ -58,7 +58,7 @@ object AvroConversionUtil extends Serializable { val map = new java.util.HashMap[String, Any] obj match { case record: IndexedRecord => - record.getSchema.getFields.zipWithIndex.foreach { case (f, i) => + record.getSchema.getFields.asScala.zipWithIndex.foreach { case (f, i) => map.put(f.name, fromAvro(record.get(i), f.schema)) } case other => throw new SparkException( @@ -68,9 +68,9 @@ object AvroConversionUtil extends Serializable { } def unpackMap(obj: Any, schema: Schema): JMap[String, Any] = { - obj.asInstanceOf[JMap[_, _]].map { case (key, value) => + obj.asInstanceOf[JMap[_, _]].asScala.map { case (key, value) => (key.toString, fromAvro(value, schema.getValueType)) - } + }.asJava } def unpackFixed(obj: Any, schema: Schema): Array[Byte] = { @@ -91,17 +91,17 @@ object AvroConversionUtil extends Serializable { def unpackArray(obj: Any, schema: Schema): JCollection[Any] = obj match { case c: JCollection[_] => - c.map(fromAvro(_, schema.getElementType)) + c.asScala.map(fromAvro(_, schema.getElementType)).toSeq.asJava case arr: Array[_] if arr.getClass.getComponentType.isPrimitive => - arr.toSeq + arr.toSeq.asJava.asInstanceOf[JCollection[Any]] case arr: Array[_] => - arr.map(fromAvro(_, schema.getElementType)).toSeq + arr.map(fromAvro(_, schema.getElementType)).toSeq.asJava case other => throw new SparkException( s"Unknown ARRAY type ${other.getClass.getName}") } def unpackUnion(obj: Any, schema: Schema): Any = { - schema.getTypes.toList match { + schema.getTypes.asScala.toList match { case List(s) => fromAvro(obj, s) case List(n, s) if n.getType == NULL => fromAvro(obj, s) case List(s, n) if n.getType == NULL => fromAvro(obj, s) diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala index 83feb5703b908..00ce47af4813d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala @@ -17,11 +17,13 @@ package org.apache.spark.examples.pythonconverters -import org.apache.spark.api.python.Converter import java.nio.ByteBuffer + +import scala.collection.JavaConverters._ + import org.apache.cassandra.utils.ByteBufferUtil -import collection.JavaConversions._ +import org.apache.spark.api.python.Converter /** * Implementation of [[org.apache.spark.api.python.Converter]] that converts Cassandra @@ -30,7 +32,7 @@ import collection.JavaConversions._ class CassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, Int]] { override def convert(obj: Any): java.util.Map[String, Int] = { val result = obj.asInstanceOf[java.util.Map[String, ByteBuffer]] - mapAsJavaMap(result.mapValues(bb => ByteBufferUtil.toInt(bb))) + result.asScala.mapValues(ByteBufferUtil.toInt).asJava } } @@ -41,7 +43,7 @@ class CassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, Int] class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, String]] { override def convert(obj: Any): java.util.Map[String, String] = { val result = obj.asInstanceOf[java.util.Map[String, ByteBuffer]] - mapAsJavaMap(result.mapValues(bb => ByteBufferUtil.string(bb))) + result.asScala.mapValues(ByteBufferUtil.string).asJava } } @@ -52,7 +54,7 @@ class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, St class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, ByteBuffer]] { override def convert(obj: Any): java.util.Map[String, ByteBuffer] = { val input = obj.asInstanceOf[java.util.Map[String, Int]] - mapAsJavaMap(input.mapValues(i => ByteBufferUtil.bytes(i))) + input.asScala.mapValues(ByteBufferUtil.bytes).asJava } } @@ -63,6 +65,6 @@ class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, By class ToCassandraCQLValueConverter extends Converter[Any, java.util.List[ByteBuffer]] { override def convert(obj: Any): java.util.List[ByteBuffer] = { val input = obj.asInstanceOf[java.util.List[String]] - seqAsJavaList(input.map(s => ByteBufferUtil.bytes(s))) + input.asScala.map(ByteBufferUtil.bytes).asJava } } diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala index 90d48a64106c7..0a25ee7ae56f4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala @@ -17,7 +17,7 @@ package org.apache.spark.examples.pythonconverters -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.parsing.json.JSONObject import org.apache.spark.api.python.Converter @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil */ class HBaseResultToStringConverter extends Converter[Any, String] { override def convert(obj: Any): String = { - import collection.JavaConverters._ val result = obj.asInstanceOf[Result] val output = result.listCells.asScala.map(cell => Map( @@ -77,7 +76,7 @@ class StringToImmutableBytesWritableConverter extends Converter[Any, ImmutableBy */ class StringListToPutConverter extends Converter[Any, Put] { override def convert(obj: Any): Put = { - val output = obj.asInstanceOf[java.util.ArrayList[String]].map(Bytes.toBytes(_)).toArray + val output = obj.asInstanceOf[java.util.ArrayList[String]].asScala.map(Bytes.toBytes).toArray val put = new Put(output(0)) put.add(output(1), output(2), output(3)) } diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index fa43629d49771..d2654700ea729 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -20,7 +20,7 @@ import java.net.InetSocketAddress import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import scala.util.{Failure, Success} @@ -166,7 +166,7 @@ class SparkSinkSuite extends FunSuite { channelContext.put("capacity", channelCapacity.toString) channelContext.put("transactionCapacity", 1000.toString) channelContext.put("keep-alive", 0.toString) - channelContext.putAll(overrides) + channelContext.putAll(overrides.asJava) channel.setName(scala.util.Random.nextString(10)) channel.configure(channelContext) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala index 65c49c131518b..48df27b26867f 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.flume import java.io.{ObjectOutput, ObjectInput} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.util.Utils import org.apache.spark.Logging @@ -60,7 +60,7 @@ private[streaming] object EventTransformer extends Logging { out.write(body) val numHeaders = headers.size() out.writeInt(numHeaders) - for ((k, v) <- headers) { + for ((k, v) <- headers.asScala) { val keyBuff = Utils.serialize(k.toString) out.writeInt(keyBuff.length) out.write(keyBuff) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala index 88cc2aa3bf022..b9d4e762ca05d 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.streaming.flume -import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Throwables @@ -155,7 +154,7 @@ private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends R val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) var j = 0 while (j < events.size()) { - val event = events(j) + val event = events.get(j) val sparkFlumeEvent = new SparkFlumeEvent() sparkFlumeEvent.event.setBody(event.getBody) sparkFlumeEvent.event.setHeaders(event.getHeaders) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 1e32a365a1eee..2bf99cb3cba1f 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -22,7 +22,7 @@ import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.nio.ByteBuffer import java.util.concurrent.Executors -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.flume.source.avro.AvroSourceProtocol @@ -99,7 +99,7 @@ class SparkFlumeEvent() extends Externalizable { val numHeaders = event.getHeaders.size() out.writeInt(numHeaders) - for ((k, v) <- event.getHeaders) { + for ((k, v) <- event.getHeaders.asScala) { val keyBuff = Utils.serialize(k.toString) out.writeInt(keyBuff.length) out.write(keyBuff) @@ -127,8 +127,7 @@ class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { } override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { - events.foreach (event => - receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) + events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 583e7dca317ad..0bc46209b8369 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress import java.util.concurrent.{LinkedBlockingQueue, Executors} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -94,9 +94,7 @@ private[streaming] class FlumePollingReceiver( override def onStop(): Unit = { logInfo("Shutting down Flume Polling Receiver") receiverExecutor.shutdownNow() - connections.foreach(connection => { - connection.transceiver.close() - }) + connections.asScala.foreach(_.transceiver.close()) channelFactory.releaseExternalResources() } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala index 9d9c3b189415f..70018c86f92be 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.flume import java.net.{InetSocketAddress, ServerSocket} import java.nio.ByteBuffer -import java.util.{List => JList} +import java.util.Collections -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.base.Charsets.UTF_8 import org.apache.avro.ipc.NettyTransceiver @@ -59,13 +59,13 @@ private[flume] class FlumeTestUtils { } /** Send data to the flume receiver */ - def writeInput(input: JList[String], enableCompression: Boolean): Unit = { + def writeInput(input: Seq[String], enableCompression: Boolean): Unit = { val testAddress = new InetSocketAddress("localhost", testPort) val inputEvents = input.map { item => val event = new AvroFlumeEvent event.setBody(ByteBuffer.wrap(item.getBytes(UTF_8))) - event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) + event.setHeaders(Collections.singletonMap("test", "header")) event } @@ -88,7 +88,7 @@ private[flume] class FlumeTestUtils { } // Send data - val status = client.appendBatch(inputEvents.toList) + val status = client.appendBatch(inputEvents.asJava) if (status != avro.Status.OK) { throw new AssertionError("Sent events unsuccessfully") } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index a65a9b921aafa..c719b80aca7ed 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -21,7 +21,7 @@ import java.net.InetSocketAddress import java.io.{DataOutputStream, ByteArrayOutputStream} import java.util.{List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.api.java.function.PairFunction import org.apache.spark.api.python.PythonRDD @@ -268,8 +268,8 @@ private[flume] class FlumeUtilsPythonHelper { maxBatchSize: Int, parallelism: Int ): JavaPairDStream[Array[Byte], Array[Byte]] = { - assert(hosts.length == ports.length) - val addresses = hosts.zip(ports).map { + assert(hosts.size() == ports.size()) + val addresses = hosts.asScala.zip(ports.asScala).map { case (host, port) => new InetSocketAddress(host, port) } val dstream = FlumeUtils.createPollingStream( @@ -286,7 +286,7 @@ private object FlumeUtilsPythonHelper { val output = new DataOutputStream(byteStream) try { output.writeInt(map.size) - map.foreach { kv => + map.asScala.foreach { kv => PythonRDD.writeUTF(kv._1.toString, output) PythonRDD.writeUTF(kv._2.toString, output) } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala index 91d63d49dbec3..a2ab320957db3 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala @@ -18,9 +18,8 @@ package org.apache.spark.streaming.flume import java.util.concurrent._ -import java.util.{List => JList, Map => JMap} +import java.util.{Map => JMap, Collections} -import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import com.google.common.base.Charsets.UTF_8 @@ -77,7 +76,7 @@ private[flume] class PollingFlumeTestUtils { /** * Start 2 sinks and return the ports */ - def startMultipleSinks(): JList[Int] = { + def startMultipleSinks(): Seq[Int] = { channels.clear() sinks.clear() @@ -138,8 +137,7 @@ private[flume] class PollingFlumeTestUtils { /** * A Python-friendly method to assert the output */ - def assertOutput( - outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { + def assertOutput(outputHeaders: Seq[JMap[String, String]], outputBodies: Seq[String]): Unit = { require(outputHeaders.size == outputBodies.size) val eventSize = outputHeaders.size if (eventSize != totalEventsPerChannel * channels.size) { @@ -149,12 +147,12 @@ private[flume] class PollingFlumeTestUtils { var counter = 0 for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { val eventBodyToVerify = s"${channels(k).getName}-$i" - val eventHeaderToVerify: JMap[String, String] = Map[String, String](s"test-$i" -> "header") + val eventHeaderToVerify: JMap[String, String] = Collections.singletonMap(s"test-$i", "header") var found = false var j = 0 while (j < eventSize && !found) { - if (eventBodyToVerify == outputBodies.get(j) && - eventHeaderToVerify == outputHeaders.get(j)) { + if (eventBodyToVerify == outputBodies(j) && + eventHeaderToVerify == outputHeaders(j)) { found = true counter += 1 } @@ -195,7 +193,7 @@ private[flume] class PollingFlumeTestUtils { tx.begin() for (j <- 0 until eventsPerBatch) { channel.put(EventBuilder.withBody(s"${channel.getName}-$t".getBytes(UTF_8), - Map[String, String](s"test-$t" -> "header"))) + Collections.singletonMap(s"test-$t", "header"))) t += 1 } tx.commit() diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index d5f9a0aa38f9f..ff2fb8eed204c 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.concurrent.duration._ import scala.language.postfixOps @@ -116,9 +116,9 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Log // The eventually is required to ensure that all data in the batch has been processed. eventually(timeout(10 seconds), interval(100 milliseconds)) { val flattenOutputBuffer = outputBuffer.flatten - val headers = flattenOutputBuffer.map(_.event.getHeaders.map { - case kv => (kv._1.toString, kv._2.toString) - }).map(mapAsJavaMap) + val headers = flattenOutputBuffer.map(_.event.getHeaders.asScala.map { + case (key, value) => (key.toString, value.toString) + }).map(_.asJava) val bodies = flattenOutputBuffer.map(e => new String(e.event.getBody.array(), UTF_8)) utils.assertOutput(headers, bodies) } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 5bc4cdf65306c..5ffb60bd602f9 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.flume -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import scala.concurrent.duration._ import scala.language.postfixOps diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index 79a9db4291bef..c9fd715d3d554 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -24,6 +24,7 @@ import java.util.concurrent.TimeoutException import java.util.{Map => JMap, Properties} import scala.annotation.tailrec +import scala.collection.JavaConverters._ import scala.language.postfixOps import scala.util.control.NonFatal @@ -159,8 +160,7 @@ private[kafka] class KafkaTestUtils extends Logging { /** Java-friendly function for sending messages to the Kafka broker */ def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { - import scala.collection.JavaConversions._ - sendMessages(topic, Map(messageToFreq.mapValues(_.intValue()).toSeq: _*)) + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) } /** Send the messages to the Kafka broker */ diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 388dbb8184106..3128222077537 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.kafka import java.lang.{Integer => JInt, Long => JLong} import java.util.{List => JList, Map => JMap, Set => JSet} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import kafka.common.TopicAndPartition @@ -96,7 +96,7 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt] ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) } /** @@ -115,7 +115,7 @@ object KafkaUtils { topics: JMap[String, JInt], storageLevel: StorageLevel ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), storageLevel) } @@ -149,7 +149,10 @@ object KafkaUtils { implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) createStream[K, V, U, T]( - jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + jssc.ssc, + kafkaParams.asScala.toMap, + Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) } /** get leaders for the given offset ranges, or throw an exception */ @@ -275,7 +278,7 @@ object KafkaUtils { implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) new JavaPairRDD(createRDD[K, V, KD, VD]( - jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges)) + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) } /** @@ -311,9 +314,9 @@ object KafkaUtils { implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val leaderMap = Map(leaders.toSeq: _*) + val leaderMap = Map(leaders.asScala.toSeq: _*) createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaderMap, messageHandler.call _) + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, messageHandler.call(_)) } /** @@ -476,8 +479,8 @@ object KafkaUtils { val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) createDirectStream[K, V, KD, VD, R]( jssc.ssc, - Map(kafkaParams.toSeq: _*), - Map(fromOffsets.mapValues { _.longValue() }.toSeq: _*), + Map(kafkaParams.asScala.toSeq: _*), + Map(fromOffsets.asScala.mapValues(_.longValue()).toSeq: _*), cleanedHandler ) } @@ -531,8 +534,8 @@ object KafkaUtils { implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) createDirectStream[K, V, KD, VD]( jssc.ssc, - Map(kafkaParams.toSeq: _*), - Set(topics.toSeq: _*) + Map(kafkaParams.asScala.toSeq: _*), + Set(topics.asScala.toSeq: _*) ) } } @@ -602,10 +605,10 @@ private[kafka] class KafkaUtilsPythonHelper { ): JavaPairInputDStream[Array[Byte], Array[Byte]] = { if (!fromOffsets.isEmpty) { - import scala.collection.JavaConversions._ - val topicsFromOffsets = fromOffsets.keySet().map(_.topic) - if (topicsFromOffsets != topics.toSet) { - throw new IllegalStateException(s"The specified topics: ${topics.toSet.mkString(" ")} " + + val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) + if (topicsFromOffsets != topics.asScala.toSet) { + throw new IllegalStateException( + s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") } } @@ -663,6 +666,6 @@ private[kafka] class KafkaUtilsPythonHelper { "with this RDD, please call this method only on a Kafka RDD.") val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] - kafkaRDD.offsetRanges.toSeq + kafkaRDD.offsetRanges.toSeq.asJava } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index 0469d0af8864a..4ea218eaa4de1 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -18,15 +18,17 @@ package org.apache.spark.streaming.zeromq import scala.reflect.ClassTag -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + import akka.actor.{Props, SupervisorStrategy} import akka.util.ByteString import akka.zeromq.Subscribe + import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream} +import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.ActorSupervisorStrategy object ZeroMQUtils { @@ -75,7 +77,8 @@ object ZeroMQUtils { ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator + val fn = + (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) } @@ -99,7 +102,8 @@ object ZeroMQUtils { ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator + val fn = + (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel) } @@ -122,7 +126,8 @@ object ZeroMQUtils { ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator + val fn = + (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala createStream[T](jssc.ssc, publisherUrl, subscribe, fn) } } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index a003ddf325e6e..5d32fa699ae5b 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kinesis -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import com.amazonaws.auth.{AWSCredentials, DefaultAWSCredentialsProviderChain} @@ -213,7 +213,7 @@ class KinesisSequenceRangeIterator( s"getting records using shard iterator") { client.getRecords(getRecordsRequest) } - (getRecordsResult.getRecords.iterator(), getRecordsResult.getNextShardIterator) + (getRecordsResult.getRecords.iterator().asScala, getRecordsResult.getNextShardIterator) } /** diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 22324e821ce94..6e0988c1af8a1 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kinesis import java.util.UUID -import scala.collection.JavaConversions.asScalaIterator +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.control.NonFatal @@ -202,7 +202,7 @@ private[kinesis] class KinesisReceiver( /** Add records of the given shard to the current block being generated */ private[kinesis] def addRecords(shardId: String, records: java.util.List[Record]): Unit = { if (records.size > 0) { - val dataIterator = records.iterator().map { record => + val dataIterator = records.iterator().asScala.map { record => val byteBuffer = record.getData() val byteArray = new Array[Byte](byteBuffer.remaining()) byteBuffer.get(byteArray) diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index c8eec13ec7dc7..634bf94521079 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kinesis import java.nio.ByteBuffer import java.util.concurrent.TimeUnit +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.{Failure, Random, Success, Try} @@ -115,7 +116,7 @@ private[kinesis] class KinesisTestUtils extends Logging { * Expose a Python friendly API. */ def pushData(testData: java.util.List[Int]): Unit = { - pushData(scala.collection.JavaConversions.asScalaBuffer(testData)) + pushData(testData.asScala) } def deleteStream(): Unit = { diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index ceb135e0651aa..3d136aec2e702 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -17,10 +17,10 @@ package org.apache.spark.streaming.kinesis import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.Arrays -import scala.collection.JavaConversions.seqAsJavaList - -import com.amazonaws.services.kinesis.clientlibrary.exceptions.{InvalidStateException, KinesisClientLibDependencyException, ShutdownException, ThrottlingException} +import com.amazonaws.services.kinesis.clientlibrary.exceptions._ import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason import com.amazonaws.services.kinesis.model.Record @@ -47,10 +47,10 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft val someSeqNum = Some(seqNum) val record1 = new Record() - record1.setData(ByteBuffer.wrap("Spark In Action".getBytes())) + record1.setData(ByteBuffer.wrap("Spark In Action".getBytes(StandardCharsets.UTF_8))) val record2 = new Record() - record2.setData(ByteBuffer.wrap("Learning Spark".getBytes())) - val batch = List[Record](record1, record2) + record2.setData(ByteBuffer.wrap("Learning Spark".getBytes(StandardCharsets.UTF_8))) + val batch = Arrays.asList(record1, record2) var receiverMock: KinesisReceiver = _ var checkpointerMock: IRecordProcessorCheckpointer = _ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 87eeb5db05d26..7a1c7796065ee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.util -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Random import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -52,7 +52,7 @@ object LinearDataGenerator { nPoints: Int, seed: Int, eps: Double): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed, eps)) + generateLinearInput(intercept, weights, nPoints, seed, eps).asJava } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index a1ee554152372..2744e020e9e49 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -20,7 +20,7 @@ import java.io.Serializable; import java.util.List; -import static scala.collection.JavaConversions.seqAsJavaList; +import scala.collection.JavaConverters; import org.junit.After; import org.junit.Assert; @@ -55,8 +55,9 @@ public void setUp() { double[] xMean = {5.843, 3.057, 3.758, 1.199}; double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; - List points = seqAsJavaList(generateMultinomialLogisticInput( - weights, xMean, xVariance, true, nPoints, 42)); + List points = JavaConverters.asJavaListConverter( + generateMultinomialLogisticInput(weights, xMean, xVariance, true, nPoints, 42) + ).asJava(); datasetRDD = jsc.parallelize(points, 2); dataset = jsql.createDataFrame(datasetRDD, LabeledPoint.class); } 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 2473510e13514..8d14bb6572155 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 @@ -17,7 +17,7 @@ package org.apache.spark.mllib.classification -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Random import scala.util.control.Breaks._ @@ -38,7 +38,7 @@ object LogisticRegressionSuite { scale: Double, nPoints: Int, seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLogisticInput(offset, scale, nPoints, seed)) + generateLogisticInput(offset, scale, nPoints, seed).asJava } // Generate input of the form Y = logistic(offset + scale*X) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index b1d78cba9e3dc..ee3c85d09a463 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.classification -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Random import org.jblas.DoubleMatrix @@ -35,7 +35,7 @@ object SVMSuite { weights: Array[Double], nPoints: Int, seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateSVMInput(intercept, weights, nPoints, seed)) + generateSVMInput(intercept, weights, nPoints, seed).asJava } // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 13b754a03943a..36ac7d267243d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.optimization -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Random import org.scalatest.Matchers @@ -35,7 +35,7 @@ object GradientDescentSuite { scale: Double, nPoints: Int, seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateGDInput(offset, scale, nPoints, seed)) + generateGDInput(offset, scale, nPoints, seed).asJava } // Generate input of the form Y = logistic(offset + scale * X) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 05b87728d6fdb..045135f7f8d60 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.math.abs import scala.util.Random @@ -38,7 +38,7 @@ object ALSSuite { negativeWeights: Boolean): (java.util.List[Rating], DoubleMatrix, DoubleMatrix) = { val (sampledRatings, trueRatings, truePrefs) = generateRatings(users, products, features, samplingRate, implicitPrefs) - (seqAsJavaList(sampledRatings), trueRatings, truePrefs) + (sampledRatings.asJava, trueRatings, truePrefs) } def generateRatings( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 04e0d49b178cf..ea52bfd67944a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -18,13 +18,13 @@ import java.io._ import scala.util.Properties -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import sbt._ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import com.typesafe.sbt.pom.{loadEffectivePom, PomBuild, SbtPomKeys} +import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings import spray.revolver.RevolverPlugin._ @@ -120,7 +120,7 @@ object SparkBuild extends PomBuild { case _ => } - override val userPropertiesMap = System.getProperties.toMap + override val userPropertiesMap = System.getProperties.asScala.toMap lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") @@ -559,7 +559,7 @@ object TestSettings { javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test += "-Dderby.system.durability=test", - javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") + javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) .map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions in Test += "-ea", javaOptions in Test ++= "-Xmx3g -Xss4096k -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 8af8637cf948d..0948f9b27cd38 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -61,6 +61,18 @@ def _to_seq(sc, cols, converter=None): return sc._jvm.PythonUtils.toSeq(cols) +def _to_list(sc, cols, converter=None): + """ + Convert a list of Column (or names) into a JVM (Scala) List of Column. + + An optional `converter` could be used to convert items in `cols` + into JVM Column objects. + """ + if converter: + cols = [converter(c) for c in cols] + return sc._jvm.PythonUtils.toList(cols) + + def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 025811f519293..e269ef4304f3f 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -32,7 +32,7 @@ from pyspark.traceback_utils import SCCallSiteSync from pyspark.sql import since from pyspark.sql.types import _parse_datatype_json_string -from pyspark.sql.column import Column, _to_seq, _to_java_column +from pyspark.sql.column import Column, _to_seq, _to_list, _to_java_column from pyspark.sql.readwriter import DataFrameWriter from pyspark.sql.types import * @@ -494,7 +494,7 @@ def randomSplit(self, weights, seed=None): if w < 0.0: raise ValueError("Weights must be positive. Found weight value: %s" % w) seed = seed if seed is not None else random.randint(0, sys.maxsize) - rdd_array = self._jdf.randomSplit(_to_seq(self.sql_ctx._sc, weights), long(seed)) + rdd_array = self._jdf.randomSplit(_to_list(self.sql_ctx._sc, weights), long(seed)) return [DataFrame(rdd, self.sql_ctx) for rdd in rdd_array] @property diff --git a/scalastyle-config.xml b/scalastyle-config.xml index b5e2e882d2254..68fdb4141cf27 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -161,6 +161,13 @@ This file is divided into 3 sections: ]]> + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index ec895af9c3037..cfd9cb0e62598 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import scala.collection.JavaConverters._ + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.types.StructType @@ -280,9 +282,8 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ - def getList[T](i: Int): java.util.List[T] = { - scala.collection.JavaConversions.seqAsJavaList(getSeq[T](i)) - } + def getList[T](i: Int): java.util.List[T] = + getSeq[T](i).asJava /** * Returns the value at position i of map type as a Scala Map. @@ -296,9 +297,8 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ - def getJavaMap[K, V](i: Int): java.util.Map[K, V] = { - scala.collection.JavaConversions.mapAsJavaMap(getMap[K, V](i)) - } + def getJavaMap[K, V](i: Int): java.util.Map[K, V] = + getMap[K, V](i).asJava /** * Returns the value at position i of struct type as an [[Row]] object. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 503c4f4b20f38..4cc9a5520a085 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -147,7 +147,7 @@ class SimpleCatalog(val conf: CatalystConf) extends Catalog { override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = { val result = ArrayBuffer.empty[(String, Boolean)] - for (name <- tables.keySet()) { + for (name <- tables.keySet().asScala) { result += ((name, true)) } result diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index a4fd4cf3b330b..77a42c0873a6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.{lang => jl} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.expressions._ @@ -209,7 +209,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def fill(valueMap: java.util.Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + def fill(valueMap: java.util.Map[String, Any]): DataFrame = fill0(valueMap.asScala.toSeq) /** * (Scala-specific) Returns a new [[DataFrame]] that replaces null values. @@ -254,7 +254,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * @since 1.3.1 */ def replace[T](col: String, replacement: java.util.Map[T, T]): DataFrame = { - replace[T](col, replacement.toMap : Map[T, T]) + replace[T](col, replacement.asScala.toMap) } /** @@ -277,7 +277,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * @since 1.3.1 */ def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { - replace(cols.toSeq, replacement.toMap) + replace(cols.toSeq, replacement.asScala.toMap) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 6dc7bfe333498..97a8b6518a832 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.util.Properties +import scala.collection.JavaConverters._ + import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Experimental @@ -90,7 +92,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging { * @since 1.4.0 */ def options(options: java.util.Map[String, String]): DataFrameReader = { - this.options(scala.collection.JavaConversions.mapAsScalaMap(options)) + this.options(options.asScala) this } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index ce8744b53175b..b2a66dd417b4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.util.Properties +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -109,7 +111,7 @@ final class DataFrameWriter private[sql](df: DataFrame) { * @since 1.4.0 */ def options(options: java.util.Map[String, String]): DataFrameWriter = { - this.options(scala.collection.JavaConversions.mapAsScalaMap(options)) + this.options(options.asScala) this } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index 99d557b03a033..ee31d83cce42c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import org.apache.spark.annotation.Experimental @@ -188,7 +188,7 @@ class GroupedData protected[sql]( * @since 1.3.0 */ def agg(exprs: java.util.Map[String, String]): DataFrame = { - agg(exprs.toMap) + agg(exprs.asScala.toMap) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index e9de14f025502..e6f7619519e6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import java.util.Properties import scala.collection.immutable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.parquet.hadoop.ParquetOutputCommitter @@ -531,7 +531,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf { /** Set Spark SQL configuration properties. */ def setConf(props: Properties): Unit = settings.synchronized { - props.foreach { case (k, v) => setConfString(k, v) } + props.asScala.foreach { case (k, v) => setConfString(k, v) } } /** Set the given Spark SQL configuration property using a `string` value. */ @@ -601,24 +601,25 @@ private[sql] class SQLConf extends Serializable with CatalystConf { * Return all the configuration properties that have been set (i.e. not the default). * This creates a new copy of the config properties in the form of a Map. */ - def getAllConfs: immutable.Map[String, String] = settings.synchronized { settings.toMap } + def getAllConfs: immutable.Map[String, String] = + settings.synchronized { settings.asScala.toMap } /** * Return all the configuration definitions that have been defined in [[SQLConf]]. Each * definition contains key, defaultValue and doc. */ def getAllDefinedConfs: Seq[(String, String, String)] = sqlConfEntries.synchronized { - sqlConfEntries.values.filter(_.isPublic).map { entry => + sqlConfEntries.values.asScala.filter(_.isPublic).map { entry => (entry.key, entry.defaultValueString, entry.doc) }.toSeq } private[spark] def unsetConf(key: String): Unit = { - settings -= key + settings.remove(key) } private[spark] def unsetConf(entry: SQLConfEntry[_]): Unit = { - settings -= entry.key + settings.remove(entry.key) } private[spark] def clear(): Unit = { 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 a1eea09e0477b..4e8414af50b44 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 @@ -21,7 +21,7 @@ import java.beans.Introspector import java.util.Properties import java.util.concurrent.atomic.AtomicReference -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal @@ -225,7 +225,7 @@ class SQLContext(@transient val sparkContext: SparkContext) conf.setConf(properties) // After we have populated SQLConf, we call setConf to populate other confs in the subclass // (e.g. hiveconf in HiveContext). - properties.foreach { + properties.asScala.foreach { case (key, value) => setConf(key, value) } } @@ -567,7 +567,7 @@ class SQLContext(@transient val sparkContext: SparkContext) tableName: String, source: String, options: java.util.Map[String, String]): DataFrame = { - createExternalTable(tableName, source, options.toMap) + createExternalTable(tableName, source, options.asScala.toMap) } /** @@ -612,7 +612,7 @@ class SQLContext(@transient val sparkContext: SparkContext) source: String, schema: StructType, options: java.util.Map[String, String]): DataFrame = { - createExternalTable(tableName, source, schema, options.toMap) + createExternalTable(tableName, source, schema, options.asScala.toMap) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala index 8fbaf3a3059db..011724436621d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.util.ServiceLoader -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} import scala.util.{Success, Failure, Try} @@ -55,7 +55,7 @@ object ResolvedDataSource extends Logging { val loader = Utils.getContextOrSparkClassLoader val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], loader) - serviceLoader.iterator().filter(_.shortName().equalsIgnoreCase(provider)).toList match { + serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match { /** the provider format did not match any given registered aliases */ case Nil => Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match { case Success(dataSource) => dataSource diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 3f8353af6e2ad..0a6bb44445f6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.{Map => JMap} -import scala.collection.JavaConversions.{iterableAsScalaIterable, mapAsJavaMap, mapAsScalaMap} +import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.parquet.hadoop.api.ReadSupport.ReadContext @@ -44,7 +44,7 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with val parquetRequestedSchema = readContext.getRequestedSchema val catalystRequestedSchema = - Option(readContext.getReadSupportMetadata).map(_.toMap).flatMap { metadata => + Option(readContext.getReadSupportMetadata).map(_.asScala).flatMap { metadata => metadata // First tries to read requested schema, which may result from projections .get(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA) @@ -123,7 +123,7 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with maybeRequestedSchema.fold(context.getFileSchema) { schemaString => val toParquet = new CatalystSchemaConverter(conf) val fileSchema = context.getFileSchema.asGroupType() - val fileFieldNames = fileSchema.getFields.map(_.getName).toSet + val fileFieldNames = fileSchema.getFields.asScala.map(_.getName).toSet StructType // Deserializes the Catalyst schema of requested columns @@ -152,7 +152,7 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with maybeRequestedSchema.map(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) - new ReadContext(parquetRequestedSchema, metadata) + new ReadContext(parquetRequestedSchema, metadata.asJava) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index cbf0704c4a9a4..f682ca0d8ff4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.math.{BigDecimal, BigInteger} import java.nio.ByteOrder -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.parquet.column.Dictionary @@ -183,7 +183,7 @@ private[parquet] class CatalystRowConverter( // those missing fields and create converters for them, although values of these fields are // always null. val paddedParquetFields = { - val parquetFields = parquetType.getFields + val parquetFields = parquetType.getFields.asScala val parquetFieldNames = parquetFields.map(_.getName).toSet val missingFields = catalystType.filterNot(f => parquetFieldNames.contains(f.name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index 535f0684e97f9..be6c0545f5a0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.parquet.schema.OriginalType._ @@ -82,7 +82,7 @@ private[parquet] class CatalystSchemaConverter( def convert(parquetSchema: MessageType): StructType = convert(parquetSchema.asGroupType()) private def convert(parquetSchema: GroupType): StructType = { - val fields = parquetSchema.getFields.map { field => + val fields = parquetSchema.getFields.asScala.map { field => field.getRepetition match { case OPTIONAL => StructField(field.getName, convertField(field), nullable = true) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index bbf682aec0f9d..64982f37cf872 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -21,7 +21,7 @@ import java.net.URI import java.util.logging.{Logger => JLogger} import java.util.{List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.{Failure, Try} @@ -336,7 +336,7 @@ private[sql] class ParquetRelation( override def getPartitions: Array[SparkPartition] = { val inputFormat = new ParquetInputFormat[InternalRow] { override def listStatus(jobContext: JobContext): JList[FileStatus] = { - if (cacheMetadata) cachedStatuses else super.listStatus(jobContext) + if (cacheMetadata) cachedStatuses.asJava else super.listStatus(jobContext) } } @@ -344,7 +344,8 @@ private[sql] class ParquetRelation( val rawSplits = inputFormat.getSplits(jobContext) Array.tabulate[SparkPartition](rawSplits.size) { i => - new SqlNewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + new SqlNewHadoopPartition( + id, i, rawSplits.get(i).asInstanceOf[InputSplit with Writable]) } } }.asInstanceOf[RDD[Row]] // type erasure hack to pass RDD[InternalRow] as RDD[Row] @@ -588,7 +589,7 @@ private[sql] object ParquetRelation extends Logging { val metadata = footer.getParquetMetadata.getFileMetaData val serializedSchema = metadata .getKeyValueMetaData - .toMap + .asScala.toMap .get(CatalystReadSupport.SPARK_METADATA_KEY) if (serializedSchema.isEmpty) { // Falls back to Parquet schema if no Spark SQL schema found. @@ -745,7 +746,7 @@ private[sql] object ParquetRelation extends Logging { // Reads footers in multi-threaded manner within each task val footers = ParquetFileReader.readAllFootersInParallel( - serializedConf.value, fakeFileStatuses, skipRowGroups) + serializedConf.value, fakeFileStatuses.asJava, skipRowGroups).asScala // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` val converter = @@ -772,7 +773,7 @@ private[sql] object ParquetRelation extends Logging { val fileMetaData = footer.getParquetMetadata.getFileMetaData fileMetaData .getKeyValueMetaData - .toMap + .asScala.toMap .get(CatalystReadSupport.SPARK_METADATA_KEY) .flatMap(deserializeSchemaString) .getOrElse(converter.convert(fileMetaData.getSchema)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypesConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypesConverter.scala index 42376ef7a9c1f..142301fe87cb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypesConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypesConverter.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.IOException +import java.util.{Collections, Arrays} -import scala.collection.JavaConversions._ import scala.util.Try import org.apache.hadoop.conf.Configuration @@ -107,7 +107,7 @@ private[parquet] object ParquetTypesConverter extends Logging { ParquetFileWriter.writeMetadataFile( conf, path, - new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil) + Arrays.asList(new Footer(path, new ParquetMetadata(metaData, Collections.emptyList())))) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala index ed282f98b7d71..d800c7456bdac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.joins -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD @@ -92,9 +92,9 @@ case class ShuffledHashOuterJoin( case FullOuter => // TODO(davies): use UnsafeRow val leftHashTable = - buildHashTable(leftIter, numLeftRows, newProjection(leftKeys, left.output)) + buildHashTable(leftIter, numLeftRows, newProjection(leftKeys, left.output)).asScala val rightHashTable = - buildHashTable(rightIter, numRightRows, newProjection(rightKeys, right.output)) + buildHashTable(rightIter, numRightRows, newProjection(rightKeys, right.output)).asScala (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => fullOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala index 59f8b079ab333..5a58d846ad80b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution import java.io.OutputStream import java.util.{List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import net.razorvine.pickle._ @@ -196,14 +196,15 @@ object EvaluatePython { case (c, BinaryType) if c.getClass.isArray && c.getClass.getComponentType.getName == "byte" => c case (c: java.util.List[_], ArrayType(elementType, _)) => - new GenericArrayData(c.map { e => fromJava(e, elementType)}.toArray) + new GenericArrayData(c.asScala.map { e => fromJava(e, elementType)}.toArray) case (c, ArrayType(elementType, _)) if c.getClass.isArray => new GenericArrayData(c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType))) case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => - val keys = c.keysIterator.map(fromJava(_, keyType)).toArray - val values = c.valuesIterator.map(fromJava(_, valueType)).toArray + val keyValues = c.asScala.toSeq + val keys = keyValues.map(kv => fromJava(kv._1, keyType)).toArray + val values = keyValues.map(kv => fromJava(kv._2, valueType)).toArray ArrayBasedMapData(keys, values) case (c, StructType(fields)) if c.getClass.isArray => @@ -367,7 +368,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: val pickle = new Unpickler iter.flatMap { pickedResult => val unpickledBatch = pickle.loads(pickedResult) - unpickledBatch.asInstanceOf[java.util.ArrayList[Any]] + unpickledBatch.asInstanceOf[java.util.ArrayList[Any]].asScala } }.mapPartitions { iter => val row = new GenericMutableRow(1) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 7abdd3db80341..4867cebf5328c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -23,7 +23,7 @@ import java.util.List; import java.util.Map; -import scala.collection.JavaConversions; +import scala.collection.JavaConverters; import scala.collection.Seq; import com.google.common.collect.ImmutableMap; @@ -96,7 +96,7 @@ public void testVarargMethods() { df.groupBy().agg(countDistinct("key", "value")); df.groupBy().agg(countDistinct(col("key"), col("value"))); df.select(coalesce(col("key"))); - + // Varargs with mathfunctions DataFrame df2 = context.table("testData2"); df2.select(exp("a"), exp("b")); @@ -172,7 +172,7 @@ public void testCreateDataFrameFromJavaBeans() { Seq outputBuffer = (Seq) first.getJavaMap(2).get("hello"); Assert.assertArrayEquals( bean.getC().get("hello"), - Ints.toArray(JavaConversions.seqAsJavaList(outputBuffer))); + Ints.toArray(JavaConverters.seqAsJavaListConverter(outputBuffer).asJava())); Seq d = first.getAs(3); Assert.assertEquals(bean.getD().size(), d.length()); for (int i = 0; i < d.length(); i++) { @@ -206,7 +206,7 @@ public void testCrosstab() { count++; } } - + @Test public void testFrequentItems() { DataFrame df = context.table("testData2"); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index cdaa14ac80785..329ffb66083b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.sql.test.SharedSQLContext @@ -153,11 +153,11 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext { // Test Java version checkAnswer( - df.na.fill(mapAsJavaMap(Map( + df.na.fill(Map( "a" -> "test", "c" -> 1, "d" -> 2.2 - ))), + ).asJava), Row("test", null, 1, 2.2)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 4adcefb7dc4b1..3649c2a97b5ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.util.{Locale, TimeZone} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ @@ -145,7 +145,7 @@ object QueryTest { } def checkAnswer(df: DataFrame, expectedAnswer: java.util.List[Row]): String = { - checkAnswer(df, expectedAnswer.toSeq) match { + checkAnswer(df, expectedAnswer.asScala) match { case Some(errorMessage) => errorMessage case None => null } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala index 45db619567a22..bd7cf8c10abef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.nio.ByteBuffer import java.util.{List => JList, Map => JMap} -import scala.collection.JavaConverters.seqAsJavaListConverter -import scala.collection.JavaConverters.mapAsJavaMapConverter +import scala.collection.JavaConverters._ import org.apache.avro.Schema import org.apache.avro.generic.IndexedRecord diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala index d85c564e3e8d1..df68432faeeb3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.parquet.hadoop.ParquetFileReader @@ -40,8 +40,9 @@ private[sql] abstract class ParquetCompatibilityTest extends QueryTest with Parq override def accept(path: Path): Boolean = pathFilter(path) }).toSeq - val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true) - footers.head.getParquetMetadata.getFileMetaData.getSchema + val footers = + ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles.asJava, true) + footers.iterator().next().getParquetMetadata.getFileMetaData.getSchema } protected def logParquetSchema(path: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index e6b0a2ea95e38..08d2b9dee99b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.execution.datasources.parquet -import scala.collection.JavaConversions._ +import java.util.Collections + +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag @@ -28,7 +30,7 @@ import org.apache.parquet.example.data.simple.SimpleGroup import org.apache.parquet.example.data.{Group, GroupWriter} import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext -import org.apache.parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata} +import org.apache.parquet.hadoop.metadata.{BlockMetaData, CompressionCodecName, FileMetaData, ParquetMetadata} import org.apache.parquet.hadoop.{Footer, ParquetFileWriter, ParquetOutputCommitter, ParquetWriter} import org.apache.parquet.io.api.RecordConsumer import org.apache.parquet.schema.{MessageType, MessageTypeParser} @@ -205,9 +207,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { test("compression codec") { def compressionCodecFor(path: String): String = { val codecs = ParquetTypesConverter - .readMetaData(new Path(path), Some(configuration)) - .getBlocks - .flatMap(_.getColumns) + .readMetaData(new Path(path), Some(configuration)).getBlocks.asScala + .flatMap(_.getColumns.asScala) .map(_.getCodec.name()) .distinct @@ -348,14 +349,16 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { """.stripMargin) withTempPath { location => - val extraMetadata = Map(CatalystReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString) + val extraMetadata = Collections.singletonMap( + CatalystReadSupport.SPARK_METADATA_KEY, sparkSchema.toString) val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, "Spark") val path = new Path(location.getCanonicalPath) ParquetFileWriter.writeMetadataFile( sqlContext.sparkContext.hadoopConfiguration, path, - new Footer(path, new ParquetMetadata(fileMetadata, Nil)) :: Nil) + Collections.singletonList( + new Footer(path, new ParquetMetadata(fileMetadata, Collections.emptyList())))) assertResult(sqlContext.read.parquet(path.toString).schema) { StructType( @@ -386,7 +389,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) } } @@ -410,7 +413,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) } } @@ -434,7 +437,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) } } } @@ -481,7 +484,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 02cc7e5efa521..306f98bcb5344 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction import java.sql.{Date, Timestamp} import java.util.concurrent.RejectedExecutionException -import java.util.{Map => JMap, UUID} +import java.util.{Arrays, Map => JMap, UUID} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} import scala.util.control.NonFatal @@ -126,13 +126,13 @@ private[hive] class SparkExecuteStatementOperation( def getResultSetSchema: TableSchema = { if (result == null || result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + new TableSchema(Arrays.asList(new FieldSchema("Result", "string", ""))) } else { logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") val schema = result.queryExecution.analyzed.output.map { attr => new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") } - new TableSchema(schema) + new TableSchema(schema.asJava) } } @@ -298,7 +298,7 @@ private[hive] class SparkExecuteStatementOperation( sqlOperationConf = new HiveConf(sqlOperationConf) // apply overlay query specific settings, if any - getConfOverlay().foreach { case (k, v) => + getConfOverlay().asScala.foreach { case (k, v) => try { sqlOperationConf.verifyAndSet(k, v) } catch { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 7799704c819d9..a29df567983b1 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ - import java.io._ import java.util.{ArrayList => JArrayList, Locale} +import scala.collection.JavaConverters._ + import jline.console.ConsoleReader import jline.console.history.FileHistory @@ -101,9 +101,9 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item => - val key = item.getKey.asInstanceOf[String] - val value = item.getValue.asInstanceOf[String] + sessionState.cmdProperties.entrySet().asScala.foreach { item => + val key = item.getKey.toString + val value = item.getValue.toString // We do not propagate metastore options to the execution copy of hive. if (key != "javax.jdo.option.ConnectionURL") { conf.set(key, value) @@ -316,15 +316,15 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { // Print the column names. - Option(driver.getSchema.getFieldSchemas).map { fields => - out.println(fields.map(_.getName).mkString("\t")) + Option(driver.getSchema.getFieldSchemas).foreach { fields => + out.println(fields.asScala.map(_.getName).mkString("\t")) } } var counter = 0 try { while (!out.checkError() && driver.getResults(res)) { - res.foreach{ l => + res.asScala.foreach { l => counter += 1 out.println(l) } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 644165acf70a7..5ad8c54f296d5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -21,6 +21,8 @@ import java.io.IOException import java.util.{List => JList} import javax.security.auth.login.LoginException +import scala.collection.JavaConverters._ + import org.apache.commons.logging.Log import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.Utils @@ -34,8 +36,6 @@ import org.apache.hive.service.{AbstractService, Service, ServiceException} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import scala.collection.JavaConversions._ - private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, hiveContext: HiveContext) extends CLIService(hiveServer) with ReflectedCompositeService { @@ -76,7 +76,7 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => def initCompositeService(hiveConf: HiveConf) { // Emulating `CompositeService.init(hiveConf)` val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") - serviceList.foreach(_.init(hiveConf)) + serviceList.asScala.foreach(_.init(hiveConf)) // Emulating `AbstractService.init(hiveConf)` invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 77272aecf2835..2619286afc148 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.{ArrayList => JArrayList, List => JList} +import java.util.{Arrays, ArrayList => JArrayList, List => JList} + +import scala.collection.JavaConverters._ import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} @@ -27,8 +29,6 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import scala.collection.JavaConversions._ - private[hive] class SparkSQLDriver( val context: HiveContext = SparkSQLEnv.hiveContext) extends Driver @@ -43,14 +43,14 @@ private[hive] class SparkSQLDriver( private def getResultSetSchema(query: context.QueryExecution): Schema = { val analyzed = query.analyzed logDebug(s"Result Schema: ${analyzed.output}") - if (analyzed.output.size == 0) { - new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) + if (analyzed.output.isEmpty) { + new Schema(Arrays.asList(new FieldSchema("Response code", "string", "")), null) } else { val fieldSchemas = analyzed.output.map { attr => new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") } - new Schema(fieldSchemas, null) + new Schema(fieldSchemas.asJava, null) } } @@ -79,7 +79,7 @@ private[hive] class SparkSQLDriver( if (hiveResponse == null) { false } else { - res.asInstanceOf[JArrayList[String]].addAll(hiveResponse) + res.asInstanceOf[JArrayList[String]].addAll(hiveResponse.asJava) hiveResponse = null true } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 1d41c46131828..bacf6cc458fd5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.PrintStream -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.HiveContext @@ -64,7 +64,7 @@ private[hive] object SparkSQLEnv extends Logging { hiveContext.setConf("spark.sql.hive.version", HiveContext.hiveExecutionVersion) if (log.isDebugEnabled) { - hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => + hiveContext.hiveconf.getAllProperties.asScala.toSeq.sorted.foreach { case (k, v) => logDebug(s"HiveConf var: $k=$v") } } 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 index 17cc83087fb1d..c0a458fa9ab8d 100644 --- 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 @@ -22,7 +22,7 @@ import java.net.{URL, URLClassLoader} import java.sql.Timestamp import java.util.concurrent.TimeUnit -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.concurrent.duration._ @@ -194,7 +194,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { logInfo("defalt warehouse location is " + defaltWarehouseLocation) // `configure` goes second to override other settings. - val allConfig = metadataConf.iterator.map(e => e.getKey -> e.getValue).toMap ++ configure + val allConfig = metadataConf.asScala.map(e => e.getKey -> e.getValue).toMap ++ configure val isolatedLoader = if (hiveMetastoreJars == "builtin") { if (hiveExecutionVersion != hiveMetastoreVersion) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 64fffdbf9b020..cfe2bb05ad89e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import scala.collection.JavaConverters._ + import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.objectinspector.{StructField => HiveStructField, _} @@ -31,9 +33,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.{AnalysisException, types} import org.apache.spark.unsafe.types.UTF8String -/* Implicit conversions */ -import scala.collection.JavaConversions._ - /** * 1. The Underlying data type in catalyst and in Hive * In catalyst: @@ -290,13 +289,13 @@ private[hive] trait HiveInspectors { DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get()) case mi: StandardConstantMapObjectInspector => // take the value from the map inspector object, rather than the input data - val map = mi.getWritableConstantValue - val keys = map.keysIterator.map(unwrap(_, mi.getMapKeyObjectInspector)).toArray - val values = map.valuesIterator.map(unwrap(_, mi.getMapValueObjectInspector)).toArray + val keyValues = mi.getWritableConstantValue.asScala.toSeq + val keys = keyValues.map(kv => unwrap(kv._1, mi.getMapKeyObjectInspector)).toArray + val values = keyValues.map(kv => unwrap(kv._2, mi.getMapValueObjectInspector)).toArray ArrayBasedMapData(keys, values) case li: StandardConstantListObjectInspector => // take the value from the list inspector object, rather than the input data - val values = li.getWritableConstantValue + val values = li.getWritableConstantValue.asScala .map(unwrap(_, li.getListElementObjectInspector)) .toArray new GenericArrayData(values) @@ -342,7 +341,7 @@ private[hive] trait HiveInspectors { case li: ListObjectInspector => Option(li.getList(data)) .map { l => - val values = l.map(unwrap(_, li.getListElementObjectInspector)).toArray + val values = l.asScala.map(unwrap(_, li.getListElementObjectInspector)).toArray new GenericArrayData(values) } .orNull @@ -351,15 +350,16 @@ private[hive] trait HiveInspectors { if (map == null) { null } else { - val keys = map.keysIterator.map(unwrap(_, mi.getMapKeyObjectInspector)).toArray - val values = map.valuesIterator.map(unwrap(_, mi.getMapValueObjectInspector)).toArray + val keyValues = map.asScala.toSeq + val keys = keyValues.map(kv => unwrap(kv._1, mi.getMapKeyObjectInspector)).toArray + val values = keyValues.map(kv => unwrap(kv._2, mi.getMapValueObjectInspector)).toArray ArrayBasedMapData(keys, values) } // currently, hive doesn't provide the ConstantStructObjectInspector case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs - InternalRow.fromSeq( - allRefs.map(r => unwrap(si.getStructFieldData(data, r), r.getFieldObjectInspector))) + InternalRow.fromSeq(allRefs.asScala.map( + r => unwrap(si.getStructFieldData(data, r), r.getFieldObjectInspector))) } @@ -403,14 +403,14 @@ private[hive] trait HiveInspectors { case soi: StandardStructObjectInspector => val schema = dataType.asInstanceOf[StructType] - val wrappers = soi.getAllStructFieldRefs.zip(schema.fields).map { case (ref, field) => - wrapperFor(ref.getFieldObjectInspector, field.dataType) + val wrappers = soi.getAllStructFieldRefs.asScala.zip(schema.fields).map { + case (ref, field) => wrapperFor(ref.getFieldObjectInspector, field.dataType) } (o: Any) => { if (o != null) { val struct = soi.create() val row = o.asInstanceOf[InternalRow] - soi.getAllStructFieldRefs.zip(wrappers).zipWithIndex.foreach { + soi.getAllStructFieldRefs.asScala.zip(wrappers).zipWithIndex.foreach { case ((field, wrapper), i) => soi.setStructFieldData(struct, field, wrapper(row.get(i, schema(i).dataType))) } @@ -537,7 +537,7 @@ private[hive] trait HiveInspectors { // 1. create the pojo (most likely) object val result = x.create() var i = 0 - while (i < fieldRefs.length) { + while (i < fieldRefs.size) { // 2. set the property for the pojo val tpe = structType(i).dataType x.setStructFieldData( @@ -552,9 +552,9 @@ private[hive] trait HiveInspectors { val fieldRefs = x.getAllStructFieldRefs val structType = dataType.asInstanceOf[StructType] val row = a.asInstanceOf[InternalRow] - val result = new java.util.ArrayList[AnyRef](fieldRefs.length) + val result = new java.util.ArrayList[AnyRef](fieldRefs.size) var i = 0 - while (i < fieldRefs.length) { + while (i < fieldRefs.size) { val tpe = structType(i).dataType result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) i += 1 @@ -712,10 +712,10 @@ private[hive] trait HiveInspectors { def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { case s: StructObjectInspector => - StructType(s.getAllStructFieldRefs.map(f => { + StructType(s.getAllStructFieldRefs.asScala.map(f => types.StructField( f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) - })) + )) case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) case m: MapObjectInspector => MapType( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 98d21aa76d64e..b8da0840ae569 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import com.google.common.base.Objects @@ -483,7 +483,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive // are empty. val partitions = metastoreRelation.getHiveQlPartitions().map { p => val location = p.getLocation - val values = InternalRow.fromSeq(p.getValues.zip(partitionColumnDataTypes).map { + val values = InternalRow.fromSeq(p.getValues.asScala.zip(partitionColumnDataTypes).map { case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) }) ParquetPartition(values, location) @@ -798,9 +798,9 @@ private[hive] case class MetastoreRelation val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tTable.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) tTable.setPartitionKeys( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) table.location.foreach(sd.setLocation) table.inputFormat.foreach(sd.setInputFormat) @@ -852,11 +852,11 @@ private[hive] case class MetastoreRelation val tPartition = new org.apache.hadoop.hive.metastore.api.Partition tPartition.setDbName(databaseName) tPartition.setTableName(tableName) - tPartition.setValues(p.values) + tPartition.setValues(p.values.asJava) val sd = new org.apache.hadoop.hive.metastore.api.StorageDescriptor() tPartition.setSd(sd) - sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + sd.setCols(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) sd.setLocation(p.storage.location) sd.setInputFormat(p.storage.inputFormat) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index ad33dee555dd2..d5cd7e98b5267 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -20,6 +20,9 @@ package org.apache.spark.sql.hive import java.sql.Date import java.util.Locale +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.serde.serdeConstants @@ -48,10 +51,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler -/* Implicit conversions */ -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer - /** * Used when we need to start parsing the AST before deciding that we are going to pass the command * back for Hive to execute natively. Will be replaced with a native command that contains the @@ -202,7 +201,7 @@ private[hive] object HiveQl extends Logging { * Returns a scala.Seq equivalent to [s] or Nil if [s] is null. */ private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] = - Option(s).map(_.toSeq).getOrElse(Nil) + Option(s).map(_.asScala).getOrElse(Nil) /** * Returns this ASTNode with the text changed to `newText`. @@ -217,7 +216,7 @@ private[hive] object HiveQl extends Logging { */ def withChildren(newChildren: Seq[ASTNode]): ASTNode = { (1 to n.getChildCount).foreach(_ => n.deleteChild(0)) - n.addChildren(newChildren) + n.addChildren(newChildren.asJava) n } @@ -323,11 +322,11 @@ private[hive] object HiveQl extends Logging { assert(tree.asInstanceOf[ASTNode].getText == "TOK_CREATETABLE", "Only CREATE TABLE supported.") val tableOps = tree.getChildren val colList = - tableOps + tableOps.asScala .find(_.asInstanceOf[ASTNode].getText == "TOK_TABCOLLIST") .getOrElse(sys.error("No columnList!")).getChildren - colList.map(nodeToAttribute) + colList.asScala.map(nodeToAttribute) } /** Extractor for matching Hive's AST Tokens. */ @@ -337,7 +336,7 @@ private[hive] object HiveQl extends Logging { case t: ASTNode => CurrentOrigin.setPosition(t.getLine, t.getCharPositionInLine) Some((t.getText, - Option(t.getChildren).map(_.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) + Option(t.getChildren).map(_.asScala.toList).getOrElse(Nil).asInstanceOf[Seq[ASTNode]])) case _ => None } } @@ -424,7 +423,9 @@ private[hive] object HiveQl extends Logging { protected def extractDbNameTableName(tableNameParts: Node): (Option[String], String) = { val (db, tableName) = - tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) } match { + tableNameParts.getChildren.asScala.map { + case Token(part, Nil) => cleanIdentifier(part) + } match { case Seq(tableOnly) => (None, tableOnly) case Seq(databaseName, table) => (Some(databaseName), table) } @@ -433,7 +434,9 @@ private[hive] object HiveQl extends Logging { } protected def extractTableIdent(tableNameParts: Node): Seq[String] = { - tableNameParts.getChildren.map { case Token(part, Nil) => cleanIdentifier(part) } match { + tableNameParts.getChildren.asScala.map { + case Token(part, Nil) => cleanIdentifier(part) + } match { case Seq(tableOnly) => Seq(tableOnly) case Seq(databaseName, table) => Seq(databaseName, table) case other => sys.error("Hive only supports tables names like 'tableName' " + @@ -624,7 +627,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val cols = BaseSemanticAnalyzer.getColumns(list, true) if (cols != null) { tableDesc = tableDesc.copy( - schema = cols.map { field => + schema = cols.asScala.map { field => HiveColumn(field.getName, field.getType, field.getComment) }) } @@ -636,7 +639,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val cols = BaseSemanticAnalyzer.getColumns(list(0), false) if (cols != null) { tableDesc = tableDesc.copy( - partitionColumns = cols.map { field => + partitionColumns = cols.asScala.map { field => HiveColumn(field.getName, field.getType, field.getComment) }) } @@ -672,7 +675,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case _ => assert(false) } tableDesc = tableDesc.copy( - serdeProperties = tableDesc.serdeProperties ++ serdeParams) + serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) case Token("TOK_TABLELOCATION", child :: Nil) => var location = BaseSemanticAnalyzer.unescapeSQLString(child.getText) location = EximUtil.relativeToAbsolutePath(hiveConf, location) @@ -684,7 +687,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val serdeParams = new java.util.HashMap[String, String]() BaseSemanticAnalyzer.readProps( (child.getChild(1).getChild(0)).asInstanceOf[ASTNode], serdeParams) - tableDesc = tableDesc.copy(serdeProperties = tableDesc.serdeProperties ++ serdeParams) + tableDesc = tableDesc.copy( + serdeProperties = tableDesc.serdeProperties ++ serdeParams.asScala) } case Token("TOK_FILEFORMAT_GENERIC", child :: Nil) => child.getText().toLowerCase(Locale.ENGLISH) match { @@ -847,7 +851,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } val withWhere = whereClause.map { whereNode => - val Seq(whereExpr) = whereNode.getChildren.toSeq + val Seq(whereExpr) = whereNode.getChildren.asScala Filter(nodeToExpr(whereExpr), relations) }.getOrElse(relations) @@ -856,7 +860,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // Script transformations are expressed as a select clause with a single expression of type // TOK_TRANSFORM - val transformation = select.getChildren.head match { + val transformation = select.getChildren.iterator().next() match { case Token("TOK_SELEXPR", Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: @@ -925,10 +929,10 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val withLateralView = lateralViewClause.map { lv => val Token("TOK_SELECT", - Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.head + Token("TOK_SELEXPR", clauses) :: Nil) = lv.getChildren.iterator().next() - val alias = - getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText + val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() + .asInstanceOf[ASTNode].getText val (generator, attributes) = nodesToGenerator(clauses) Generate( @@ -944,7 +948,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // (if there is a group by) or a script transformation. val withProject: LogicalPlan = transformation.getOrElse { val selectExpressions = - select.getChildren.flatMap(selExprNodeToExpr).map(UnresolvedAlias(_)).toSeq + select.getChildren.asScala.flatMap(selExprNodeToExpr).map(UnresolvedAlias) Seq( groupByClause.map(e => e match { case Token("TOK_GROUPBY", children) => @@ -973,7 +977,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // Handle HAVING clause. val withHaving = havingClause.map { h => - val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } + val havingExpr = h.getChildren.asScala match { case Seq(hexpr) => nodeToExpr(hexpr) } // Note that we added a cast to boolean. If the expression itself is already boolean, // the optimizer will get rid of the unnecessary cast. Filter(Cast(havingExpr, BooleanType), withProject) @@ -983,32 +987,42 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val withDistinct = if (selectDistinctClause.isDefined) Distinct(withHaving) else withHaving - // Handle ORDER BY, SORT BY, DISTRIBETU BY, and CLUSTER BY clause. + // Handle ORDER BY, SORT BY, DISTRIBUTE BY, and CLUSTER BY clause. val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withDistinct) + Sort(totalOrdering.getChildren.asScala.map(nodeToSortOrder), true, withDistinct) case (None, Some(perPartitionOrdering), None, None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withDistinct) + Sort( + perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), + false, withDistinct) case (None, None, Some(partitionExprs), None) => - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct) + RepartitionByExpression( + partitionExprs.getChildren.asScala.map(nodeToExpr), withDistinct) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, - RepartitionByExpression(partitionExprs.getChildren.map(nodeToExpr), withDistinct)) + Sort( + perPartitionOrdering.getChildren.asScala.map(nodeToSortOrder), false, + RepartitionByExpression( + partitionExprs.getChildren.asScala.map(nodeToExpr), + withDistinct)) case (None, None, None, Some(clusterExprs)) => - Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, - RepartitionByExpression(clusterExprs.getChildren.map(nodeToExpr), withDistinct)) + Sort( + clusterExprs.getChildren.asScala.map(nodeToExpr).map(SortOrder(_, Ascending)), + false, + RepartitionByExpression( + clusterExprs.getChildren.asScala.map(nodeToExpr), + withDistinct)) case (None, None, None, None) => withDistinct case _ => sys.error("Unsupported set of ordering / distribution clauses.") } val withLimit = - limitClause.map(l => nodeToExpr(l.getChildren.head)) + limitClause.map(l => nodeToExpr(l.getChildren.iterator().next())) .map(Limit(_, withSort)) .getOrElse(withSort) // Collect all window specifications defined in the WINDOW clause. - val windowDefinitions = windowClause.map(_.getChildren.toSeq.collect { + val windowDefinitions = windowClause.map(_.getChildren.asScala.collect { case Token("TOK_WINDOWDEF", Token(windowName, Nil) :: Token("TOK_WINDOWSPEC", spec) :: Nil) => windowName -> nodesToWindowSpecification(spec) @@ -1063,7 +1077,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val Token("TOK_SELECT", Token("TOK_SELEXPR", clauses) :: Nil) = selectClause - val alias = getClause("TOK_TABALIAS", clauses).getChildren.head.asInstanceOf[ASTNode].getText + val alias = getClause("TOK_TABALIAS", clauses).getChildren.iterator().next() + .asInstanceOf[ASTNode].getText val (generator, attributes) = nodesToGenerator(clauses) Generate( @@ -1092,7 +1107,9 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } val tableIdent = - tableNameParts.getChildren.map{ case Token(part, Nil) => cleanIdentifier(part)} match { + tableNameParts.getChildren.asScala.map { + case Token(part, Nil) => cleanIdentifier(part) + } match { case Seq(tableOnly) => Seq(tableOnly) case Seq(databaseName, table) => Seq(databaseName, table) case other => sys.error("Hive only supports tables names like 'tableName' " + @@ -1139,7 +1156,8 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val isPreserved = tableOrdinals.map(i => (i - 1 < 0) || joinArgs(i - 1).getText == "PRESERVE") val tables = tableOrdinals.map(i => nodeToRelation(joinArgs(i))) - val joinExpressions = tableOrdinals.map(i => joinArgs(i + 1).getChildren.map(nodeToExpr)) + val joinExpressions = + tableOrdinals.map(i => joinArgs(i + 1).getChildren.asScala.map(nodeToExpr)) val joinConditions = joinExpressions.sliding(2).map { case Seq(c1, c2) => @@ -1164,7 +1182,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C joinType = joinType.remove(joinType.length - 1)) } - val groups = (0 until joinExpressions.head.size).map(i => Coalesce(joinExpressions.map(_(i)))) + val groups = joinExpressions.head.indices.map(i => Coalesce(joinExpressions.map(_(i)))) // Unique join is not really the same as an outer join so we must group together results where // the joinExpressions are the same, taking the First of each value is only okay because the @@ -1229,7 +1247,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val tableIdent = extractTableIdent(tableNameParts) - val partitionKeys = partitionClause.map(_.getChildren.map { + val partitionKeys = partitionClause.map(_.getChildren.asScala.map { // Parse partitions. We also make keys case insensitive. case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) @@ -1249,7 +1267,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val tableIdent = extractTableIdent(tableNameParts) - val partitionKeys = partitionClause.map(_.getChildren.map { + val partitionKeys = partitionClause.map(_.getChildren.asScala.map { // Parse partitions. We also make keys case insensitive. case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) @@ -1590,18 +1608,18 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val (partitionByClause :: orderByClause :: sortByClause :: clusterByClause :: Nil) = getClauses( Seq("TOK_DISTRIBUTEBY", "TOK_ORDERBY", "TOK_SORTBY", "TOK_CLUSTERBY"), - partitionAndOrdering.getChildren.toSeq.asInstanceOf[Seq[ASTNode]]) + partitionAndOrdering.getChildren.asScala.asInstanceOf[Seq[ASTNode]]) (partitionByClause, orderByClause.orElse(sortByClause), clusterByClause) match { case (Some(partitionByExpr), Some(orderByExpr), None) => - (partitionByExpr.getChildren.map(nodeToExpr), - orderByExpr.getChildren.map(nodeToSortOrder)) + (partitionByExpr.getChildren.asScala.map(nodeToExpr), + orderByExpr.getChildren.asScala.map(nodeToSortOrder)) case (Some(partitionByExpr), None, None) => - (partitionByExpr.getChildren.map(nodeToExpr), Nil) + (partitionByExpr.getChildren.asScala.map(nodeToExpr), Nil) case (None, Some(orderByExpr), None) => - (Nil, orderByExpr.getChildren.map(nodeToSortOrder)) + (Nil, orderByExpr.getChildren.asScala.map(nodeToSortOrder)) case (None, None, Some(clusterByExpr)) => - val expressions = clusterByExpr.getChildren.map(nodeToExpr) + val expressions = clusterByExpr.getChildren.asScala.map(nodeToExpr) (expressions, expressions.map(SortOrder(_, Ascending))) case _ => throw new NotImplementedError( @@ -1639,7 +1657,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } rowFrame.orElse(rangeFrame).map { frame => - frame.getChildren.toList match { + frame.getChildren.asScala.toList match { case precedingNode :: followingNode :: Nil => SpecifiedWindowFrame( frameType, @@ -1701,7 +1719,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case other => sys.error(s"Non ASTNode encountered: $other") } - Option(node.getChildren).map(_.toList).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) + Option(node.getChildren).map(_.asScala).getOrElse(Nil).foreach(dumpTree(_, builder, indent + 1)) builder } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index 267074f3ad102..004805f3aed0b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -22,8 +22,7 @@ import java.rmi.server.UID import org.apache.avro.Schema -/* Implicit conversions */ -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -73,7 +72,7 @@ private[hive] object HiveShim { */ def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { if (ids != null && ids.nonEmpty) { - ColumnProjectionUtils.appendReadColumns(conf, ids) + ColumnProjectionUtils.appendReadColumns(conf, ids.asJava) } if (names != null && names.nonEmpty) { appendReadColumnNames(conf, names) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index f49c97de8ff4e..4d1e3ed9198e6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -21,7 +21,7 @@ import java.io.{File, PrintStream} import java.util.{Map => JMap} import javax.annotation.concurrent.GuardedBy -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path @@ -305,10 +305,11 @@ private[hive] class ClientWrapper( HiveTable( name = h.getTableName, specifiedDatabase = Option(h.getDbName), - schema = h.getCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), - partitionColumns = h.getPartCols.map(f => HiveColumn(f.getName, f.getType, f.getComment)), - properties = h.getParameters.toMap, - serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, + schema = h.getCols.asScala.map(f => HiveColumn(f.getName, f.getType, f.getComment)), + partitionColumns = h.getPartCols.asScala.map(f => + HiveColumn(f.getName, f.getType, f.getComment)), + properties = h.getParameters.asScala.toMap, + serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.asScala.toMap, tableType = h.getTableType match { case HTableType.MANAGED_TABLE => ManagedTable case HTableType.EXTERNAL_TABLE => ExternalTable @@ -334,9 +335,9 @@ private[hive] class ClientWrapper( private def toQlTable(table: HiveTable): metadata.Table = { val qlTable = new metadata.Table(table.database, table.name) - qlTable.setFields(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + qlTable.setFields(table.schema.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) qlTable.setPartCols( - table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment))) + table.partitionColumns.map(c => new FieldSchema(c.name, c.hiveType, c.comment)).asJava) table.properties.foreach { case (k, v) => qlTable.setProperty(k, v) } table.serdeProperties.foreach { case (k, v) => qlTable.setSerdeParam(k, v) } @@ -366,13 +367,13 @@ private[hive] class ClientWrapper( private def toHivePartition(partition: metadata.Partition): HivePartition = { val apiPartition = partition.getTPartition HivePartition( - values = Option(apiPartition.getValues).map(_.toSeq).getOrElse(Seq.empty), + values = Option(apiPartition.getValues).map(_.asScala).getOrElse(Seq.empty), storage = HiveStorageDescriptor( location = apiPartition.getSd.getLocation, inputFormat = apiPartition.getSd.getInputFormat, outputFormat = apiPartition.getSd.getOutputFormat, serde = apiPartition.getSd.getSerdeInfo.getSerializationLib, - serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.toMap)) + serdeProperties = apiPartition.getSd.getSerdeInfo.getParameters.asScala.toMap)) } override def getPartitionOption( @@ -397,7 +398,7 @@ private[hive] class ClientWrapper( } override def listTables(dbName: String): Seq[String] = withHiveState { - client.getAllTables(dbName) + client.getAllTables(dbName).asScala } /** @@ -514,17 +515,17 @@ private[hive] class ClientWrapper( } def reset(): Unit = withHiveState { - client.getAllTables("default").foreach { t => + client.getAllTables("default").asScala.foreach { t => logDebug(s"Deleting table $t") val table = client.getTable("default", t) - client.getIndexes("default", t, 255).foreach { index => + client.getIndexes("default", t, 255).asScala.foreach { index => shim.dropIndex(client, "default", t, index.getIndexName) } if (!table.isIndexTable) { client.dropTable("default", t) } } - client.getAllDatabases.filterNot(_ == "default").foreach { db => + client.getAllDatabases.asScala.filterNot(_ == "default").foreach { db => logDebug(s"Dropping Database: $db") client.dropDatabase(db, true, false, true) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 8fc8935b1dc3c..48bbb21e6c1de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -23,7 +23,7 @@ import java.net.URI import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JSet} import java.util.concurrent.TimeUnit -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -201,7 +201,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { setDataLocationMethod.invoke(table, new URI(loc)) override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = - getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq override def getPartitionsByFilter( hive: Hive, @@ -220,7 +220,7 @@ private[client] class Shim_v0_12 extends Shim with Logging { override def getDriverResults(driver: Driver): Seq[String] = { val res = new JArrayList[String]() getDriverResultsMethod.invoke(driver, res) - res.toSeq + res.asScala } override def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long = { @@ -310,7 +310,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { setDataLocationMethod.invoke(table, new Path(loc)) override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = - getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].asScala.toSeq /** * Converts catalyst expression to the format that Hive's getPartitionsByFilter() expects, i.e. @@ -320,7 +320,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { */ def convertFilters(table: Table, filters: Seq[Expression]): String = { // hive varchar is treated as catalyst string, but hive varchar can't be pushed down. - val varcharKeys = table.getPartitionKeys + val varcharKeys = table.getPartitionKeys.asScala .filter(col => col.getType.startsWith(serdeConstants.VARCHAR_TYPE_NAME)) .map(col => col.getName).toSet @@ -354,7 +354,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { getPartitionsByFilterMethod.invoke(hive, table, filter).asInstanceOf[JArrayList[Partition]] } - partitions.toSeq + partitions.asScala.toSeq } override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor = @@ -363,7 +363,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { override def getDriverResults(driver: Driver): Seq[String] = { val res = new JArrayList[Object]() getDriverResultsMethod.invoke(driver, res) - res.map { r => + res.asScala.map { r => r match { case s: String => s case a: Array[Object] => a(0).asInstanceOf[String] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 5f0ed5393d191..441b6b6033e1f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.hive.metastore.api.FieldSchema @@ -39,8 +39,8 @@ case class DescribeHiveTableCommand( // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil - val columns: Seq[FieldSchema] = table.hiveQlTable.getCols - val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols + val columns: Seq[FieldSchema] = table.hiveQlTable.getCols.asScala + val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols.asScala results ++= columns.map(field => (field.getName, field.getType, field.getComment)) if (partitionColumns.nonEmpty) { val partColumnInfo = @@ -48,7 +48,7 @@ case class DescribeHiveTableCommand( results ++= partColumnInfo ++ Seq(("# Partition Information", "", "")) ++ - Seq((s"# ${output.get(0).name}", output.get(1).name, output.get(2).name)) ++ + Seq((s"# ${output(0).name}", output(1).name, output(2).name)) ++ partColumnInfo } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index ba7eb15a1c0c6..806d2b9b0b7d4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} @@ -98,7 +98,7 @@ case class HiveTableScan( .asInstanceOf[StructObjectInspector] val columnTypeNames = structOI - .getAllStructFieldRefs + .getAllStructFieldRefs.asScala .map(_.getFieldObjectInspector) .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) .mkString(",") @@ -118,9 +118,8 @@ case class HiveTableScan( case None => partitions case Some(shouldKeep) => partitions.filter { part => val dataTypes = relation.partitionKeys.map(_.dataType) - val castedValues = for ((value, dataType) <- part.getValues.zip(dataTypes)) yield { - castFromString(value, dataType) - } + val castedValues = part.getValues.asScala.zip(dataTypes) + .map { case (value, dataType) => castFromString(value, dataType) } // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 62efda613a176..58f7fa640e8a9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.hive.execution import java.util +import scala.collection.JavaConverters._ + import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer @@ -38,8 +39,6 @@ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.sql.hive._ import org.apache.spark.sql.types.DataType import org.apache.spark.{SparkException, TaskContext} - -import scala.collection.JavaConversions._ import org.apache.spark.util.SerializableJobConf private[hive] @@ -94,7 +93,8 @@ case class InsertIntoHiveTable( ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val fieldOIs = standardOI.getAllStructFieldRefs.asScala + .map(_.getFieldObjectInspector).toArray val dataTypes: Array[DataType] = child.output.map(_.dataType).toArray val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt)} val outputData = new Array[Any](fieldOIs.length) @@ -198,7 +198,7 @@ case class InsertIntoHiveTable( // loadPartition call orders directories created on the iteration order of the this map val orderedPartitionSpec = new util.LinkedHashMap[String, String]() - table.hiveQlTable.getPartCols().foreach { entry => + table.hiveQlTable.getPartCols.asScala.foreach { entry => orderedPartitionSpec.put(entry.getName, partitionSpec.get(entry.getName).getOrElse("")) } @@ -226,7 +226,7 @@ case class InsertIntoHiveTable( val oldPart = catalog.client.getPartitionOption( catalog.client.getTable(table.databaseName, table.tableName), - partitionSpec) + partitionSpec.asJava) if (oldPart.isEmpty || !ifNotExists) { catalog.client.loadPartition( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index ade27454b9d29..c7651daffe36e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -21,7 +21,7 @@ import java.io._ import java.util.Properties import javax.annotation.Nullable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import org.apache.hadoop.hive.serde.serdeConstants @@ -61,7 +61,7 @@ case class ScriptTransformation( protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow]): Iterator[InternalRow] = { val cmd = List("/bin/bash", "-c", script) - val builder = new ProcessBuilder(cmd) + val builder = new ProcessBuilder(cmd.asJava) val proc = builder.start() val inputStream = proc.getInputStream @@ -172,10 +172,10 @@ case class ScriptTransformation( val fieldList = outputSoi.getAllStructFieldRefs() var i = 0 while (i < dataList.size()) { - if (dataList(i) == null) { + if (dataList.get(i) == null) { mutableRow.setNullAt(i) } else { - mutableRow(i) = unwrap(dataList(i), fieldList(i).getFieldObjectInspector) + mutableRow(i) = unwrap(dataList.get(i), fieldList.get(i).getFieldObjectInspector) } i += 1 } @@ -307,7 +307,7 @@ case class HiveScriptIOSchema ( val serde = initSerDe(serdeClass, columns, columnTypes, inputSerdeProps) val fieldObjectInspectors = columnTypes.map(toInspector) val objectInspector = ObjectInspectorFactory - .getStandardStructObjectInspector(columns, fieldObjectInspectors) + .getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava) .asInstanceOf[ObjectInspector] (serde, objectInspector) } @@ -342,7 +342,7 @@ case class HiveScriptIOSchema ( propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) val properties = new Properties() - properties.putAll(propsMap) + properties.putAll(propsMap.asJava) serde.initialize(null, properties) serde diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 7182246e466a4..cad02373e5ba1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.Try import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ConstantObjectInspector} @@ -81,8 +81,7 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry) /* List all of the registered function names. */ override def listFunction(): Seq[String] = { - val a = FunctionRegistry.getFunctionNames ++ underlying.listFunction() - a.toList.sorted + (FunctionRegistry.getFunctionNames.asScala ++ underlying.listFunction()).toList.sorted } /* Get the class of the registered function by specified name. */ @@ -116,7 +115,7 @@ private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, childre @transient private lazy val method = - function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo).asJava) @transient private lazy val arguments = children.map(toInspector).toArray @@ -541,7 +540,7 @@ private[hive] case class HiveGenericUDTF( @transient protected lazy val collector = new UDTFCollector - lazy val elementTypes = outputInspector.getAllStructFieldRefs.map { + lazy val elementTypes = outputInspector.getAllStructFieldRefs.asScala.map { field => (inspectorToDataType(field.getFieldObjectInspector), true) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 9f4f8b5789afe..1cff5cf9c3543 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.orc import java.util.Properties +import scala.collection.JavaConverters._ + import com.google.common.base.Objects import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -43,9 +45,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.util.SerializableConfiguration -/* Implicit conversions */ -import scala.collection.JavaConversions._ - private[sql] class DefaultSource extends HadoopFsRelationProvider with DataSourceRegister { override def shortName(): String = "orc" @@ -97,7 +96,8 @@ private[orc] class OrcOutputWriter( private val reusableOutputBuffer = new Array[Any](dataSchema.length) // Used to convert Catalyst values into Hadoop `Writable`s. - private val wrappers = structOI.getAllStructFieldRefs.zip(dataSchema.fields.map(_.dataType)) + private val wrappers = structOI.getAllStructFieldRefs.asScala + .zip(dataSchema.fields.map(_.dataType)) .map { case (ref, dt) => wrapperFor(ref.getFieldObjectInspector, dt) }.toArray diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 4da86636ac100..572eaebe81ac2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.implicitConversions @@ -37,9 +38,6 @@ import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.{SparkConf, SparkContext} -/* Implicit conversions */ -import scala.collection.JavaConversions._ - // SPARK-3729: Test key required to check for initialization errors with config. object TestHive extends TestHiveContext( @@ -405,7 +403,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { def reset() { try { // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach { log => + org.apache.log4j.LogManager.getCurrentLoggers.asScala.foreach { log => log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } @@ -415,9 +413,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { catalog.client.reset() catalog.unregisterAllTables() - FunctionRegistry.getFunctionNames.filterNot(originalUDFs.contains(_)).foreach { udfName => - FunctionRegistry.unregisterTemporaryUDF(udfName) - } + FunctionRegistry.getFunctionNames.asScala.filterNot(originalUDFs.contains(_)). + foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } // Some tests corrupt this value on purpose, which breaks the RESET call below. hiveconf.set("fs.default.name", new File(".").toURI.toString) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 0efcf80bd4ea7..5e7b93d457106 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.client -import scala.collection.JavaConversions._ +import java.util.Collections import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.serde.serdeConstants @@ -38,7 +38,7 @@ class FiltersSuite extends SparkFunSuite with Logging { private val varCharCol = new FieldSchema() varCharCol.setName("varchar") varCharCol.setType(serdeConstants.VARCHAR_TYPE_NAME) - testTable.setPartCols(varCharCol :: Nil) + testTable.setPartCols(Collections.singletonList(varCharCol)) filterTest("string filter", (a("stringcol", StringType) > Literal("test")) :: Nil, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index b03a35132325d..9c10ffe1113dc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.{DataInput, DataOutput} -import java.util -import java.util.Properties +import java.util.{ArrayList, Arrays, Properties} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.ql.udf.generic.{GenericUDAFAverage, GenericUDF} @@ -33,8 +32,6 @@ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.util.Utils -import scala.collection.JavaConversions._ - case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int) // Case classes for the custom UDF's. @@ -326,11 +323,11 @@ class PairSerDe extends AbstractSerDe { override def getObjectInspector: ObjectInspector = { ObjectInspectorFactory .getStandardStructObjectInspector( - Seq("pair"), - Seq(ObjectInspectorFactory.getStandardStructObjectInspector( - Seq("id", "value"), - Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, - PrimitiveObjectInspectorFactory.javaIntObjectInspector)) + Arrays.asList("pair"), + Arrays.asList(ObjectInspectorFactory.getStandardStructObjectInspector( + Arrays.asList("id", "value"), + Arrays.asList(PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.javaIntObjectInspector)) )) } @@ -343,10 +340,10 @@ class PairSerDe extends AbstractSerDe { override def deserialize(value: Writable): AnyRef = { val pair = value.asInstanceOf[TestPair] - val row = new util.ArrayList[util.ArrayList[AnyRef]] - row.add(new util.ArrayList[AnyRef](2)) - row(0).add(Integer.valueOf(pair.entry._1)) - row(0).add(Integer.valueOf(pair.entry._2)) + val row = new ArrayList[ArrayList[AnyRef]] + row.add(new ArrayList[AnyRef](2)) + row.get(0).add(Integer.valueOf(pair.entry._1)) + row.get(0).add(Integer.valueOf(pair.entry._2)) row } @@ -355,9 +352,9 @@ class PairSerDe extends AbstractSerDe { class PairUDF extends GenericUDF { override def initialize(p1: Array[ObjectInspector]): ObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( - Seq("id", "value"), - Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, - PrimitiveObjectInspectorFactory.javaIntObjectInspector) + Arrays.asList("id", "value"), + Arrays.asList(PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.javaIntObjectInspector) ) override def evaluate(args: Array[DeferredObject]): AnyRef = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 3bf8f3ac20480..210d566745415 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.hive.execution +import scala.collection.JavaConverters._ + import org.scalatest.BeforeAndAfter import org.apache.spark.sql.hive.test.TestHive -/* Implicit conversions */ -import scala.collection.JavaConversions._ - /** * A set of test cases that validate partition and column pruning. */ @@ -161,7 +160,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") - val actualPartitions = actualPartValues.map(_.toSeq.mkString(",")).sorted + val actualPartitions = actualPartValues.map(_.asScala.mkString(",")).sorted val expectedPartitions = expectedPartValues.map(_.mkString(",")).sorted assert(actualPartitions === expectedPartitions, "Partitions selected do not match") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 55ecbd5b5f21d..1ff1d9a2934cc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import java.sql.{Date, Timestamp} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.DefaultParserDialect @@ -164,7 +164,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils { test("show functions") { val allFunctions = (FunctionRegistry.builtin.listFunction().toSet[String] ++ - org.apache.hadoop.hive.ql.exec.FunctionRegistry.getFunctionNames).toList.sorted + org.apache.hadoop.hive.ql.exec.FunctionRegistry.getFunctionNames.asScala).toList.sorted checkAnswer(sql("SHOW functions"), allFunctions.map(Row(_))) checkAnswer(sql("SHOW functions abs"), Row("abs")) checkAnswer(sql("SHOW functions 'abs'"), Row("abs")) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 5bbca14bad320..7966b43596e75 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -17,9 +17,7 @@ package org.apache.spark.sql.sources -import java.sql.Date - -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -552,7 +550,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) } } @@ -600,7 +598,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils { } finally { // Hadoop 1 doesn't have `Configuration.unset` configuration.clear() - clonedConf.foreach(entry => configuration.set(entry.getKey, entry.getValue)) + clonedConf.asScala.foreach(entry => configuration.set(entry.getKey, entry.getValue)) sqlContext.sparkContext.conf.set("spark.speculation", speculationEnabled.toString) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 214cd80108b9b..edfa474677f15 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -17,11 +17,10 @@ package org.apache.spark.streaming.api.java -import java.util import java.lang.{Long => JLong} import java.util.{List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -145,8 +144,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * an array. */ def glom(): JavaDStream[JList[T]] = - new JavaDStream(dstream.glom().map(x => new java.util.ArrayList[T](x.toSeq))) - + new JavaDStream(dstream.glom().map(_.toSeq.asJava)) /** Return the [[org.apache.spark.streaming.StreamingContext]] associated with this DStream */ @@ -191,7 +189,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = { def fn: (Iterator[T]) => Iterator[U] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } new JavaDStream(dstream.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } @@ -204,7 +202,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) : JavaPairDStream[K2, V2] = { def fn: (Iterator[T]) => Iterator[(K2, V2)] = { - (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + (x: Iterator[T]) => f.call(x.asJava).iterator().asScala } new JavaPairDStream(dstream.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } @@ -282,7 +280,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return all the RDDs between 'fromDuration' to 'toDuration' (both included) */ def slice(fromTime: Time, toTime: Time): JList[R] = { - new util.ArrayList(dstream.slice(fromTime, toTime).map(wrapRDD(_)).toSeq) + dstream.slice(fromTime, toTime).map(wrapRDD).asJava } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 26383e420101e..e2aec6c2f63e7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.api.java import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -116,14 +116,14 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * generate the RDDs with Spark's default number of partitions. */ def groupByKey(): JavaPairDStream[K, JIterable[V]] = - dstream.groupByKey().mapValues(asJavaIterable _) + dstream.groupByKey().mapValues(_.asJava) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterable[V]] = - dstream.groupByKey(numPartitions).mapValues(asJavaIterable _) + dstream.groupByKey(numPartitions).mapValues(_.asJava) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -132,7 +132,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterable[V]] = - dstream.groupByKey(partitioner).mapValues(asJavaIterable _) + dstream.groupByKey(partitioner).mapValues(_.asJava) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -197,7 +197,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * batching interval */ def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterable[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterable _) + dstream.groupByKeyAndWindow(windowDuration).mapValues(_.asJava) } /** @@ -212,7 +212,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) : JavaPairDStream[K, JIterable[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterable _) + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(_.asJava) } /** @@ -228,8 +228,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) : JavaPairDStream[K, JIterable[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(asJavaIterable _) + dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions).mapValues(_.asJava) } /** @@ -248,8 +247,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( slideDuration: Duration, partitioner: Partitioner ): JavaPairDStream[K, JIterable[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(asJavaIterable _) + dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner).mapValues(_.asJava) } /** @@ -431,7 +429,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]): (Seq[V], Option[S]) => Option[S] = { val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => { - val list: JList[V] = values + val list: JList[V] = values.asJava val scalaState: Optional[S] = JavaUtils.optionToOptional(state) val result: Optional[S] = in.apply(list, scalaState) result.isPresent match { @@ -539,7 +537,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (t._1.asJava, t._2.asJava)) } /** @@ -551,8 +549,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( numPartitions: Int ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) + dstream.cogroup(other.dstream, numPartitions).mapValues(t => (t._1.asJava, t._2.asJava)) } /** @@ -564,8 +561,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( partitioner: Partitioner ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) + dstream.cogroup(other.dstream, partitioner).mapValues(t => (t._1.asJava, t._2.asJava)) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 35cc3ce5cf468..13f371f29603a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -21,7 +21,7 @@ import java.lang.{Boolean => JBoolean} import java.io.{Closeable, InputStream} import java.util.{List => JList, Map => JMap} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} @@ -115,7 +115,13 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, environment)) + this(new StreamingContext( + master, + appName, + batchDuration, + sparkHome, + jars, + environment.asScala)) /** * Create a JavaStreamingContext using an existing JavaSparkContext. @@ -197,7 +203,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) : JavaReceiverInputDStream[T] = { - def fn: (InputStream) => Iterator[T] = (x: InputStream) => converter.call(x).toIterator + def fn: (InputStream) => Iterator[T] = (x: InputStream) => converter.call(x).iterator().asScala implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.socketStream(hostname, port, fn, storageLevel) @@ -432,7 +438,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.map(_.rdd).toSeq: _*) + sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue) } @@ -456,7 +462,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.map(_.rdd).toSeq: _*) + sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime) } @@ -481,7 +487,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] - sQueue.enqueue(queue.map(_.rdd).toSeq: _*) + sQueue.enqueue(queue.asScala.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } @@ -500,7 +506,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ def union[T](first: JavaDStream[T], rest: JList[JavaDStream[T]]): JavaDStream[T] = { - val dstreams: Seq[DStream[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream) + val dstreams: Seq[DStream[T]] = (Seq(first) ++ rest.asScala).map(_.dstream) implicit val cm: ClassTag[T] = first.classTag ssc.union(dstreams)(cm) } @@ -512,7 +518,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { first: JavaPairDStream[K, V], rest: JList[JavaPairDStream[K, V]] ): JavaPairDStream[K, V] = { - val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream) + val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ rest.asScala).map(_.dstream) implicit val cm: ClassTag[(K, V)] = first.classTag implicit val kcm: ClassTag[K] = first.kManifest implicit val vcm: ClassTag[V] = first.vManifest @@ -534,12 +540,11 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ): JavaDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val scalaDStreams = dstreams.map(_.dstream).toSeq val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { - val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList + val jrdds = rdds.map(JavaRDD.fromRDD(_)).asJava transformFunc.call(jrdds, time).rdd } - ssc.transform(scalaDStreams, scalaTransformFunc) + ssc.transform(dstreams.asScala.map(_.dstream).toSeq, scalaTransformFunc) } /** @@ -559,12 +564,11 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val cmv: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] - val scalaDStreams = dstreams.map(_.dstream).toSeq val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { - val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList + val jrdds = rdds.map(JavaRDD.fromRDD(_)).asJava transformFunc.call(jrdds, time).rdd } - ssc.transform(scalaDStreams, scalaTransformFunc) + ssc.transform(dstreams.asScala.map(_.dstream).toSeq, scalaTransformFunc) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index d06401245ff17..2c373640d2fd9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -20,14 +20,13 @@ package org.apache.spark.streaming.api.python import java.io.{ObjectInputStream, ObjectOutputStream} import java.lang.reflect.Proxy import java.util.{ArrayList => JArrayList, List => JList} -import scala.collection.JavaConversions._ + import scala.collection.JavaConverters._ import scala.language.existentials import py4j.GatewayServer import org.apache.spark.api.java._ -import org.apache.spark.api.python._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Interval, Duration, Time} @@ -161,7 +160,7 @@ private[python] object PythonDStream { */ def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = { val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]] - rdds.forall(queue.add(_)) + rdds.asScala.foreach(queue.add) queue } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 554aae0117b24..2252e28f22af8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import org.apache.spark.storage.StorageLevel import org.apache.spark.annotation.DeveloperApi @@ -144,12 +144,12 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable * for being used in the corresponding InputDStream. */ def store(dataIterator: java.util.Iterator[T], metadata: Any) { - supervisor.pushIterator(dataIterator, Some(metadata), None) + supervisor.pushIterator(dataIterator.asScala, Some(metadata), None) } /** Store an iterator of received data as a data block into Spark's memory. */ def store(dataIterator: java.util.Iterator[T]) { - supervisor.pushIterator(dataIterator, None, None) + supervisor.pushIterator(dataIterator.asScala, None, None) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 6d4cdc4aa6b10..0cd39594ee923 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.scheduler import java.util.concurrent.{ConcurrentHashMap, TimeUnit} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.util.{Failure, Success} import org.apache.spark.Logging @@ -128,7 +128,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } def getPendingTimes(): Seq[Time] = { - jobSets.keySet.toSeq + jobSets.asScala.keys.toSeq } def reportError(msg: String, e: Throwable) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index 53b96d51c9180..f2711d1355e60 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.scheduler import java.nio.ByteBuffer +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.implicitConversions @@ -196,8 +197,7 @@ private[streaming] class ReceivedBlockTracker( writeAheadLogOption.foreach { writeAheadLog => logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") - import scala.collection.JavaConversions._ - writeAheadLog.readAll().foreach { byteBuffer => + writeAheadLog.readAll().asScala.foreach { byteBuffer => logTrace("Recovering record " + byteBuffer) Utils.deserialize[ReceivedBlockTrackerLogEvent]( byteBuffer.array, Thread.currentThread().getContextClassLoader) match { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index fe6328b1ce727..9f4a4d6806ab5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import java.util.{Iterator => JIterator} +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.{Await, ExecutionContext, Future} import scala.language.postfixOps @@ -118,7 +119,6 @@ private[streaming] class FileBasedWriteAheadLog( * hence the implementation is kept simple. */ def readAll(): JIterator[ByteBuffer] = synchronized { - import scala.collection.JavaConversions._ val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) @@ -126,7 +126,7 @@ private[streaming] class FileBasedWriteAheadLog( logDebug(s"Creating log reader with $file") val reader = new FileBasedWriteAheadLogReader(file, hadoopConf) CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, reader.close _) - } flatMap { x => x } + }.flatten.asJava } /** diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index bb80bff6dc2e6..57b50bdfd6520 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -17,16 +17,13 @@ package org.apache.spark.streaming -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import java.util.{List => JList} + +import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import java.util.{List => JList} -import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming._ -import java.util.ArrayList -import collection.JavaConversions._ import org.apache.spark.api.java.JavaRDDLike -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaDStreamLike, JavaDStream, JavaStreamingContext} /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { @@ -39,7 +36,7 @@ trait JavaTestBase extends TestSuiteBase { ssc: JavaStreamingContext, data: JList[JList[T]], numPartitions: Int) = { - val seqData = data.map(Seq(_:_*)) + val seqData = data.asScala.map(_.asScala) implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] @@ -72,9 +69,7 @@ trait JavaTestBase extends TestSuiteBase { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] ssc.getState() val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput) - val out = new ArrayList[JList[V]]() - res.map(entry => out.append(new ArrayList[V](entry))) - out + res.map(_.asJava).asJava } /** @@ -90,12 +85,7 @@ trait JavaTestBase extends TestSuiteBase { implicit val cm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] val res = runStreamsWithPartitions[V](ssc.ssc, numBatches, numExpectedOutput) - val out = new ArrayList[JList[JList[V]]]() - res.map{entry => - val lists = entry.map(new ArrayList[V](_)) - out.append(new ArrayList[JList[V]](lists)) - } - out + res.map(entry => entry.map(_.asJava).asJava).asJava } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 325ff7c74c39d..5e49fd00769ad 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -20,6 +20,7 @@ import java.io._ import java.nio.ByteBuffer import java.util +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.{implicitConversions, postfixOps} @@ -417,9 +418,8 @@ object WriteAheadLogSuite { /** Read all the data in the log file in a directory using the WriteAheadLog class. */ def readDataUsingWriteAheadLog(logDirectory: String): Seq[String] = { - import scala.collection.JavaConversions._ val wal = new FileBasedWriteAheadLog(new SparkConf(), logDirectory, hadoopConf, 1, 1) - val data = wal.readAll().map(byteBufferToString).toSeq + val data = wal.readAll().asScala.map(byteBufferToString).toSeq wal.close() data } diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 9418beb6b3e3a..a0524cabff2d4 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -22,7 +22,7 @@ import java.io.File import java.util.jar.JarFile import scala.collection.mutable -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.runtimeMirror import scala.reflect.runtime.{universe => unv} import scala.util.Try @@ -161,7 +161,7 @@ object GenerateMIMAIgnore { val path = packageName.replace('.', '/') val resources = classLoader.getResources(path) - val jars = resources.filter(x => x.getProtocol == "jar") + val jars = resources.asScala.filter(_.getProtocol == "jar") .map(_.getFile.split(":")(1).split("!")(0)).toSeq jars.flatMap(getClassesFromJar(_, path)) @@ -175,7 +175,7 @@ object GenerateMIMAIgnore { private def getClassesFromJar(jarPath: String, packageName: String) = { import scala.collection.mutable val jar = new JarFile(new File(jarPath)) - val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) + val enums = jar.entries().asScala.map(_.getName).filter(_.startsWith(packageName)) val classes = mutable.HashSet[Class[_]]() for (entry <- enums if entry.endsWith(".class")) { try { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index bff585b46cbbe..e9a02baafd28e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -25,7 +25,7 @@ import java.security.PrivilegedExceptionAction import java.util.{Properties, UUID} import java.util.zip.{ZipEntry, ZipOutputStream} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} @@ -511,7 +511,7 @@ private[spark] class Client( val nns = YarnSparkHadoopUtil.get.getNameNodesToAccess(sparkConf) + stagingDirPath YarnSparkHadoopUtil.get.obtainTokensForNamenodes( nns, hadoopConf, creds, Some(sparkConf.get("spark.yarn.principal"))) - val t = creds.getAllTokens + val t = creds.getAllTokens.asScala .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) .head val newExpiration = t.renew(hadoopConf) @@ -650,8 +650,8 @@ private[spark] class Client( distCacheMgr.setDistArchivesEnv(launchEnv) val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources) - amContainer.setEnvironment(launchEnv) + amContainer.setLocalResources(localResources.asJava) + amContainer.setEnvironment(launchEnv.asJava) val javaOpts = ListBuffer[String]() @@ -782,7 +782,7 @@ private[spark] class Client( // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList - amContainer.setCommands(printableCommands) + amContainer.setCommands(printableCommands.asJava) logDebug("===============================================================================") logDebug("YARN AM launch context:") @@ -797,7 +797,8 @@ private[spark] class Client( // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) - amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + amContainer.setApplicationACLs( + YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager).asJava) setupSecurityToken(amContainer) UserGroupInformation.getCurrentUser().addCredentials(credentials) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 4cc50483a17ff..9abd09b3cc7a5 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -20,14 +20,13 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI import java.nio.ByteBuffer +import java.util.Collections -import org.apache.hadoop.fs.Path -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.spark.util.Utils - -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} +import org.apache.hadoop.fs.Path +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation @@ -40,6 +39,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.network.util.JavaUtils +import org.apache.spark.util.Utils class ExecutorRunnable( container: Container, @@ -74,9 +74,9 @@ class ExecutorRunnable( .asInstanceOf[ContainerLaunchContext] val localResources = prepareLocalResources - ctx.setLocalResources(localResources) + ctx.setLocalResources(localResources.asJava) - ctx.setEnvironment(env) + ctx.setEnvironment(env.asJava) val credentials = UserGroupInformation.getCurrentUser().getCredentials() val dob = new DataOutputBuffer() @@ -96,8 +96,9 @@ class ExecutorRunnable( |=============================================================================== """.stripMargin) - ctx.setCommands(commands) - ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + ctx.setCommands(commands.asJava) + ctx.setApplicationACLs( + YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr).asJava) // If external shuffle service is enabled, register with the Yarn shuffle service already // started on the NodeManager and, if authentication is enabled, provide it with our secret @@ -112,7 +113,7 @@ class ExecutorRunnable( // Authentication is not enabled, so just provide dummy metadata ByteBuffer.allocate(0) } - ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) + ctx.setServiceData(Collections.singletonMap("spark_shuffle", secretBytes)) } // Send the start request to the ContainerManager @@ -314,7 +315,8 @@ class ExecutorRunnable( env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" } - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } + System.getenv().asScala.filterKeys(_.startsWith("SPARK")) + .foreach { case (k, v) => env(k) = v } env } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index ccf753e69f4b6..5f897cbcb4e9f 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -21,9 +21,7 @@ import java.util.Collections import java.util.concurrent._ import java.util.regex.Pattern -import org.apache.spark.util.Utils - -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -39,8 +37,8 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.util.Utils /** * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding @@ -164,7 +162,7 @@ private[yarn] class YarnAllocator( * Number of container requests at the given location that have not yet been fulfilled. */ private def getNumPendingAtLocation(location: String): Int = - amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).map(_.size).sum + amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala.map(_.size).sum /** * Request as many executors from the ResourceManager as needed to reach the desired total. If @@ -231,14 +229,14 @@ private[yarn] class YarnAllocator( numExecutorsRunning, allocateResponse.getAvailableResources)) - handleAllocatedContainers(allocatedContainers) + handleAllocatedContainers(allocatedContainers.asScala) } val completedContainers = allocateResponse.getCompletedContainersStatuses() if (completedContainers.size > 0) { logDebug("Completed %d containers".format(completedContainers.size)) - processCompletedContainers(completedContainers) + processCompletedContainers(completedContainers.asScala) logDebug("Finished processing %d completed containers. Current running executor count: %d." .format(completedContainers.size, numExecutorsRunning)) @@ -271,7 +269,7 @@ private[yarn] class YarnAllocator( val request = createContainerRequest(resource, locality.nodes, locality.racks) amClient.addContainerRequest(request) val nodes = request.getNodes - val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.last + val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.asScala.last logInfo(s"Container request (host: $hostStr, capability: $resource)") } } else if (missing < 0) { @@ -280,7 +278,8 @@ private[yarn] class YarnAllocator( val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) if (!matchingRequests.isEmpty) { - matchingRequests.head.take(numToCancel).foreach(amClient.removeContainerRequest) + matchingRequests.iterator().next().asScala + .take(numToCancel).foreach(amClient.removeContainerRequest) } else { logWarning("Expected to find pending requests, but found none.") } @@ -459,7 +458,7 @@ private[yarn] class YarnAllocator( } } - if (allocatedContainerToHostMap.containsKey(containerId)) { + if (allocatedContainerToHostMap.contains(containerId)) { val host = allocatedContainerToHostMap.get(containerId).get val containerSet = allocatedHostToContainersMap.get(host).get diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 4999f9c06210a..df042bf291de7 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -19,17 +19,15 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.{Map, Set} import scala.util.Try import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -108,8 +106,8 @@ private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logg val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", classOf[Configuration]) val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] - val hosts = proxies.map { proxy => proxy.split(":")(0) } - val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } + val hosts = proxies.asScala.map { proxy => proxy.split(":")(0) } + val uriBases = proxies.asScala.map { proxy => prefix + proxy + proxyBase } Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) } catch { case e: NoSuchMethodException => diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 128e996b71fe5..b4f8049bff577 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -21,7 +21,7 @@ import java.io.{File, FileOutputStream, OutputStreamWriter} import java.util.Properties import java.util.concurrent.TimeUnit -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -132,7 +132,7 @@ abstract class BaseYarnClusterSuite props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") props.setProperty("spark.executor.extraJavaOptions", "-Dfoo=\"one two three\"") - yarnCluster.getConfig().foreach { e => + yarnCluster.getConfig.asScala.foreach { e => props.setProperty("spark.hadoop." + e.getKey(), e.getValue()) } @@ -149,7 +149,7 @@ abstract class BaseYarnClusterSuite props.store(writer, "Spark properties.") writer.close() - val extraJarArgs = if (!extraJars.isEmpty()) Seq("--jars", extraJars.mkString(",")) else Nil + val extraJarArgs = if (extraJars.nonEmpty) Seq("--jars", extraJars.mkString(",")) else Nil val mainArgs = if (klass.endsWith(".py")) { Seq(klass) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 0a5402c89e764..e7f2501e7899f 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ HashMap => MutableHashMap } +import scala.collection.JavaConverters._ +import scala.collection.mutable.{HashMap => MutableHashMap} import scala.reflect.ClassTag import scala.util.Try @@ -38,7 +38,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterAll, Matchers} -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.util.Utils class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll { @@ -201,7 +201,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll { appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method => val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]] tags should contain allOf ("tag1", "dup", "tag2", "multi word") - tags.filter(!_.isEmpty).size should be (4) + tags.asScala.filter(_.nonEmpty).size should be (4) } appContext.getMaxAppAttempts should be (42) } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 128350b648992..5a4ea2ea2f4ff 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -21,7 +21,6 @@ import java.io.File import java.net.URL import scala.collection.mutable -import scala.collection.JavaConversions._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.{ByteStreams, Files} @@ -216,8 +215,8 @@ private object YarnClusterDriver extends Logging with Matchers { assert(listener.driverLogs.nonEmpty) val driverLogs = listener.driverLogs.get assert(driverLogs.size === 2) - assert(driverLogs.containsKey("stderr")) - assert(driverLogs.containsKey("stdout")) + assert(driverLogs.contains("stderr")) + assert(driverLogs.contains("stdout")) val urlStr = driverLogs("stderr") // Ensure that this is a valid URL, else this will throw an exception new URL(urlStr) From 5c08c86bfa43462fb2ca5f7c5980ddfb44dd57f8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 25 Aug 2015 10:22:54 -0700 Subject: [PATCH 081/232] [SPARK-10198] [SQL] Turn off partition verification by default Author: Michael Armbrust Closes #8404 from marmbrus/turnOffPartitionVerification. --- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../spark/sql/hive/QueryPartitionSuite.scala | 64 ++++++++++--------- 2 files changed, 35 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index e6f7619519e6a..9de75f4c4d084 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -312,7 +312,7 @@ private[spark] object SQLConf { doc = "When true, enable filter pushdown for ORC files.") val HIVE_VERIFY_PARTITION_PATH = booleanConf("spark.sql.hive.verifyPartitionPath", - defaultValue = Some(true), + defaultValue = Some(false), doc = "") val HIVE_METASTORE_PARTITION_PRUNING = booleanConf("spark.sql.hive.metastorePartitionPruning", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index 017bc2adc103b..1cc8a93e83411 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -18,50 +18,54 @@ package org.apache.spark.sql.hive import com.google.common.io.Files +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.util.Utils -class QueryPartitionSuite extends QueryTest { +class QueryPartitionSuite extends QueryTest with SQLTestUtils { private lazy val ctx = org.apache.spark.sql.hive.test.TestHive import ctx.implicits._ - import ctx.sql + + protected def _sqlContext = ctx test("SPARK-5068: query data when path doesn't exist"){ - val testData = ctx.sparkContext.parallelize( - (1 to 10).map(i => TestData(i, i.toString))).toDF() - testData.registerTempTable("testData") + withSQLConf((SQLConf.HIVE_VERIFY_PARTITION_PATH.key, "true")) { + val testData = ctx.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") - val tmpDir = Files.createTempDir() - // create the table for test - sql(s"CREATE TABLE table_with_partition(key int,value string) " + - s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + - "SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + - "SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') " + - "SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') " + - "SELECT key,value FROM testData") + val tmpDir = Files.createTempDir() + // create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) " + + s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') " + + "SELECT key,value FROM testData") - // test for the exist path - checkAnswer(sql("select key,value from table_with_partition"), - testData.toDF.collect ++ testData.toDF.collect - ++ testData.toDF.collect ++ testData.toDF.collect) + // test for the exist path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toDF.collect ++ testData.toDF.collect + ++ testData.toDF.collect ++ testData.toDF.collect) - // delete the path of one partition - tmpDir.listFiles - .find { f => f.isDirectory && f.getName().startsWith("ds=") } - .foreach { f => Utils.deleteRecursively(f) } + // delete the path of one partition + tmpDir.listFiles + .find { f => f.isDirectory && f.getName().startsWith("ds=") } + .foreach { f => Utils.deleteRecursively(f) } - // test for after delete the path - checkAnswer(sql("select key,value from table_with_partition"), - testData.toDF.collect ++ testData.toDF.collect ++ testData.toDF.collect) + // test for after delete the path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toDF.collect ++ testData.toDF.collect ++ testData.toDF.collect) - sql("DROP TABLE table_with_partition") - sql("DROP TABLE createAndInsertTest") + sql("DROP TABLE table_with_partition") + sql("DROP TABLE createAndInsertTest") + } } } From b37f0cc1b4c064d6f09edb161250fa8b783de52a Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Tue, 25 Aug 2015 10:54:03 -0700 Subject: [PATCH 082/232] [SPARK-8531] [ML] Update ML user guide for MinMaxScaler jira: https://issues.apache.org/jira/browse/SPARK-8531 Update ML user guide for MinMaxScaler Author: Yuhao Yang Author: unknown Closes #7211 from hhbyyh/minmaxdoc. --- docs/ml-features.md | 71 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/docs/ml-features.md b/docs/ml-features.md index 642a4b4c53183..62de4838981cb 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1133,6 +1133,7 @@ val scaledData = scalerModel.transform(dataFrame) {% highlight java %} import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.StandardScaler; +import org.apache.spark.ml.feature.StandardScalerModel; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.sql.DataFrame; @@ -1173,6 +1174,76 @@ scaledData = scalerModel.transform(dataFrame)
+## MinMaxScaler + +`MinMaxScaler` transforms a dataset of `Vector` rows, rescaling each feature to a specific range (often [0, 1]). It takes parameters: + +* `min`: 0.0 by default. Lower bound after transformation, shared by all features. +* `max`: 1.0 by default. Upper bound after transformation, shared by all features. + +`MinMaxScaler` computes summary statistics on a data set and produces a `MinMaxScalerModel`. The model can then transform each feature individually such that it is in the given range. + +The rescaled value for a feature E is calculated as, +`\begin{equation} + Rescaled(e_i) = \frac{e_i - E_{min}}{E_{max} - E_{min}} * (max - min) + min +\end{equation}` +For the case `E_{max} == E_{min}`, `Rescaled(e_i) = 0.5 * (max + min)` + +Note that since zero values will probably be transformed to non-zero values, output of the transformer will be DenseVector even for sparse input. + +The following example demonstrates how to load a dataset in libsvm format and then rescale each feature to [0, 1]. + +
+
+More details can be found in the API docs for +[MinMaxScaler](api/scala/index.html#org.apache.spark.ml.feature.MinMaxScaler) and +[MinMaxScalerModel](api/scala/index.html#org.apache.spark.ml.feature.MinMaxScalerModel). +{% highlight scala %} +import org.apache.spark.ml.feature.MinMaxScaler +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +val dataFrame = sqlContext.createDataFrame(data) +val scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures") + +// Compute summary statistics and generate MinMaxScalerModel +val scalerModel = scaler.fit(dataFrame) + +// rescale each feature to range [min, max]. +val scaledData = scalerModel.transform(dataFrame) +{% endhighlight %} +
+ +
+More details can be found in the API docs for +[MinMaxScaler](api/java/org/apache/spark/ml/feature/MinMaxScaler.html) and +[MinMaxScalerModel](api/java/org/apache/spark/ml/feature/MinMaxScalerModel.html). +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.MinMaxScaler; +import org.apache.spark.ml.feature.MinMaxScalerModel; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.sql.DataFrame; + +JavaRDD data = + MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt").toJavaRDD(); +DataFrame dataFrame = jsql.createDataFrame(data, LabeledPoint.class); +MinMaxScaler scaler = new MinMaxScaler() + .setInputCol("features") + .setOutputCol("scaledFeatures"); + +// Compute summary statistics and generate MinMaxScalerModel +MinMaxScalerModel scalerModel = scaler.fit(dataFrame); + +// rescale each feature to range [min, max]. +DataFrame scaledData = scalerModel.transform(dataFrame); +{% endhighlight %} +
+
+ ## Bucketizer `Bucketizer` transforms a column of continuous features to a column of feature buckets, where the buckets are specified by users. It takes a parameter: From 881208a8e849facf54166bdd69d3634407f952e7 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 25 Aug 2015 11:58:47 -0700 Subject: [PATCH 083/232] [SPARK-10230] [MLLIB] Rename optimizeAlpha to optimizeDocConcentration See [discussion](https://github.com/apache/spark/pull/8254#discussion_r37837770) CC jkbradley Author: Feynman Liang Closes #8422 from feynmanliang/SPARK-10230. --- .../spark/mllib/clustering/LDAOptimizer.scala | 16 ++++++++-------- .../apache/spark/mllib/clustering/LDASuite.scala | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala index 5c2aae6403bea..38486e949bbcf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala @@ -258,7 +258,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { private var tau0: Double = 1024 private var kappa: Double = 0.51 private var miniBatchFraction: Double = 0.05 - private var optimizeAlpha: Boolean = false + private var optimizeDocConcentration: Boolean = false // internal data structure private var docs: RDD[(Long, Vector)] = null @@ -335,20 +335,20 @@ final class OnlineLDAOptimizer extends LDAOptimizer { } /** - * Optimize alpha, indicates whether alpha (Dirichlet parameter for document-topic distribution) - * will be optimized during training. + * Optimize docConcentration, indicates whether docConcentration (Dirichlet parameter for + * document-topic distribution) will be optimized during training. */ @Since("1.5.0") - def getOptimzeAlpha: Boolean = this.optimizeAlpha + def getOptimizeDocConcentration: Boolean = this.optimizeDocConcentration /** - * Sets whether to optimize alpha parameter during training. + * Sets whether to optimize docConcentration parameter during training. * * Default: false */ @Since("1.5.0") - def setOptimzeAlpha(optimizeAlpha: Boolean): this.type = { - this.optimizeAlpha = optimizeAlpha + def setOptimizeDocConcentration(optimizeDocConcentration: Boolean): this.type = { + this.optimizeDocConcentration = optimizeDocConcentration this } @@ -458,7 +458,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer { // Note that this is an optimization to avoid batch.count updateLambda(batchResult, (miniBatchFraction * corpusSize).ceil.toInt) - if (optimizeAlpha) updateAlpha(gammat) + if (optimizeDocConcentration) updateAlpha(gammat) this } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 8a714f9b79e02..746a76a7e5fa1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -423,7 +423,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { val k = 2 val docs = sc.parallelize(toyData) val op = new OnlineLDAOptimizer().setMiniBatchFraction(1).setTau0(1024).setKappa(0.51) - .setGammaShape(100).setOptimzeAlpha(true).setSampleWithReplacement(false) + .setGammaShape(100).setOptimizeDocConcentration(true).setSampleWithReplacement(false) val lda = new LDA().setK(k) .setDocConcentration(1D / k) .setTopicConcentration(0.01) From 16a2be1a84c0a274a60c0a584faaf58b55d4942b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 12:16:23 -0700 Subject: [PATCH 084/232] [SPARK-10231] [MLLIB] update @Since annotation for mllib.classification Update `Since` annotation in `mllib.classification`: 1. add version to classes, objects, constructors, and public variables declared in constructors 2. correct some versions 3. remove `Since` on `toString` MechCoder dbtsai Author: Xiangrui Meng Closes #8421 from mengxr/SPARK-10231 and squashes the following commits: b2dce80 [Xiangrui Meng] update @Since annotation for mllib.classification --- .../classification/ClassificationModel.scala | 7 +++-- .../classification/LogisticRegression.scala | 20 +++++++++---- .../mllib/classification/NaiveBayes.scala | 28 +++++++++++++++---- .../spark/mllib/classification/SVM.scala | 15 ++++++---- .../StreamingLogisticRegressionWithSGD.scala | 9 +++++- 5 files changed, 58 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index a29b425a71fd6..85a413243b049 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD * belongs. The categories are represented by double values: 0.0, 1.0, 2.0, etc. */ @Experimental +@Since("0.8.0") trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. @@ -37,7 +38,7 @@ trait ClassificationModel extends Serializable { * @param testData RDD representing data points to be predicted * @return an RDD[Double] where each entry contains the corresponding prediction */ - @Since("0.8.0") + @Since("1.0.0") def predict(testData: RDD[Vector]): RDD[Double] /** @@ -46,7 +47,7 @@ trait ClassificationModel extends Serializable { * @param testData array representing a single data point * @return predicted category from the trained model */ - @Since("0.8.0") + @Since("1.0.0") def predict(testData: Vector): Double /** @@ -54,7 +55,7 @@ trait ClassificationModel extends Serializable { * @param testData JavaRDD representing data points to be predicted * @return a JavaRDD[java.lang.Double] where each entry contains the corresponding prediction */ - @Since("0.8.0") + @Since("1.0.0") def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } 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 e03e662227d14..5ceff5b2259ea 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 @@ -41,11 +41,12 @@ import org.apache.spark.rdd.RDD * Multinomial Logistic Regression. By default, it is binary logistic regression * so numClasses will be set to 2. */ -class LogisticRegressionModel ( - override val weights: Vector, - override val intercept: Double, - val numFeatures: Int, - val numClasses: Int) +@Since("0.8.0") +class LogisticRegressionModel @Since("1.3.0") ( + @Since("1.0.0") override val weights: Vector, + @Since("1.0.0") override val intercept: Double, + @Since("1.3.0") val numFeatures: Int, + @Since("1.3.0") val numClasses: Int) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable with Saveable with PMMLExportable { @@ -75,6 +76,7 @@ class LogisticRegressionModel ( /** * Constructs a [[LogisticRegressionModel]] with weights and intercept for binary classification. */ + @Since("1.0.0") def this(weights: Vector, intercept: Double) = this(weights, intercept, weights.size, 2) private var threshold: Option[Double] = Some(0.5) @@ -166,12 +168,12 @@ class LogisticRegressionModel ( override protected def formatVersion: String = "1.0" - @Since("1.4.0") override def toString: String = { s"${super.toString}, numClasses = ${numClasses}, threshold = ${threshold.getOrElse("None")}" } } +@Since("1.3.0") object LogisticRegressionModel extends Loader[LogisticRegressionModel] { @Since("1.3.0") @@ -207,6 +209,7 @@ object LogisticRegressionModel extends Loader[LogisticRegressionModel] { * for k classes multi-label classification problem. * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ +@Since("0.8.0") class LogisticRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -216,6 +219,7 @@ class LogisticRegressionWithSGD private[mllib] ( private val gradient = new LogisticGradient() private val updater = new SquaredL2Updater() + @Since("0.8.0") override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) @@ -227,6 +231,7 @@ class LogisticRegressionWithSGD private[mllib] ( * Construct a LogisticRegression object with default parameters: {stepSize: 1.0, * numIterations: 100, regParm: 0.01, miniBatchFraction: 1.0}. */ + @Since("0.8.0") def this() = this(1.0, 100, 0.01, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { @@ -238,6 +243,7 @@ class LogisticRegressionWithSGD private[mllib] ( * 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") object LogisticRegressionWithSGD { // NOTE(shivaram): We use multiple train methods instead of default arguments to support // Java programs. @@ -333,11 +339,13 @@ object LogisticRegressionWithSGD { * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} * for k classes multi-label classification problem. */ +@Since("1.1.0") class LogisticRegressionWithLBFGS extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { this.setFeatureScaling(true) + @Since("1.1.0") override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) override protected val validators = List(multiLabelValidator) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index dab369207cc9a..a956084ae06e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -41,11 +41,12 @@ import org.apache.spark.sql.{DataFrame, SQLContext} * where D is number of features * @param modelType The type of NB model to fit can be "multinomial" or "bernoulli" */ +@Since("0.9.0") class NaiveBayesModel private[spark] ( - val labels: Array[Double], - val pi: Array[Double], - val theta: Array[Array[Double]], - val modelType: String) + @Since("1.0.0") val labels: Array[Double], + @Since("0.9.0") val pi: Array[Double], + @Since("0.9.0") val theta: Array[Array[Double]], + @Since("1.4.0") val modelType: String) extends ClassificationModel with Serializable with Saveable { import NaiveBayes.{Bernoulli, Multinomial, supportedModelTypes} @@ -83,6 +84,7 @@ class NaiveBayesModel private[spark] ( throw new UnknownError(s"Invalid modelType: $modelType.") } + @Since("1.0.0") override def predict(testData: RDD[Vector]): RDD[Double] = { val bcModel = testData.context.broadcast(this) testData.mapPartitions { iter => @@ -91,6 +93,7 @@ class NaiveBayesModel private[spark] ( } } + @Since("1.0.0") override def predict(testData: Vector): Double = { modelType match { case Multinomial => @@ -107,6 +110,7 @@ class NaiveBayesModel private[spark] ( * @return an RDD[Vector] where each entry contains the predicted posterior class probabilities, * in the same order as class labels */ + @Since("1.5.0") def predictProbabilities(testData: RDD[Vector]): RDD[Vector] = { val bcModel = testData.context.broadcast(this) testData.mapPartitions { iter => @@ -122,6 +126,7 @@ class NaiveBayesModel private[spark] ( * @return predicted posterior class probabilities from the trained model, * in the same order as class labels */ + @Since("1.5.0") def predictProbabilities(testData: Vector): Vector = { modelType match { case Multinomial => @@ -158,6 +163,7 @@ class NaiveBayesModel private[spark] ( new DenseVector(scaledProbs.map(_ / probSum)) } + @Since("1.3.0") override def save(sc: SparkContext, path: String): Unit = { val data = NaiveBayesModel.SaveLoadV2_0.Data(labels, pi, theta, modelType) NaiveBayesModel.SaveLoadV2_0.save(sc, path, data) @@ -166,6 +172,7 @@ class NaiveBayesModel private[spark] ( override protected def formatVersion: String = "2.0" } +@Since("1.3.0") object NaiveBayesModel extends Loader[NaiveBayesModel] { import org.apache.spark.mllib.util.Loader._ @@ -199,6 +206,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { dataRDD.write.parquet(dataPath(path)) } + @Since("1.3.0") def load(sc: SparkContext, path: String): NaiveBayesModel = { val sqlContext = new SQLContext(sc) // Load Parquet data. @@ -301,30 +309,35 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. */ - +@Since("0.9.0") class NaiveBayes private ( private var lambda: Double, private var modelType: String) extends Serializable with Logging { import NaiveBayes.{Bernoulli, Multinomial} + @Since("1.4.0") def this(lambda: Double) = this(lambda, NaiveBayes.Multinomial) + @Since("0.9.0") def this() = this(1.0, NaiveBayes.Multinomial) /** Set the smoothing parameter. Default: 1.0. */ + @Since("0.9.0") def setLambda(lambda: Double): NaiveBayes = { this.lambda = lambda this } /** Get the smoothing parameter. */ + @Since("1.4.0") def getLambda: Double = lambda /** * Set the model type using a string (case-sensitive). * Supported options: "multinomial" (default) and "bernoulli". */ + @Since("1.4.0") def setModelType(modelType: String): NaiveBayes = { require(NaiveBayes.supportedModelTypes.contains(modelType), s"NaiveBayes was created with an unknown modelType: $modelType.") @@ -333,6 +346,7 @@ class NaiveBayes private ( } /** Get the model type. */ + @Since("1.4.0") def getModelType: String = this.modelType /** @@ -340,6 +354,7 @@ class NaiveBayes private ( * * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ + @Since("0.9.0") def run(data: RDD[LabeledPoint]): NaiveBayesModel = { val requireNonnegativeValues: Vector => Unit = (v: Vector) => { val values = v match { @@ -423,6 +438,7 @@ class NaiveBayes private ( /** * Top-level methods for calling naive Bayes. */ +@Since("0.9.0") object NaiveBayes { /** String name for multinomial model type. */ @@ -485,7 +501,7 @@ object NaiveBayes { * @param modelType The type of NB model to fit from the enumeration NaiveBayesModels, can be * multinomial or bernoulli */ - @Since("0.9.0") + @Since("1.4.0") def train(input: RDD[LabeledPoint], lambda: Double, modelType: String): NaiveBayesModel = { require(supportedModelTypes.contains(modelType), s"NaiveBayes was created with an unknown modelType: $modelType.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 5f87269863572..896565cd90e89 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -33,9 +33,10 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class SVMModel ( - override val weights: Vector, - override val intercept: Double) +@Since("0.8.0") +class SVMModel @Since("1.1.0") ( + @Since("1.0.0") override val weights: Vector, + @Since("0.8.0") override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable with Saveable with PMMLExportable { @@ -47,7 +48,7 @@ class SVMModel ( * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.0. */ - @Since("1.3.0") + @Since("1.0.0") @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) @@ -92,12 +93,12 @@ class SVMModel ( override protected def formatVersion: String = "1.0" - @Since("1.4.0") override def toString: String = { s"${super.toString}, numClasses = 2, threshold = ${threshold.getOrElse("None")}" } } +@Since("1.3.0") object SVMModel extends Loader[SVMModel] { @Since("1.3.0") @@ -132,6 +133,7 @@ object SVMModel extends Loader[SVMModel] { * regularization is used, which can be changed via [[SVMWithSGD.optimizer]]. * NOTE: Labels used in SVM should be {0, 1}. */ +@Since("0.8.0") class SVMWithSGD private ( private var stepSize: Double, private var numIterations: Int, @@ -141,6 +143,7 @@ class SVMWithSGD private ( private val gradient = new HingeGradient() private val updater = new SquaredL2Updater() + @Since("0.8.0") override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) @@ -152,6 +155,7 @@ class SVMWithSGD private ( * Construct a SVM object with default parameters: {stepSize: 1.0, numIterations: 100, * regParm: 0.01, miniBatchFraction: 1.0}. */ + @Since("0.8.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { @@ -162,6 +166,7 @@ class SVMWithSGD private ( /** * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1}. */ +@Since("0.8.0") object SVMWithSGD { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala index 7d33df3221fbf..75630054d1368 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.classification -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.StreamingLinearAlgorithm @@ -44,6 +44,7 @@ import org.apache.spark.mllib.regression.StreamingLinearAlgorithm * }}} */ @Experimental +@Since("1.3.0") class StreamingLogisticRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -58,6 +59,7 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( * Initial weights must be set before using trainOn or predictOn * (see `StreamingLinearAlgorithm`) */ + @Since("1.3.0") def this() = this(0.1, 50, 1.0, 0.0) protected val algorithm = new LogisticRegressionWithSGD( @@ -66,30 +68,35 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( protected var model: Option[LogisticRegressionModel] = None /** Set the step size for gradient descent. Default: 0.1. */ + @Since("1.3.0") def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) this } /** Set the number of iterations of gradient descent to run per update. Default: 50. */ + @Since("1.3.0") def setNumIterations(numIterations: Int): this.type = { this.algorithm.optimizer.setNumIterations(numIterations) this } /** Set the fraction of each batch to use for updates. Default: 1.0. */ + @Since("1.3.0") def setMiniBatchFraction(miniBatchFraction: Double): this.type = { this.algorithm.optimizer.setMiniBatchFraction(miniBatchFraction) this } /** Set the regularization parameter. Default: 0.0. */ + @Since("1.3.0") def setRegParam(regParam: Double): this.type = { this.algorithm.optimizer.setRegParam(regParam) this } /** Set the initial weights. Default: [0.0, 0.0]. */ + @Since("1.3.0") def setInitialWeights(initialWeights: Vector): this.type = { this.model = Some(algorithm.createModel(initialWeights, 0.0)) this From 71a138cd0e0a14e8426f97877e3b52a562bbd02c Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Tue, 25 Aug 2015 13:14:10 -0700 Subject: [PATCH 085/232] [SPARK-10048] [SPARKR] Support arbitrary nested Java array in serde. This PR: 1. supports transferring arbitrary nested array from JVM to R side in SerDe; 2. based on 1, collect() implemenation is improved. Now it can support collecting data of complex types from a DataFrame. Author: Sun Rui Closes #8276 from sun-rui/SPARK-10048. --- R/pkg/R/DataFrame.R | 55 +++++++++--- R/pkg/R/deserialize.R | 72 +++++++--------- R/pkg/R/serialize.R | 10 +-- R/pkg/inst/tests/test_Serde.R | 77 +++++++++++++++++ R/pkg/inst/worker/worker.R | 4 +- .../apache/spark/api/r/RBackendHandler.scala | 7 ++ .../scala/org/apache/spark/api/r/SerDe.scala | 86 +++++++++++-------- .../org/apache/spark/sql/api/r/SQLUtils.scala | 32 +------ 8 files changed, 216 insertions(+), 127 deletions(-) create mode 100644 R/pkg/inst/tests/test_Serde.R diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 10f3c4ea59864..ae1d912cf6da1 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -652,18 +652,49 @@ setMethod("dim", setMethod("collect", signature(x = "DataFrame"), function(x, stringsAsFactors = FALSE) { - # listCols is a list of raw vectors, one per column - listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) - cols <- lapply(listCols, function(col) { - objRaw <- rawConnection(col) - numRows <- readInt(objRaw) - col <- readCol(objRaw, numRows) - close(objRaw) - col - }) - names(cols) <- columns(x) - do.call(cbind.data.frame, list(cols, stringsAsFactors = stringsAsFactors)) - }) + names <- columns(x) + ncol <- length(names) + if (ncol <= 0) { + # empty data.frame with 0 columns and 0 rows + data.frame() + } else { + # listCols is a list of columns + listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) + stopifnot(length(listCols) == ncol) + + # An empty data.frame with 0 columns and number of rows as collected + nrow <- length(listCols[[1]]) + if (nrow <= 0) { + df <- data.frame() + } else { + df <- data.frame(row.names = 1 : nrow) + } + + # Append columns one by one + for (colIndex in 1 : ncol) { + # Note: appending a column of list type into a data.frame so that + # data of complex type can be held. But getting a cell from a column + # of list type returns a list instead of a vector. So for columns of + # non-complex type, append them as vector. + col <- listCols[[colIndex]] + if (length(col) <= 0) { + df[[names[colIndex]]] <- col + } else { + # TODO: more robust check on column of primitive types + vec <- do.call(c, col) + if (class(vec) != "list") { + df[[names[colIndex]]] <- vec + } else { + # For columns of complex type, be careful to access them. + # Get a column of complex type returns a list. + # Get a cell from a column of complex type returns a list instead of a vector. + df[[names[colIndex]]] <- col + } + } + } + df + } + }) #' Limit #' diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index 33bf13ec9e784..6cf628e3007de 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -48,6 +48,7 @@ readTypedObject <- function(con, type) { "r" = readRaw(con), "D" = readDate(con), "t" = readTime(con), + "a" = readArray(con), "l" = readList(con), "n" = NULL, "j" = getJobj(readString(con)), @@ -85,8 +86,7 @@ readTime <- function(con) { as.POSIXct(t, origin = "1970-01-01") } -# We only support lists where all elements are of same type -readList <- function(con) { +readArray <- function(con) { type <- readType(con) len <- readInt(con) if (len > 0) { @@ -100,6 +100,25 @@ readList <- function(con) { } } +# Read a list. Types of each element may be different. +# Null objects are read as NA. +readList <- function(con) { + len <- readInt(con) + if (len > 0) { + l <- vector("list", len) + for (i in 1:len) { + elem <- readObject(con) + if (is.null(elem)) { + elem <- NA + } + l[[i]] <- elem + } + l + } else { + list() + } +} + readRaw <- function(con) { dataLen <- readInt(con) readBin(con, raw(), as.integer(dataLen), endian = "big") @@ -132,18 +151,19 @@ readDeserialize <- function(con) { } } -readDeserializeRows <- function(inputCon) { - # readDeserializeRows will deserialize a DataOutputStream composed of - # a list of lists. Since the DOS is one continuous stream and - # the number of rows varies, we put the readRow function in a while loop - # that termintates when the next row is empty. +readMultipleObjects <- function(inputCon) { + # readMultipleObjects will read multiple continuous objects from + # a DataOutputStream. There is no preceding field telling the count + # of the objects, so the number of objects varies, we try to read + # all objects in a loop until the end of the stream. data <- list() while(TRUE) { - row <- readRow(inputCon) - if (length(row) == 0) { + # If reaching the end of the stream, type returned should be "". + type <- readType(inputCon) + if (type == "") { break } - data[[length(data) + 1L]] <- row + data[[length(data) + 1L]] <- readTypedObject(inputCon, type) } data # this is a list of named lists now } @@ -155,35 +175,5 @@ readRowList <- function(obj) { # deserialize the row. rawObj <- rawConnection(obj, "r+") on.exit(close(rawObj)) - readRow(rawObj) -} - -readRow <- function(inputCon) { - numCols <- readInt(inputCon) - if (length(numCols) > 0 && numCols > 0) { - lapply(1:numCols, function(x) { - obj <- readObject(inputCon) - if (is.null(obj)) { - NA - } else { - obj - } - }) # each row is a list now - } else { - list() - } -} - -# Take a single column as Array[Byte] and deserialize it into an atomic vector -readCol <- function(inputCon, numRows) { - if (numRows > 0) { - # sapply can not work with POSIXlt - do.call(c, lapply(1:numRows, function(x) { - value <- readObject(inputCon) - # Replace NULL with NA so we can coerce to vectors - if (is.null(value)) NA else value - })) - } else { - vector() - } + readObject(rawObj) } diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 311021e5d8473..e3676f57f907f 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -110,18 +110,10 @@ writeRowSerialize <- function(outputCon, rows) { serializeRow <- function(row) { rawObj <- rawConnection(raw(0), "wb") on.exit(close(rawObj)) - writeRow(rawObj, row) + writeGenericList(rawObj, row) rawConnectionValue(rawObj) } -writeRow <- function(con, row) { - numCols <- length(row) - writeInt(con, numCols) - for (i in 1:numCols) { - writeObject(con, row[[i]]) - } -} - writeRaw <- function(con, batch) { writeInt(con, length(batch)) writeBin(batch, con, endian = "big") diff --git a/R/pkg/inst/tests/test_Serde.R b/R/pkg/inst/tests/test_Serde.R new file mode 100644 index 0000000000000..009db85da2beb --- /dev/null +++ b/R/pkg/inst/tests/test_Serde.R @@ -0,0 +1,77 @@ +# +# 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. +# + +context("SerDe functionality") + +sc <- sparkR.init() + +test_that("SerDe of primitive types", { + x <- callJStatic("SparkRHandler", "echo", 1L) + expect_equal(x, 1L) + expect_equal(class(x), "integer") + + x <- callJStatic("SparkRHandler", "echo", 1) + expect_equal(x, 1) + expect_equal(class(x), "numeric") + + x <- callJStatic("SparkRHandler", "echo", TRUE) + expect_true(x) + expect_equal(class(x), "logical") + + x <- callJStatic("SparkRHandler", "echo", "abc") + expect_equal(x, "abc") + expect_equal(class(x), "character") +}) + +test_that("SerDe of list of primitive types", { + x <- list(1L, 2L, 3L) + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) + expect_equal(class(y[[1]]), "integer") + + x <- list(1, 2, 3) + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) + expect_equal(class(y[[1]]), "numeric") + + x <- list(TRUE, FALSE) + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) + expect_equal(class(y[[1]]), "logical") + + x <- list("a", "b", "c") + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) + expect_equal(class(y[[1]]), "character") + + # Empty list + x <- list() + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) +}) + +test_that("SerDe of list of lists", { + x <- list(list(1L, 2L, 3L), list(1, 2, 3), + list(TRUE, FALSE), list("a", "b", "c")) + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) + + # List of empty lists + x <- list(list(), list()) + y <- callJStatic("SparkRHandler", "echo", x) + expect_equal(x, y) +}) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 7e3b5fc403b25..0c3b0d1f4be20 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -94,7 +94,7 @@ if (isEmpty != 0) { } else if (deserializer == "string") { data <- as.list(readLines(inputCon)) } else if (deserializer == "row") { - data <- SparkR:::readDeserializeRows(inputCon) + data <- SparkR:::readMultipleObjects(inputCon) } # Timing reading input data for execution inputElap <- elapsedSecs() @@ -120,7 +120,7 @@ if (isEmpty != 0) { } else if (deserializer == "string") { data <- readLines(inputCon) } else if (deserializer == "row") { - data <- SparkR:::readDeserializeRows(inputCon) + data <- SparkR:::readMultipleObjects(inputCon) } # Timing reading input data for execution inputElap <- elapsedSecs() diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 6ce02e2ea336a..bb82f3285f1d9 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -53,6 +53,13 @@ private[r] class RBackendHandler(server: RBackend) if (objId == "SparkRHandler") { methodName match { + // This function is for test-purpose only + case "echo" => + val args = readArgs(numArgs, dis) + assert(numArgs == 1) + + writeInt(dos, 0) + writeObject(dos, args(0)) case "stopBackend" => writeInt(dos, 0) writeType(dos, "void") diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index dbbbcf40c1e96..26ad4f1d4697e 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -149,6 +149,10 @@ private[spark] object SerDe { case 'b' => readBooleanArr(dis) case 'j' => readStringArr(dis).map(x => JVMObjectTracker.getObject(x)) case 'r' => readBytesArr(dis) + case 'l' => { + val len = readInt(dis) + (0 until len).map(_ => readList(dis)).toArray + } case _ => throw new IllegalArgumentException(s"Invalid array type $arrType") } } @@ -200,6 +204,9 @@ private[spark] object SerDe { case "date" => dos.writeByte('D') case "time" => dos.writeByte('t') case "raw" => dos.writeByte('r') + // Array of primitive types + case "array" => dos.writeByte('a') + // Array of objects case "list" => dos.writeByte('l') case "jobj" => dos.writeByte('j') case _ => throw new IllegalArgumentException(s"Invalid type $typeStr") @@ -211,26 +218,35 @@ private[spark] object SerDe { writeType(dos, "void") } else { value.getClass.getName match { + case "java.lang.Character" => + writeType(dos, "character") + writeString(dos, value.asInstanceOf[Character].toString) case "java.lang.String" => writeType(dos, "character") writeString(dos, value.asInstanceOf[String]) - case "long" | "java.lang.Long" => + case "java.lang.Long" => writeType(dos, "double") writeDouble(dos, value.asInstanceOf[Long].toDouble) - case "float" | "java.lang.Float" => + case "java.lang.Float" => writeType(dos, "double") writeDouble(dos, value.asInstanceOf[Float].toDouble) - case "decimal" | "java.math.BigDecimal" => + case "java.math.BigDecimal" => writeType(dos, "double") val javaDecimal = value.asInstanceOf[java.math.BigDecimal] writeDouble(dos, scala.math.BigDecimal(javaDecimal).toDouble) - case "double" | "java.lang.Double" => + case "java.lang.Double" => writeType(dos, "double") writeDouble(dos, value.asInstanceOf[Double]) - case "int" | "java.lang.Integer" => + case "java.lang.Byte" => + writeType(dos, "integer") + writeInt(dos, value.asInstanceOf[Byte].toInt) + case "java.lang.Short" => + writeType(dos, "integer") + writeInt(dos, value.asInstanceOf[Short].toInt) + case "java.lang.Integer" => writeType(dos, "integer") writeInt(dos, value.asInstanceOf[Int]) - case "boolean" | "java.lang.Boolean" => + case "java.lang.Boolean" => writeType(dos, "logical") writeBoolean(dos, value.asInstanceOf[Boolean]) case "java.sql.Date" => @@ -242,43 +258,48 @@ private[spark] object SerDe { case "java.sql.Timestamp" => writeType(dos, "time") writeTime(dos, value.asInstanceOf[Timestamp]) + + // Handle arrays + + // Array of primitive types + + // Special handling for byte array case "[B" => writeType(dos, "raw") writeBytes(dos, value.asInstanceOf[Array[Byte]]) - // TODO: Types not handled right now include - // byte, char, short, float - // Handle arrays - case "[Ljava.lang.String;" => - writeType(dos, "list") - writeStringArr(dos, value.asInstanceOf[Array[String]]) + case "[C" => + writeType(dos, "array") + writeStringArr(dos, value.asInstanceOf[Array[Char]].map(_.toString)) + case "[S" => + writeType(dos, "array") + writeIntArr(dos, value.asInstanceOf[Array[Short]].map(_.toInt)) case "[I" => - writeType(dos, "list") + writeType(dos, "array") writeIntArr(dos, value.asInstanceOf[Array[Int]]) case "[J" => - writeType(dos, "list") + writeType(dos, "array") writeDoubleArr(dos, value.asInstanceOf[Array[Long]].map(_.toDouble)) + case "[F" => + writeType(dos, "array") + writeDoubleArr(dos, value.asInstanceOf[Array[Float]].map(_.toDouble)) case "[D" => - writeType(dos, "list") + writeType(dos, "array") writeDoubleArr(dos, value.asInstanceOf[Array[Double]]) case "[Z" => - writeType(dos, "list") + writeType(dos, "array") writeBooleanArr(dos, value.asInstanceOf[Array[Boolean]]) - case "[[B" => + + // Array of objects, null objects use "void" type + case c if c.startsWith("[") => writeType(dos, "list") - writeBytesArr(dos, value.asInstanceOf[Array[Array[Byte]]]) - case otherName => - // Handle array of objects - if (otherName.startsWith("[L")) { - val objArr = value.asInstanceOf[Array[Object]] - writeType(dos, "list") - writeType(dos, "jobj") - dos.writeInt(objArr.length) - objArr.foreach(o => writeJObj(dos, o)) - } else { - writeType(dos, "jobj") - writeJObj(dos, value) - } + val array = value.asInstanceOf[Array[Object]] + writeInt(dos, array.length) + array.foreach(elem => writeObject(dos, elem)) + + case _ => + writeType(dos, "jobj") + writeJObj(dos, value) } } } @@ -350,11 +371,6 @@ private[spark] object SerDe { value.foreach(v => writeString(out, v)) } - def writeBytesArr(out: DataOutputStream, value: Array[Array[Byte]]): Unit = { - writeType(out, "raw") - out.writeInt(value.length) - value.foreach(v => writeBytes(out, v)) - } } private[r] object SerializationFormats { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index 92861ab038f19..7f3defec3d42e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -98,27 +98,17 @@ private[r] object SQLUtils { val bos = new ByteArrayOutputStream() val dos = new DataOutputStream(bos) - SerDe.writeInt(dos, row.length) - (0 until row.length).map { idx => - val obj: Object = row(idx).asInstanceOf[Object] - SerDe.writeObject(dos, obj) - } + val cols = (0 until row.length).map(row(_).asInstanceOf[Object]).toArray + SerDe.writeObject(dos, cols) bos.toByteArray() } - def dfToCols(df: DataFrame): Array[Array[Byte]] = { + def dfToCols(df: DataFrame): Array[Array[Any]] = { // localDF is Array[Row] val localDF = df.collect() val numCols = df.columns.length - // dfCols is Array[Array[Any]] - val dfCols = convertRowsToColumns(localDF, numCols) - - dfCols.map { col => - colToRBytes(col) - } - } - def convertRowsToColumns(localDF: Array[Row], numCols: Int): Array[Array[Any]] = { + // result is Array[Array[Any]] (0 until numCols).map { colIdx => localDF.map { row => row(colIdx) @@ -126,20 +116,6 @@ private[r] object SQLUtils { }.toArray } - def colToRBytes(col: Array[Any]): Array[Byte] = { - val numRows = col.length - val bos = new ByteArrayOutputStream() - val dos = new DataOutputStream(bos) - - SerDe.writeInt(dos, numRows) - - col.map { item => - val obj: Object = item.asInstanceOf[Object] - SerDe.writeObject(dos, obj) - } - bos.toByteArray() - } - def saveMode(mode: String): SaveMode = { mode match { case "append" => SaveMode.Append From c0e9ff1588b4d9313cc6ec6e00e5c7663eb67910 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 25 Aug 2015 13:21:05 -0700 Subject: [PATCH 086/232] [SPARK-9800] Adds docs for GradientDescent$.runMiniBatchSGD alias * Adds doc for alias of runMIniBatchSGD documenting default value for convergeTol * Cleans up a note in code Author: Feynman Liang Closes #8425 from feynmanliang/SPARK-9800. --- .../apache/spark/mllib/optimization/GradientDescent.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 8f0d1e4aa010a..3b663b5defb03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -235,7 +235,7 @@ object GradientDescent extends Logging { if (miniBatchSize > 0) { /** - * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * lossSum is computed using the weights from the previous iteration * and regVal is the regularization value computed in the previous iteration as well. */ stochasticLossHistory.append(lossSum / miniBatchSize + regVal) @@ -264,6 +264,9 @@ object GradientDescent extends Logging { } + /** + * Alias of [[runMiniBatchSGD]] with convergenceTol set to default value of 0.001. + */ def runMiniBatchSGD( data: RDD[(Double, Vector)], gradient: Gradient, From c619c7552f22d28cfa321ce671fc9ca854dd655f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 13:22:38 -0700 Subject: [PATCH 087/232] [SPARK-10237] [MLLIB] update since versions in mllib.fpm Same as #8421 but for `mllib.fpm`. cc feynmanliang Author: Xiangrui Meng Closes #8429 from mengxr/SPARK-10237. --- .../spark/mllib/fpm/AssociationRules.scala | 7 ++++-- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 9 ++++++-- .../apache/spark/mllib/fpm/PrefixSpan.scala | 23 ++++++++++++++++--- 3 files changed, 32 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index ba3b447a83398..95c688c86a7e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -82,12 +82,15 @@ class AssociationRules private[fpm] ( }.filter(_.confidence >= minConfidence) } + /** Java-friendly version of [[run]]. */ + @Since("1.5.0") def run[Item](freqItemsets: JavaRDD[FreqItemset[Item]]): JavaRDD[Rule[Item]] = { val tag = fakeClassTag[Item] run(freqItemsets.rdd)(tag) } } +@Since("1.5.0") object AssociationRules { /** @@ -104,8 +107,8 @@ object AssociationRules { @Since("1.5.0") @Experimental class Rule[Item] private[fpm] ( - val antecedent: Array[Item], - val consequent: Array[Item], + @Since("1.5.0") val antecedent: Array[Item], + @Since("1.5.0") val consequent: Array[Item], freqUnion: Double, freqAntecedent: Double) extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index e37f806271680..aea5c4f8a8a7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -42,7 +42,8 @@ import org.apache.spark.storage.StorageLevel */ @Since("1.3.0") @Experimental -class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable { +class FPGrowthModel[Item: ClassTag] @Since("1.3.0") ( + @Since("1.3.0") val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable { /** * Generates association rules for the [[Item]]s in [[freqItemsets]]. * @param confidence minimal confidence of the rules produced @@ -126,6 +127,8 @@ class FPGrowth private ( new FPGrowthModel(freqItemsets) } + /** Java-friendly version of [[run]]. */ + @Since("1.3.0") def run[Item, Basket <: JavaIterable[Item]](data: JavaRDD[Basket]): FPGrowthModel[Item] = { implicit val tag = fakeClassTag[Item] run(data.rdd.map(_.asScala.toArray)) @@ -226,7 +229,9 @@ object FPGrowth { * */ @Since("1.3.0") - class FreqItemset[Item](val items: Array[Item], val freq: Long) extends Serializable { + class FreqItemset[Item] @Since("1.3.0") ( + @Since("1.3.0") val items: Array[Item], + @Since("1.3.0") val freq: Long) extends Serializable { /** * Returns items in a Java List. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index dc4ae1d0b69ed..97916daa2e9ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.spark.Logging -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.rdd.RDD @@ -51,6 +51,7 @@ import org.apache.spark.storage.StorageLevel * (Wikipedia)]] */ @Experimental +@Since("1.5.0") class PrefixSpan private ( private var minSupport: Double, private var maxPatternLength: Int, @@ -61,17 +62,20 @@ class PrefixSpan private ( * Constructs a default instance with default parameters * {minSupport: `0.1`, maxPatternLength: `10`, maxLocalProjDBSize: `32000000L`}. */ + @Since("1.5.0") def this() = this(0.1, 10, 32000000L) /** * Get the minimal support (i.e. the frequency of occurrence before a pattern is considered * frequent). */ + @Since("1.5.0") def getMinSupport: Double = minSupport /** * Sets the minimal support level (default: `0.1`). */ + @Since("1.5.0") def setMinSupport(minSupport: Double): this.type = { require(minSupport >= 0 && minSupport <= 1, s"The minimum support value must be in [0, 1], but got $minSupport.") @@ -82,11 +86,13 @@ class PrefixSpan private ( /** * Gets the maximal pattern length (i.e. the length of the longest sequential pattern to consider. */ + @Since("1.5.0") def getMaxPatternLength: Int = maxPatternLength /** * Sets maximal pattern length (default: `10`). */ + @Since("1.5.0") def setMaxPatternLength(maxPatternLength: Int): this.type = { // TODO: support unbounded pattern length when maxPatternLength = 0 require(maxPatternLength >= 1, @@ -98,12 +104,14 @@ class PrefixSpan private ( /** * Gets the maximum number of items allowed in a projected database before local processing. */ + @Since("1.5.0") def getMaxLocalProjDBSize: Long = maxLocalProjDBSize /** * Sets the maximum number of items (including delimiters used in the internal storage format) * allowed in a projected database before local processing (default: `32000000L`). */ + @Since("1.5.0") def setMaxLocalProjDBSize(maxLocalProjDBSize: Long): this.type = { require(maxLocalProjDBSize >= 0L, s"The maximum local projected database size must be nonnegative, but got $maxLocalProjDBSize") @@ -116,6 +124,7 @@ class PrefixSpan private ( * @param data sequences of itemsets. * @return a [[PrefixSpanModel]] that contains the frequent patterns */ + @Since("1.5.0") def run[Item: ClassTag](data: RDD[Array[Array[Item]]]): PrefixSpanModel[Item] = { if (data.getStorageLevel == StorageLevel.NONE) { logWarning("Input data is not cached.") @@ -202,6 +211,7 @@ class PrefixSpan private ( * @tparam Sequence sequence type, which is an Iterable of Itemsets * @return a [[PrefixSpanModel]] that contains the frequent sequential patterns */ + @Since("1.5.0") def run[Item, Itemset <: jl.Iterable[Item], Sequence <: jl.Iterable[Itemset]]( data: JavaRDD[Sequence]): PrefixSpanModel[Item] = { implicit val tag = fakeClassTag[Item] @@ -211,6 +221,7 @@ class PrefixSpan private ( } @Experimental +@Since("1.5.0") object PrefixSpan extends Logging { /** @@ -535,10 +546,14 @@ object PrefixSpan extends Logging { * @param freq frequency * @tparam Item item type */ - class FreqSequence[Item](val sequence: Array[Array[Item]], val freq: Long) extends Serializable { + @Since("1.5.0") + class FreqSequence[Item] @Since("1.5.0") ( + @Since("1.5.0") val sequence: Array[Array[Item]], + @Since("1.5.0") val freq: Long) extends Serializable { /** * Returns sequence as a Java List of lists for Java users. */ + @Since("1.5.0") def javaSequence: ju.List[ju.List[Item]] = sequence.map(_.toList.asJava).toList.asJava } } @@ -548,5 +563,7 @@ object PrefixSpan extends Logging { * @param freqSequences frequent sequences * @tparam Item item type */ -class PrefixSpanModel[Item](val freqSequences: RDD[PrefixSpan.FreqSequence[Item]]) +@Since("1.5.0") +class PrefixSpanModel[Item] @Since("1.5.0") ( + @Since("1.5.0") val freqSequences: RDD[PrefixSpan.FreqSequence[Item]]) extends Serializable From 9205907876cf65695e56c2a94bedd83df3675c03 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 25 Aug 2015 13:23:15 -0700 Subject: [PATCH 088/232] [SPARK-9797] [MLLIB] [DOC] StreamingLinearRegressionWithSGD.setConvergenceTol default value Adds default convergence tolerance (0.001, set in `GradientDescent.convergenceTol`) to `setConvergenceTol`'s scaladoc Author: Feynman Liang Closes #8424 from feynmanliang/SPARK-9797. --- .../mllib/regression/StreamingLinearRegressionWithSGD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index 537a05274eec2..26654e4a06838 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -93,7 +93,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( } /** - * Set the convergence tolerance. + * Set the convergence tolerance. Default: 0.001. */ def setConvergenceTol(tolerance: Double): this.type = { this.algorithm.optimizer.setConvergenceTol(tolerance) From 00ae4be97f7b205432db2967ba6d506286ef2ca6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 14:11:38 -0700 Subject: [PATCH 089/232] [SPARK-10239] [SPARK-10244] [MLLIB] update since versions in mllib.pmml and mllib.util Same as #8421 but for `mllib.pmml` and `mllib.util`. cc dbtsai Author: Xiangrui Meng Closes #8430 from mengxr/SPARK-10239 and squashes the following commits: a189acf [Xiangrui Meng] update since versions in mllib.pmml and mllib.util --- .../org/apache/spark/mllib/pmml/PMMLExportable.scala | 7 ++++++- .../org/apache/spark/mllib/util/DataValidators.scala | 7 +++++-- .../apache/spark/mllib/util/KMeansDataGenerator.scala | 5 ++++- .../apache/spark/mllib/util/LinearDataGenerator.scala | 10 ++++++++-- .../mllib/util/LogisticRegressionDataGenerator.scala | 5 ++++- .../org/apache/spark/mllib/util/MFDataGenerator.scala | 4 +++- .../scala/org/apache/spark/mllib/util/MLUtils.scala | 2 ++ .../org/apache/spark/mllib/util/SVMDataGenerator.scala | 6 ++++-- .../org/apache/spark/mllib/util/modelSaveLoad.scala | 6 +++++- 9 files changed, 41 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala index 5e882d4ebb10b..274ac7c99553b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/PMMLExportable.scala @@ -23,7 +23,7 @@ import javax.xml.transform.stream.StreamResult import org.jpmml.model.JAXBUtil import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory /** @@ -33,6 +33,7 @@ import org.apache.spark.mllib.pmml.export.PMMLModelExportFactory * developed by the Data Mining Group (www.dmg.org). */ @DeveloperApi +@Since("1.4.0") trait PMMLExportable { /** @@ -48,6 +49,7 @@ trait PMMLExportable { * Export the model to a local file in PMML format */ @Experimental + @Since("1.4.0") def toPMML(localPath: String): Unit = { toPMML(new StreamResult(new File(localPath))) } @@ -57,6 +59,7 @@ trait PMMLExportable { * Export the model to a directory on a distributed file system in PMML format */ @Experimental + @Since("1.4.0") def toPMML(sc: SparkContext, path: String): Unit = { val pmml = toPMML() sc.parallelize(Array(pmml), 1).saveAsTextFile(path) @@ -67,6 +70,7 @@ trait PMMLExportable { * Export the model to the OutputStream in PMML format */ @Experimental + @Since("1.4.0") def toPMML(outputStream: OutputStream): Unit = { toPMML(new StreamResult(outputStream)) } @@ -76,6 +80,7 @@ trait PMMLExportable { * Export the model to a String in PMML format */ @Experimental + @Since("1.4.0") def toPMML(): String = { val writer = new StringWriter toPMML(new StreamResult(writer)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index be335a1aca58a..dffe6e78939e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,16 +17,17 @@ package org.apache.spark.mllib.util -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: * A collection of methods used to validate data before applying ML algorithms. */ @DeveloperApi +@Since("0.8.0") object DataValidators extends Logging { /** @@ -34,6 +35,7 @@ object DataValidators extends Logging { * * @return True if labels are all zero or one, false otherwise. */ + @Since("1.0.0") val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { @@ -48,6 +50,7 @@ object DataValidators extends Logging { * * @return True if labels are all in the range of {0, 1, ..., k-1}, false otherwise. */ + @Since("1.3.0") def multiLabelValidator(k: Int): RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label - x.label.toInt != 0.0 || x.label < 0 || x.label > k - 1).count() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index e6bcff48b022c..00fd1606a369c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.rdd.RDD /** @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD * cluster with scale 1 around each center. */ @DeveloperApi +@Since("0.8.0") object KMeansDataGenerator { /** @@ -42,6 +43,7 @@ object KMeansDataGenerator { * @param r Scaling factor for the distribution of the initial centers * @param numPartitions Number of partitions of the generated RDD; default 2 */ + @Since("0.8.0") def generateKMeansRDD( sc: SparkContext, numPoints: Int, @@ -62,6 +64,7 @@ object KMeansDataGenerator { } } + @Since("0.8.0") def main(args: Array[String]) { if (args.length < 6) { // scalastyle:off println diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 7a1c7796065ee..d0ba454f379a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,11 +22,11 @@ import scala.util.Random import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: @@ -35,6 +35,7 @@ import org.apache.spark.mllib.regression.LabeledPoint * response variable `Y`. */ @DeveloperApi +@Since("0.8.0") object LinearDataGenerator { /** @@ -46,6 +47,7 @@ object LinearDataGenerator { * @param seed Random seed * @return Java List of input. */ + @Since("0.8.0") def generateLinearInputAsList( intercept: Double, weights: Array[Double], @@ -68,6 +70,7 @@ object LinearDataGenerator { * @param eps Epsilon scaling factor. * @return Seq of input. */ + @Since("0.8.0") def generateLinearInput( intercept: Double, weights: Array[Double], @@ -92,6 +95,7 @@ object LinearDataGenerator { * @param eps Epsilon scaling factor. * @return Seq of input. */ + @Since("0.8.0") def generateLinearInput( intercept: Double, weights: Array[Double], @@ -132,6 +136,7 @@ object LinearDataGenerator { * * @return RDD of LabeledPoint containing sample data. */ + @Since("0.8.0") def generateLinearRDD( sc: SparkContext, nexamples: Int, @@ -151,6 +156,7 @@ object LinearDataGenerator { data } + @Since("0.8.0") def main(args: Array[String]) { if (args.length < 2) { // scalastyle:off println diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index c09cbe69bb971..33477ee20ebbd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Since, DeveloperApi} import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint @@ -31,6 +31,7 @@ import org.apache.spark.mllib.linalg.Vectors * with probability `probOne` and scales features for positive examples by `eps`. */ @DeveloperApi +@Since("0.8.0") object LogisticRegressionDataGenerator { /** @@ -43,6 +44,7 @@ object LogisticRegressionDataGenerator { * @param nparts Number of partitions of the generated RDD. Default value is 2. * @param probOne Probability that a label is 1 (and not 0). Default value is 0.5. */ + @Since("0.8.0") def generateLogisticRDD( sc: SparkContext, nexamples: Int, @@ -62,6 +64,7 @@ object LogisticRegressionDataGenerator { data } + @Since("0.8.0") def main(args: Array[String]) { if (args.length != 5) { // scalastyle:off println diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 16f430599a515..906bd30563bd0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -23,7 +23,7 @@ import scala.language.postfixOps import scala.util.Random import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Since, DeveloperApi} import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD @@ -52,7 +52,9 @@ import org.apache.spark.rdd.RDD * testSampFact (Double) Percentage of training data to use as test data. */ @DeveloperApi +@Since("0.8.0") object MFDataGenerator { + @Since("0.8.0") def main(args: Array[String]) { if (args.length < 2) { // scalastyle:off println diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 4940974bf4f41..81c2f0ce6e12c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -36,6 +36,7 @@ import org.apache.spark.streaming.dstream.DStream /** * Helper methods to load, save and pre-process data used in ML Lib. */ +@Since("0.8.0") object MLUtils { private[mllib] lazy val EPSILON = { @@ -168,6 +169,7 @@ object MLUtils { * * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] */ + @Since("1.0.0") def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { // TODO: allow to specify label precision and feature precision. val dataStr = data.map { case LabeledPoint(label, features) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index ad20b7694a779..cde5979396178 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -21,11 +21,11 @@ import scala.util.Random import com.github.fommil.netlib.BLAS.{getInstance => blas} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: @@ -33,8 +33,10 @@ import org.apache.spark.mllib.regression.LabeledPoint * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ @DeveloperApi +@Since("0.8.0") object SVMDataGenerator { + @Since("0.8.0") def main(args: Array[String]) { if (args.length < 2) { // scalastyle:off println diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala index 30d642c754b7c..4d71d534a0774 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -24,7 +24,7 @@ import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types.{DataType, StructField, StructType} @@ -35,6 +35,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * This should be inherited by the class which implements model instances. */ @DeveloperApi +@Since("1.3.0") trait Saveable { /** @@ -50,6 +51,7 @@ trait Saveable { * @param path Path specifying the directory in which to save this model. * If the directory already exists, this method throws an exception. */ + @Since("1.3.0") def save(sc: SparkContext, path: String): Unit /** Current version of model save/load format. */ @@ -64,6 +66,7 @@ trait Saveable { * This should be inherited by an object paired with the model class. */ @DeveloperApi +@Since("1.3.0") trait Loader[M <: Saveable] { /** @@ -75,6 +78,7 @@ trait Loader[M <: Saveable] { * @param path Path specifying the directory to which the model was saved. * @return Model instance */ + @Since("1.3.0") def load(sc: SparkContext, path: String): M } From ec89bd840a6862751999d612f586a962cae63f6d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 25 Aug 2015 14:55:34 -0700 Subject: [PATCH 090/232] [SPARK-10245] [SQL] Fix decimal literals with precision < scale In BigDecimal or java.math.BigDecimal, the precision could be smaller than scale, for example, BigDecimal("0.001") has precision = 1 and scale = 3. But DecimalType require that the precision should be larger than scale, so we should use the maximum of precision and scale when inferring the schema from decimal literal. Author: Davies Liu Closes #8428 from davies/smaller_decimal. --- .../spark/sql/catalyst/expressions/literals.scala | 7 ++++--- .../catalyst/expressions/LiteralExpressionSuite.scala | 8 +++++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 3 files changed, 19 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 34bad23802ba4..8c0c5d5b1e31e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -36,9 +36,10 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(UTF8String.fromString(s), StringType) case b: Boolean => Literal(b, BooleanType) - case d: BigDecimal => Literal(Decimal(d), DecimalType(d.precision, d.scale)) - case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType(d.precision(), d.scale())) - case d: Decimal => Literal(d, DecimalType(d.precision, d.scale)) + case d: BigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale)) + case d: java.math.BigDecimal => + Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale())) + case d: Decimal => Literal(d, DecimalType(Math.max(d.precision, d.scale), d.scale)) case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index f6404d21611e5..015eb1897fb8c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -83,12 +83,14 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("decimal") { - List(0.0, 1.2, 1.1111, 5).foreach { d => + List(-0.0001, 0.0, 0.001, 1.2, 1.1111, 5).foreach { d => checkEvaluation(Literal(Decimal(d)), Decimal(d)) checkEvaluation(Literal(Decimal(d.toInt)), Decimal(d.toInt)) checkEvaluation(Literal(Decimal(d.toLong)), Decimal(d.toLong)) - checkEvaluation(Literal(Decimal((d * 1000L).toLong, 10, 1)), - Decimal((d * 1000L).toLong, 10, 1)) + checkEvaluation(Literal(Decimal((d * 1000L).toLong, 10, 3)), + Decimal((d * 1000L).toLong, 10, 3)) + checkEvaluation(Literal(BigDecimal(d.toString)), Decimal(d)) + checkEvaluation(Literal(new java.math.BigDecimal(d.toString)), Decimal(d)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index dcb4e83710982..aa07665c6b705 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1627,6 +1627,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(null)) } + test("precision smaller than scale") { + checkAnswer(sql("select 10.00"), Row(BigDecimal("10.00"))) + checkAnswer(sql("select 1.00"), Row(BigDecimal("1.00"))) + checkAnswer(sql("select 0.10"), Row(BigDecimal("0.10"))) + checkAnswer(sql("select 0.01"), Row(BigDecimal("0.01"))) + checkAnswer(sql("select 0.001"), Row(BigDecimal("0.001"))) + checkAnswer(sql("select -0.01"), Row(BigDecimal("-0.01"))) + checkAnswer(sql("select -0.001"), Row(BigDecimal("-0.001"))) + } + test("external sorting updates peak execution memory") { withSQLConf((SQLConf.EXTERNAL_SORT.key, "true")) { val sc = sqlContext.sparkContext From 7467b52ed07f174d93dfc4cb544dc4b69a2c2826 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 25 Aug 2015 15:19:41 -0700 Subject: [PATCH 091/232] [SPARK-10215] [SQL] Fix precision of division (follow the rule in Hive) Follow the rule in Hive for decimal division. see https://github.com/apache/hive/blob/ac755ebe26361a4647d53db2a28500f71697b276/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPDivide.java#L113 cc chenghao-intel Author: Davies Liu Closes #8415 from davies/decimal_div2. --- .../catalyst/analysis/HiveTypeCoercion.scala | 10 ++++++-- .../sql/catalyst/analysis/AnalysisSuite.scala | 9 +++---- .../analysis/DecimalPrecisionSuite.scala | 8 +++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 25 +++++++++++++++++-- 4 files changed, 39 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index a1aa2a2b2c680..87c11abbad490 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -396,8 +396,14 @@ object HiveTypeCoercion { resultType) case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val resultType = DecimalType.bounded(p1 - s1 + s2 + max(6, s1 + p2 + 1), - max(6, s1 + p2 + 1)) + var intDig = min(DecimalType.MAX_SCALE, p1 - s1 + s2) + var decDig = min(DecimalType.MAX_SCALE, max(6, s1 + p2 + 1)) + val diff = (intDig + decDig) - DecimalType.MAX_SCALE + if (diff > 0) { + decDig -= diff / 2 + 1 + intDig = DecimalType.MAX_SCALE - decDig + } + val resultType = DecimalType.bounded(intDig + decDig, decDig) val widerType = widerDecimalType(p1, s1, p2, s2) CheckOverflow(Divide(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), resultType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 1e0cc81dae974..820b336aac759 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,15 +17,14 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.SimpleCatalystConf -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.dsl.plans._ class AnalysisSuite extends AnalysisTest { - import TestRelations._ + import org.apache.spark.sql.catalyst.analysis.TestRelations._ test("union project *") { val plan = (1 to 100) @@ -96,7 +95,7 @@ class AnalysisSuite extends AnalysisTest { assert(pl(1).dataType == DoubleType) assert(pl(2).dataType == DoubleType) // StringType will be promoted into Decimal(38, 18) - assert(pl(3).dataType == DecimalType(38, 29)) + assert(pl(3).dataType == DecimalType(38, 22)) assert(pl(4).dataType == DoubleType) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index fc11627da6fd1..b4ad618c23e39 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -136,10 +136,10 @@ class DecimalPrecisionSuite extends SparkFunSuite with BeforeAndAfter { checkType(Multiply(i, u), DecimalType(38, 18)) checkType(Multiply(u, u), DecimalType(38, 36)) - checkType(Divide(u, d1), DecimalType(38, 21)) - checkType(Divide(u, d2), DecimalType(38, 24)) - checkType(Divide(u, i), DecimalType(38, 29)) - checkType(Divide(u, u), DecimalType(38, 38)) + checkType(Divide(u, d1), DecimalType(38, 18)) + checkType(Divide(u, d2), DecimalType(38, 19)) + checkType(Divide(u, i), DecimalType(38, 23)) + checkType(Divide(u, u), DecimalType(38, 18)) checkType(Remainder(d1, u), DecimalType(19, 18)) checkType(Remainder(d2, u), DecimalType(21, 18)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index aa07665c6b705..9e172b2c264cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1622,9 +1622,30 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select 10.3000 / 3.0"), Row(BigDecimal("3.4333333"))) checkAnswer(sql("select 10.30000 / 30.0"), Row(BigDecimal("0.343333333"))) checkAnswer(sql("select 10.300000000000000000 / 3.00000000000000000"), - Row(BigDecimal("3.4333333333333333333333333333333333333", new MathContext(38)))) + Row(BigDecimal("3.433333333333333333333333333", new MathContext(38)))) checkAnswer(sql("select 10.3000000000000000000 / 3.00000000000000000"), - Row(null)) + Row(BigDecimal("3.4333333333333333333333333333", new MathContext(38)))) + } + + test("SPARK-10215 Div of Decimal returns null") { + val d = Decimal(1.12321) + val df = Seq((d, 1)).toDF("a", "b") + + checkAnswer( + df.selectExpr("b * a / b"), + Seq(Row(d.toBigDecimal))) + checkAnswer( + df.selectExpr("b * a / b / b"), + Seq(Row(d.toBigDecimal))) + checkAnswer( + df.selectExpr("b * a + b"), + Seq(Row(BigDecimal(2.12321)))) + checkAnswer( + df.selectExpr("b * a - b"), + Seq(Row(BigDecimal(0.12321)))) + checkAnswer( + df.selectExpr("b * a * b"), + Seq(Row(d.toBigDecimal))) } test("precision smaller than scale") { From 125205cdb35530cdb4a8fff3e1ee49cf4a299583 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Tue, 25 Aug 2015 17:39:20 -0700 Subject: [PATCH 092/232] [SPARK-9888] [MLLIB] User guide for new LDA features * Adds two new sections to LDA's user guide; one for each optimizer/model * Documents new features added to LDA (e.g. topXXXperXXX, asymmetric priors, hyperpam optimization) * Cleans up a TODO and sets a default parameter in LDA code jkbradley hhbyyh Author: Feynman Liang Closes #8254 from feynmanliang/SPARK-9888. --- docs/mllib-clustering.md | 135 +++++++++++++++--- .../spark/mllib/clustering/LDAModel.scala | 1 - .../spark/mllib/clustering/LDASuite.scala | 1 + 3 files changed, 117 insertions(+), 20 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index fd9ab258e196d..3fb35d3c50b06 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -438,28 +438,125 @@ sameModel = PowerIterationClusteringModel.load(sc, "myModelPath") is a topic model which infers topics from a collection of text documents. LDA can be thought of as a clustering algorithm as follows: -* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset. -* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts. -* Rather than estimating a clustering using a traditional distance, LDA uses a function based - on a statistical model of how text documents are generated. - -LDA takes in a collection of documents as vectors of word counts. -It supports different inference algorithms via `setOptimizer` function. EMLDAOptimizer learns clustering using [expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) -on the likelihood function and yields comprehensive results, while OnlineLDAOptimizer uses iterative mini-batch sampling for [online variational inference](https://www.cs.princeton.edu/~blei/papers/HoffmanBleiBach2010b.pdf) and is generally memory friendly. After fitting on the documents, LDA provides: - -* Topics: Inferred topics, each of which is a probability distribution over terms (words). -* Topic distributions for documents: For each non empty document in the training set, LDA gives a probability distribution over topics. (EM only). Note that for empty documents, we don't create the topic distributions. (EM only) +* Topics correspond to cluster centers, and documents correspond to +examples (rows) in a dataset. +* Topics and documents both exist in a feature space, where feature +vectors are vectors of word counts (bag of words). +* Rather than estimating a clustering using a traditional distance, LDA +uses a function based on a statistical model of how text documents are +generated. + +LDA supports different inference algorithms via `setOptimizer` function. +`EMLDAOptimizer` learns clustering using +[expectation-maximization](http://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) +on the likelihood function and yields comprehensive results, while +`OnlineLDAOptimizer` uses iterative mini-batch sampling for [online +variational +inference](https://www.cs.princeton.edu/~blei/papers/HoffmanBleiBach2010b.pdf) +and is generally memory friendly. -LDA takes the following parameters: +LDA takes in a collection of documents as vectors of word counts and the +following parameters (set using the builder pattern): * `k`: Number of topics (i.e., cluster centers) -* `maxIterations`: Limit on the number of iterations of EM used for learning -* `docConcentration`: Hyperparameter for prior over documents' distributions over topics. Currently must be > 1, where larger values encourage smoother inferred distributions. -* `topicConcentration`: Hyperparameter for prior over topics' distributions over terms (words). Currently must be > 1, where larger values encourage smoother inferred distributions. -* `checkpointInterval`: If using checkpointing (set in the Spark configuration), this parameter specifies the frequency with which checkpoints will be created. If `maxIterations` is large, using checkpointing can help reduce shuffle file sizes on disk and help with failure recovery. - -*Note*: LDA is a new feature with some missing functionality. In particular, it does not yet -support prediction on new documents, and it does not have a Python API. These will be added in the future. +* `optimizer`: Optimizer to use for learning the LDA model, either +`EMLDAOptimizer` or `OnlineLDAOptimizer` +* `docConcentration`: Dirichlet parameter for prior over documents' +distributions over topics. Larger values encourage smoother inferred +distributions. +* `topicConcentration`: Dirichlet parameter for prior over topics' +distributions over terms (words). Larger values encourage smoother +inferred distributions. +* `maxIterations`: Limit on the number of iterations. +* `checkpointInterval`: If using checkpointing (set in the Spark +configuration), this parameter specifies the frequency with which +checkpoints will be created. If `maxIterations` is large, using +checkpointing can help reduce shuffle file sizes on disk and help with +failure recovery. + + +All of MLlib's LDA models support: + +* `describeTopics`: Returns topics as arrays of most important terms and +term weights +* `topicsMatrix`: Returns a `vocabSize` by `k` matrix where each column +is a topic + +*Note*: LDA is still an experimental feature under active development. +As a result, certain features are only available in one of the two +optimizers / models generated by the optimizer. Currently, a distributed +model can be converted into a local model, but not vice-versa. + +The following discussion will describe each optimizer/model pair +separately. + +**Expectation Maximization** + +Implemented in +[`EMLDAOptimizer`](api/scala/index.html#org.apache.spark.mllib.clustering.EMLDAOptimizer) +and +[`DistributedLDAModel`](api/scala/index.html#org.apache.spark.mllib.clustering.DistributedLDAModel). + +For the parameters provided to `LDA`: + +* `docConcentration`: Only symmetric priors are supported, so all values +in the provided `k`-dimensional vector must be identical. All values +must also be $> 1.0$. Providing `Vector(-1)` results in default behavior +(uniform `k` dimensional vector with value $(50 / k) + 1$ +* `topicConcentration`: Only symmetric priors supported. Values must be +$> 1.0$. Providing `-1` results in defaulting to a value of $0.1 + 1$. +* `maxIterations`: The maximum number of EM iterations. + +`EMLDAOptimizer` produces a `DistributedLDAModel`, which stores not only +the inferred topics but also the full training corpus and topic +distributions for each document in the training corpus. A +`DistributedLDAModel` supports: + + * `topTopicsPerDocument`: The top topics and their weights for + each document in the training corpus + * `topDocumentsPerTopic`: The top documents for each topic and + the corresponding weight of the topic in the documents. + * `logPrior`: log probability of the estimated topics and + document-topic distributions given the hyperparameters + `docConcentration` and `topicConcentration` + * `logLikelihood`: log likelihood of the training corpus, given the + inferred topics and document-topic distributions + +**Online Variational Bayes** + +Implemented in +[`OnlineLDAOptimizer`](api/scala/org/apache/spark/mllib/clustering/OnlineLDAOptimizer.html) +and +[`LocalLDAModel`](api/scala/org/apache/spark/mllib/clustering/LocalLDAModel.html). + +For the parameters provided to `LDA`: + +* `docConcentration`: Asymmetric priors can be used by passing in a +vector with values equal to the Dirichlet parameter in each of the `k` +dimensions. Values should be $>= 0$. Providing `Vector(-1)` results in +default behavior (uniform `k` dimensional vector with value $(1.0 / k)$) +* `topicConcentration`: Only symmetric priors supported. Values must be +$>= 0$. Providing `-1` results in defaulting to a value of $(1.0 / k)$. +* `maxIterations`: Maximum number of minibatches to submit. + +In addition, `OnlineLDAOptimizer` accepts the following parameters: + +* `miniBatchFraction`: Fraction of corpus sampled and used at each +iteration +* `optimizeDocConcentration`: If set to true, performs maximum-likelihood +estimation of the hyperparameter `docConcentration` (aka `alpha`) +after each minibatch and sets the optimized `docConcentration` in the +returned `LocalLDAModel` +* `tau0` and `kappa`: Used for learning-rate decay, which is computed by +$(\tau_0 + iter)^{-\kappa}$ where $iter$ is the current number of iterations. + +`OnlineLDAOptimizer` produces a `LocalLDAModel`, which only stores the +inferred topics. A `LocalLDAModel` supports: + +* `logLikelihood(documents)`: Calculates a lower bound on the provided +`documents` given the inferred topics. +* `logPerplexity(documents)`: Calculates an upper bound on the +perplexity of the provided `documents` given the inferred topics. **Examples** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 667374a2bc418..432bbedc8d6f8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -435,7 +435,6 @@ object LocalLDAModel extends Loader[LocalLDAModel] { } val topicsMat = Matrices.fromBreeze(brzTopics) - // TODO: initialize with docConcentration, topicConcentration, and gammaShape after SPARK-9940 new LocalLDAModel(topicsMat, docConcentration, topicConcentration, gammaShape) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 746a76a7e5fa1..37fb69d68f6be 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -68,6 +68,7 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { // Train a model val lda = new LDA() lda.setK(k) + .setOptimizer(new EMLDAOptimizer) .setDocConcentration(topicSmoothing) .setTopicConcentration(termSmoothing) .setMaxIterations(5) From 8668ead2e7097b9591069599fbfccf67c53db659 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 18:17:54 -0700 Subject: [PATCH 093/232] [SPARK-10233] [MLLIB] update since version in mllib.evaluation Same as #8421 but for `mllib.evaluation`. cc avulanov Author: Xiangrui Meng Closes #8423 from mengxr/SPARK-10233. --- .../evaluation/BinaryClassificationMetrics.scala | 8 ++++---- .../spark/mllib/evaluation/MulticlassMetrics.scala | 11 ++++++++++- .../spark/mllib/evaluation/MultilabelMetrics.scala | 12 +++++++++++- .../spark/mllib/evaluation/RegressionMetrics.scala | 3 ++- 4 files changed, 27 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 76ae847921f44..508fe532b1306 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -42,11 +42,11 @@ import org.apache.spark.sql.DataFrame * be smaller as a result, meaning there may be an extra sample at * partition boundaries. */ -@Since("1.3.0") +@Since("1.0.0") @Experimental -class BinaryClassificationMetrics( - val scoreAndLabels: RDD[(Double, Double)], - val numBins: Int) extends Logging { +class BinaryClassificationMetrics @Since("1.3.0") ( + @Since("1.3.0") val scoreAndLabels: RDD[(Double, Double)], + @Since("1.3.0") val numBins: Int) extends Logging { require(numBins >= 0, "numBins must be nonnegative") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 02e89d921033c..00e837661dfc2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.DataFrame */ @Since("1.1.0") @Experimental -class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { +class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Double)]) { /** * An auxiliary constructor taking a DataFrame. @@ -140,6 +140,7 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns precision */ + @Since("1.1.0") lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount /** @@ -148,23 +149,27 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { * because sum of all false positives is equal to sum * of all false negatives) */ + @Since("1.1.0") lazy val recall: Double = precision /** * Returns f-measure * (equals to precision and recall because precision equals recall) */ + @Since("1.1.0") lazy val fMeasure: Double = precision /** * Returns weighted true positive rate * (equals to precision, recall and f-measure) */ + @Since("1.1.0") lazy val weightedTruePositiveRate: Double = weightedRecall /** * Returns weighted false positive rate */ + @Since("1.1.0") lazy val weightedFalsePositiveRate: Double = labelCountByClass.map { case (category, count) => falsePositiveRate(category) * count.toDouble / labelCount }.sum @@ -173,6 +178,7 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { * Returns weighted averaged recall * (equals to precision, recall and f-measure) */ + @Since("1.1.0") lazy val weightedRecall: Double = labelCountByClass.map { case (category, count) => recall(category) * count.toDouble / labelCount }.sum @@ -180,6 +186,7 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns weighted averaged precision */ + @Since("1.1.0") lazy val weightedPrecision: Double = labelCountByClass.map { case (category, count) => precision(category) * count.toDouble / labelCount }.sum @@ -196,6 +203,7 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns weighted averaged f1-measure */ + @Since("1.1.0") lazy val weightedFMeasure: Double = labelCountByClass.map { case (category, count) => fMeasure(category, 1.0) * count.toDouble / labelCount }.sum @@ -203,5 +211,6 @@ class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { /** * Returns the sequence of labels in ascending order */ + @Since("1.1.0") lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala index a0a8d9c56847b..c100b3c9ec14a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.DataFrame * both are non-null Arrays, each with unique elements. */ @Since("1.2.0") -class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])]) { +class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double], Array[Double])]) { /** * An auxiliary constructor taking a DataFrame. @@ -46,6 +46,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns subset accuracy * (for equal sets of labels) */ + @Since("1.2.0") lazy val subsetAccuracy: Double = predictionAndLabels.filter { case (predictions, labels) => predictions.deep == labels.deep }.count().toDouble / numDocs @@ -53,6 +54,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns accuracy */ + @Since("1.2.0") lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) => labels.intersect(predictions).size.toDouble / (labels.size + predictions.size - labels.intersect(predictions).size)}.sum / numDocs @@ -61,6 +63,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns Hamming-loss */ + @Since("1.2.0") lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) => labels.size + predictions.size - 2 * labels.intersect(predictions).size }.sum / (numDocs * numLabels) @@ -68,6 +71,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns document-based precision averaged by the number of documents */ + @Since("1.2.0") lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) => if (predictions.size > 0) { predictions.intersect(labels).size.toDouble / predictions.size @@ -79,6 +83,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns document-based recall averaged by the number of documents */ + @Since("1.2.0") lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) => labels.intersect(predictions).size.toDouble / labels.size }.sum / numDocs @@ -86,6 +91,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] /** * Returns document-based f1-measure averaged by the number of documents */ + @Since("1.2.0") lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) => 2.0 * predictions.intersect(labels).size / (predictions.size + labels.size) }.sum / numDocs @@ -143,6 +149,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based precision * (equals to micro-averaged document-based precision) */ + @Since("1.2.0") lazy val microPrecision: Double = { val sumFp = fpPerClass.foldLeft(0L){ case(cum, (_, fp)) => cum + fp} sumTp.toDouble / (sumTp + sumFp) @@ -152,6 +159,7 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based recall * (equals to micro-averaged document-based recall) */ + @Since("1.2.0") lazy val microRecall: Double = { val sumFn = fnPerClass.foldLeft(0.0){ case(cum, (_, fn)) => cum + fn} sumTp.toDouble / (sumTp + sumFn) @@ -161,10 +169,12 @@ class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])] * Returns micro-averaged label-based f1-measure * (equals to micro-averaged document-based f1-measure) */ + @Since("1.2.0") lazy val microF1Measure: Double = 2.0 * sumTp / (2 * sumTp + sumFnClass + sumFpClass) /** * Returns the sequence of labels in ascending order */ + @Since("1.2.0") lazy val labels: Array[Double] = tpPerClass.keys.toArray.sorted } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 36a6c357c3897..799ebb980ef01 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -32,7 +32,8 @@ import org.apache.spark.sql.DataFrame */ @Since("1.2.0") @Experimental -class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extends Logging { +class RegressionMetrics @Since("1.2.0") ( + predictionAndObservations: RDD[(Double, Double)]) extends Logging { /** * An auxiliary constructor taking a DataFrame. From ab431f8a970b85fba34ccb506c0f8815e55c63bf Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 20:07:56 -0700 Subject: [PATCH 094/232] [SPARK-10238] [MLLIB] update since versions in mllib.linalg Same as #8421 but for `mllib.linalg`. cc dbtsai Author: Xiangrui Meng Closes #8440 from mengxr/SPARK-10238 and squashes the following commits: b38437e [Xiangrui Meng] update since versions in mllib.linalg --- .../apache/spark/mllib/linalg/Matrices.scala | 44 ++++++++++++------- .../linalg/SingularValueDecomposition.scala | 1 + .../apache/spark/mllib/linalg/Vectors.scala | 25 ++++++++--- .../linalg/distributed/BlockMatrix.scala | 10 +++-- .../linalg/distributed/CoordinateMatrix.scala | 4 +- .../distributed/DistributedMatrix.scala | 2 + .../linalg/distributed/IndexedRowMatrix.scala | 4 +- .../mllib/linalg/distributed/RowMatrix.scala | 5 ++- 8 files changed, 64 insertions(+), 31 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 28b5b4637bf17..c02ba426fcc3a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -32,18 +32,23 @@ import org.apache.spark.sql.types._ * Trait for a local matrix. */ @SQLUserDefinedType(udt = classOf[MatrixUDT]) +@Since("1.0.0") sealed trait Matrix extends Serializable { /** Number of rows. */ + @Since("1.0.0") def numRows: Int /** Number of columns. */ + @Since("1.0.0") def numCols: Int /** Flag that keeps track whether the matrix is transposed or not. False by default. */ + @Since("1.3.0") val isTransposed: Boolean = false /** Converts to a dense array in column major. */ + @Since("1.0.0") def toArray: Array[Double] = { val newArray = new Array[Double](numRows * numCols) foreachActive { (i, j, v) => @@ -56,6 +61,7 @@ sealed trait Matrix extends Serializable { private[mllib] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ + @Since("1.3.0") def apply(i: Int, j: Int): Double /** Return the index for the (i, j)-th element in the backing array. */ @@ -65,12 +71,15 @@ sealed trait Matrix extends Serializable { private[mllib] def update(i: Int, j: Int, v: Double): Unit /** Get a deep copy of the matrix. */ + @Since("1.2.0") def copy: Matrix /** Transpose the Matrix. Returns a new `Matrix` instance sharing the same underlying data. */ + @Since("1.3.0") def transpose: Matrix /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + @Since("1.2.0") def multiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = DenseMatrix.zeros(numRows, y.numCols) BLAS.gemm(1.0, this, y, 0.0, C) @@ -78,11 +87,13 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`-`DenseVector` multiplication. For binary compatibility. */ + @Since("1.2.0") def multiply(y: DenseVector): DenseVector = { multiply(y.asInstanceOf[Vector]) } /** Convenience method for `Matrix`-`Vector` multiplication. */ + @Since("1.4.0") def multiply(y: Vector): DenseVector = { val output = new DenseVector(new Array[Double](numRows)) BLAS.gemv(1.0, this, y, 0.0, output) @@ -93,6 +104,7 @@ sealed trait Matrix extends Serializable { override def toString: String = toBreeze.toString() /** A human readable representation of the matrix with maximum lines and width */ + @Since("1.4.0") def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) /** Map the values of this matrix using a function. Generates a new matrix. Performs the @@ -118,11 +130,13 @@ sealed trait Matrix extends Serializable { /** * Find the number of non-zero active values. */ + @Since("1.5.0") def numNonzeros: Int /** * Find the number of values stored explicitly. These values can be zero as well. */ + @Since("1.5.0") def numActives: Int } @@ -230,11 +244,11 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { */ @Since("1.0.0") @SQLUserDefinedType(udt = classOf[MatrixUDT]) -class DenseMatrix( - val numRows: Int, - val numCols: Int, - val values: Array[Double], - override val isTransposed: Boolean) extends Matrix { +class DenseMatrix @Since("1.3.0") ( + @Since("1.0.0") val numRows: Int, + @Since("1.0.0") val numCols: Int, + @Since("1.0.0") val values: Array[Double], + @Since("1.3.0") override val isTransposed: Boolean) extends Matrix { require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + s"size of the matrix! values.length: ${values.length}, numRows * numCols: ${numRows * numCols}") @@ -254,7 +268,7 @@ class DenseMatrix( * @param numCols number of columns * @param values matrix entries in column major */ - @Since("1.3.0") + @Since("1.0.0") def this(numRows: Int, numCols: Int, values: Array[Double]) = this(numRows, numCols, values, false) @@ -491,13 +505,13 @@ object DenseMatrix { */ @Since("1.2.0") @SQLUserDefinedType(udt = classOf[MatrixUDT]) -class SparseMatrix( - val numRows: Int, - val numCols: Int, - val colPtrs: Array[Int], - val rowIndices: Array[Int], - val values: Array[Double], - override val isTransposed: Boolean) extends Matrix { +class SparseMatrix @Since("1.3.0") ( + @Since("1.2.0") val numRows: Int, + @Since("1.2.0") val numCols: Int, + @Since("1.2.0") val colPtrs: Array[Int], + @Since("1.2.0") val rowIndices: Array[Int], + @Since("1.2.0") val values: Array[Double], + @Since("1.3.0") override val isTransposed: Boolean) extends Matrix { require(values.length == rowIndices.length, "The number of row indices and values don't match! " + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") @@ -527,7 +541,7 @@ class SparseMatrix( * order for each column * @param values non-zero matrix entries in column major */ - @Since("1.3.0") + @Since("1.2.0") def this( numRows: Int, numCols: Int, @@ -549,8 +563,6 @@ class SparseMatrix( } } - /** - */ @Since("1.3.0") override def apply(i: Int, j: Int): Double = { val ind = index(i, j) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index a37aca99d5e72..4dcf8f28f2023 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -31,6 +31,7 @@ case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VTyp * :: Experimental :: * Represents QR factors. */ +@Since("1.5.0") @Experimental case class QRDecomposition[QType, RType](Q: QType, R: RType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 3d577edbe23e1..06ebb15869909 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -38,16 +38,19 @@ import org.apache.spark.sql.types._ * Note: Users should not implement this interface. */ @SQLUserDefinedType(udt = classOf[VectorUDT]) +@Since("1.0.0") sealed trait Vector extends Serializable { /** * Size of the vector. */ + @Since("1.0.0") def size: Int /** * Converts the instance to a double array. */ + @Since("1.0.0") def toArray: Array[Double] override def equals(other: Any): Boolean = { @@ -99,11 +102,13 @@ sealed trait Vector extends Serializable { * Gets the value of the ith element. * @param i index */ + @Since("1.1.0") def apply(i: Int): Double = toBreeze(i) /** * Makes a deep copy of this vector. */ + @Since("1.1.0") def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } @@ -121,26 +126,31 @@ sealed trait Vector extends Serializable { * Number of active entries. An "active entry" is an element which is explicitly stored, * regardless of its value. Note that inactive entries have value 0. */ + @Since("1.4.0") def numActives: Int /** * Number of nonzero elements. This scans all active values and count nonzeros. */ + @Since("1.4.0") def numNonzeros: Int /** * Converts this vector to a sparse vector with all explicit zeros removed. */ + @Since("1.4.0") def toSparse: SparseVector /** * Converts this vector to a dense vector. */ + @Since("1.4.0") def toDense: DenseVector = new DenseVector(this.toArray) /** * Returns a vector in either dense or sparse format, whichever uses less storage. */ + @Since("1.4.0") def compressed: Vector = { val nnz = numNonzeros // A dense vector needs 8 * size + 8 bytes, while a sparse vector needs 12 * nnz + 20 bytes. @@ -155,6 +165,7 @@ sealed trait Vector extends Serializable { * Find the index of a maximal element. Returns the first maximal element in case of a tie. * Returns -1 if vector has length 0. */ + @Since("1.5.0") def argmax: Int } @@ -532,7 +543,8 @@ object Vectors { */ @Since("1.0.0") @SQLUserDefinedType(udt = classOf[VectorUDT]) -class DenseVector(val values: Array[Double]) extends Vector { +class DenseVector @Since("1.0.0") ( + @Since("1.0.0") val values: Array[Double]) extends Vector { @Since("1.0.0") override def size: Int = values.length @@ -632,7 +644,9 @@ class DenseVector(val values: Array[Double]) extends Vector { @Since("1.3.0") object DenseVector { + /** Extracts the value array from a dense vector. */ + @Since("1.3.0") def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) } @@ -645,10 +659,10 @@ object DenseVector { */ @Since("1.0.0") @SQLUserDefinedType(udt = classOf[VectorUDT]) -class SparseVector( - override val size: Int, - val indices: Array[Int], - val values: Array[Double]) extends Vector { +class SparseVector @Since("1.0.0") ( + @Since("1.0.0") override val size: Int, + @Since("1.0.0") val indices: Array[Int], + @Since("1.0.0") val values: Array[Double]) extends Vector { require(indices.length == values.length, "Sparse vectors require that the dimension of the" + s" indices match the dimension of the values. You provided ${indices.length} indices and " + @@ -819,6 +833,7 @@ class SparseVector( @Since("1.3.0") object SparseVector { + @Since("1.3.0") def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] = Some((sv.size, sv.indices, sv.values)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 94376c24a7ac6..a33b6137cf9cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -131,10 +131,10 @@ private[mllib] object GridPartitioner { */ @Since("1.3.0") @Experimental -class BlockMatrix( - val blocks: RDD[((Int, Int), Matrix)], - val rowsPerBlock: Int, - val colsPerBlock: Int, +class BlockMatrix @Since("1.3.0") ( + @Since("1.3.0") val blocks: RDD[((Int, Int), Matrix)], + @Since("1.3.0") val rowsPerBlock: Int, + @Since("1.3.0") val colsPerBlock: Int, private var nRows: Long, private var nCols: Long) extends DistributedMatrix with Logging { @@ -171,7 +171,9 @@ class BlockMatrix( nCols } + @Since("1.3.0") val numRowBlocks = math.ceil(numRows() * 1.0 / rowsPerBlock).toInt + @Since("1.3.0") val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt private[mllib] def createPartitioner(): GridPartitioner = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 4bb27ec840902..644f293d88a75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -46,8 +46,8 @@ case class MatrixEntry(i: Long, j: Long, value: Double) */ @Since("1.0.0") @Experimental -class CoordinateMatrix( - val entries: RDD[MatrixEntry], +class CoordinateMatrix @Since("1.0.0") ( + @Since("1.0.0") val entries: RDD[MatrixEntry], private var nRows: Long, private var nCols: Long) extends DistributedMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index e51327ebb7b58..db3433a5e2456 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -28,9 +28,11 @@ import org.apache.spark.annotation.Since trait DistributedMatrix extends Serializable { /** Gets or computes the number of rows. */ + @Since("1.0.0") def numRows(): Long /** Gets or computes the number of columns. */ + @Since("1.0.0") def numCols(): Long /** Collects data and assembles a local dense breeze matrix (for test only). */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 6d2c05a47d049..b20ea0dc50da5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -45,8 +45,8 @@ case class IndexedRow(index: Long, vector: Vector) */ @Since("1.0.0") @Experimental -class IndexedRowMatrix( - val rows: RDD[IndexedRow], +class IndexedRowMatrix @Since("1.0.0") ( + @Since("1.0.0") val rows: RDD[IndexedRow], private var nRows: Long, private var nCols: Int) extends DistributedMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 78036eba5c3e6..9a423ddafdc09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -47,8 +47,8 @@ import org.apache.spark.storage.StorageLevel */ @Since("1.0.0") @Experimental -class RowMatrix( - val rows: RDD[Vector], +class RowMatrix @Since("1.0.0") ( + @Since("1.0.0") val rows: RDD[Vector], private var nRows: Long, private var nCols: Int) extends DistributedMatrix with Logging { @@ -519,6 +519,7 @@ class RowMatrix( * @param computeQ whether to computeQ * @return QRDecomposition(Q, R), Q = null if computeQ = false. */ + @Since("1.5.0") def tallSkinnyQR(computeQ: Boolean = false): QRDecomposition[RowMatrix, Matrix] = { val col = numCols().toInt // split rows horizontally into smaller matrices, and compute QR for each of them From c3a54843c0c8a14059da4e6716c1ad45c69bbe6c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 22:31:23 -0700 Subject: [PATCH 095/232] [SPARK-10240] [SPARK-10242] [MLLIB] update since versions in mlilb.random and mllib.stat The same as #8241 but for `mllib.stat` and `mllib.random`. cc feynmanliang Author: Xiangrui Meng Closes #8439 from mengxr/SPARK-10242. --- .../mllib/random/RandomDataGenerator.scala | 43 ++++++++++-- .../spark/mllib/random/RandomRDDs.scala | 69 ++++++++++++++++--- .../distribution/MultivariateGaussian.scala | 6 +- .../spark/mllib/stat/test/TestResult.scala | 24 ++++--- 4 files changed, 117 insertions(+), 25 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 9349ecaa13f56..a2d85a68cd327 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.random import org.apache.commons.math3.distribution.{ExponentialDistribution, GammaDistribution, LogNormalDistribution, PoissonDistribution} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Since, DeveloperApi} import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} /** @@ -28,17 +28,20 @@ import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} * Trait for random data generators that generate i.i.d. data. */ @DeveloperApi +@Since("1.1.0") trait RandomDataGenerator[T] extends Pseudorandom with Serializable { /** * Returns an i.i.d. sample as a generic type from an underlying distribution. */ + @Since("1.1.0") def nextValue(): T /** * Returns a copy of the RandomDataGenerator with a new instance of the rng object used in the * class when applicable for non-locking concurrent usage. */ + @Since("1.1.0") def copy(): RandomDataGenerator[T] } @@ -47,17 +50,21 @@ trait RandomDataGenerator[T] extends Pseudorandom with Serializable { * Generates i.i.d. samples from U[0.0, 1.0] */ @DeveloperApi +@Since("1.1.0") class UniformGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. private val random = new XORShiftRandom() + @Since("1.1.0") override def nextValue(): Double = { random.nextDouble() } + @Since("1.1.0") override def setSeed(seed: Long): Unit = random.setSeed(seed) + @Since("1.1.0") override def copy(): UniformGenerator = new UniformGenerator() } @@ -66,17 +73,21 @@ class UniformGenerator extends RandomDataGenerator[Double] { * Generates i.i.d. samples from the standard normal distribution. */ @DeveloperApi +@Since("1.1.0") class StandardNormalGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. private val random = new XORShiftRandom() + @Since("1.1.0") override def nextValue(): Double = { random.nextGaussian() } + @Since("1.1.0") override def setSeed(seed: Long): Unit = random.setSeed(seed) + @Since("1.1.0") override def copy(): StandardNormalGenerator = new StandardNormalGenerator() } @@ -87,16 +98,21 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { * @param mean mean for the Poisson distribution. */ @DeveloperApi -class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { +@Since("1.1.0") +class PoissonGenerator @Since("1.1.0") ( + @Since("1.1.0") val mean: Double) extends RandomDataGenerator[Double] { private val rng = new PoissonDistribution(mean) + @Since("1.1.0") override def nextValue(): Double = rng.sample() + @Since("1.1.0") override def setSeed(seed: Long) { rng.reseedRandomGenerator(seed) } + @Since("1.1.0") override def copy(): PoissonGenerator = new PoissonGenerator(mean) } @@ -107,16 +123,21 @@ class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { * @param mean mean for the exponential distribution. */ @DeveloperApi -class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] { +@Since("1.3.0") +class ExponentialGenerator @Since("1.3.0") ( + @Since("1.3.0") val mean: Double) extends RandomDataGenerator[Double] { private val rng = new ExponentialDistribution(mean) + @Since("1.3.0") override def nextValue(): Double = rng.sample() + @Since("1.3.0") override def setSeed(seed: Long) { rng.reseedRandomGenerator(seed) } + @Since("1.3.0") override def copy(): ExponentialGenerator = new ExponentialGenerator(mean) } @@ -128,16 +149,22 @@ class ExponentialGenerator(val mean: Double) extends RandomDataGenerator[Double] * @param scale scale for the gamma distribution */ @DeveloperApi -class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGenerator[Double] { +@Since("1.3.0") +class GammaGenerator @Since("1.3.0") ( + @Since("1.3.0") val shape: Double, + @Since("1.3.0") val scale: Double) extends RandomDataGenerator[Double] { private val rng = new GammaDistribution(shape, scale) + @Since("1.3.0") override def nextValue(): Double = rng.sample() + @Since("1.3.0") override def setSeed(seed: Long) { rng.reseedRandomGenerator(seed) } + @Since("1.3.0") override def copy(): GammaGenerator = new GammaGenerator(shape, scale) } @@ -150,15 +177,21 @@ class GammaGenerator(val shape: Double, val scale: Double) extends RandomDataGen * @param std standard deviation for the log normal distribution */ @DeveloperApi -class LogNormalGenerator(val mean: Double, val std: Double) extends RandomDataGenerator[Double] { +@Since("1.3.0") +class LogNormalGenerator @Since("1.3.0") ( + @Since("1.3.0") val mean: Double, + @Since("1.3.0") val std: Double) extends RandomDataGenerator[Double] { private val rng = new LogNormalDistribution(mean, std) + @Since("1.3.0") override def nextValue(): Double = rng.sample() + @Since("1.3.0") override def setSeed(seed: Long) { rng.reseedRandomGenerator(seed) } + @Since("1.3.0") override def copy(): LogNormalGenerator = new LogNormalGenerator(mean, std) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index 174d5e0f6c9f0..4dd5ea214d678 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.random import scala.reflect.ClassTag import org.apache.spark.SparkContext -import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD, JavaSparkContext} import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.rdd.{RandomRDD, RandomVectorRDD} @@ -32,6 +32,7 @@ import org.apache.spark.util.Utils * Generator methods for creating RDDs comprised of `i.i.d.` samples from some distribution. */ @Experimental +@Since("1.1.0") object RandomRDDs { /** @@ -46,6 +47,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ `U(0.0, 1.0)`. */ + @Since("1.1.0") def uniformRDD( sc: SparkContext, size: Long, @@ -58,6 +60,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#uniformRDD]]. */ + @Since("1.1.0") def uniformJavaRDD( jsc: JavaSparkContext, size: Long, @@ -69,6 +72,7 @@ object RandomRDDs { /** * [[RandomRDDs#uniformJavaRDD]] with the default seed. */ + @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size, numPartitions)) } @@ -76,6 +80,7 @@ object RandomRDDs { /** * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size)) } @@ -92,6 +97,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ N(0.0, 1.0). */ + @Since("1.1.0") def normalRDD( sc: SparkContext, size: Long, @@ -104,6 +110,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#normalRDD]]. */ + @Since("1.1.0") def normalJavaRDD( jsc: JavaSparkContext, size: Long, @@ -115,6 +122,7 @@ object RandomRDDs { /** * [[RandomRDDs#normalJavaRDD]] with the default seed. */ + @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size, numPartitions)) } @@ -122,6 +130,7 @@ object RandomRDDs { /** * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size)) } @@ -137,6 +146,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ + @Since("1.1.0") def poissonRDD( sc: SparkContext, mean: Double, @@ -150,6 +160,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#poissonRDD]]. */ + @Since("1.1.0") def poissonJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -162,6 +173,7 @@ object RandomRDDs { /** * [[RandomRDDs#poissonJavaRDD]] with the default seed. */ + @Since("1.1.0") def poissonJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -173,6 +185,7 @@ object RandomRDDs { /** * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size)) } @@ -188,6 +201,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ + @Since("1.3.0") def exponentialRDD( sc: SparkContext, mean: Double, @@ -201,6 +215,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#exponentialRDD]]. */ + @Since("1.3.0") def exponentialJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -213,6 +228,7 @@ object RandomRDDs { /** * [[RandomRDDs#exponentialJavaRDD]] with the default seed. */ + @Since("1.3.0") def exponentialJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -224,6 +240,7 @@ object RandomRDDs { /** * [[RandomRDDs#exponentialJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.3.0") def exponentialJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(exponentialRDD(jsc.sc, mean, size)) } @@ -240,6 +257,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ + @Since("1.3.0") def gammaRDD( sc: SparkContext, shape: Double, @@ -254,6 +272,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#gammaRDD]]. */ + @Since("1.3.0") def gammaJavaRDD( jsc: JavaSparkContext, shape: Double, @@ -267,6 +286,7 @@ object RandomRDDs { /** * [[RandomRDDs#gammaJavaRDD]] with the default seed. */ + @Since("1.3.0") def gammaJavaRDD( jsc: JavaSparkContext, shape: Double, @@ -279,11 +299,12 @@ object RandomRDDs { /** * [[RandomRDDs#gammaJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.3.0") def gammaJavaRDD( - jsc: JavaSparkContext, - shape: Double, - scale: Double, - size: Long): JavaDoubleRDD = { + jsc: JavaSparkContext, + shape: Double, + scale: Double, + size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(gammaRDD(jsc.sc, shape, scale, size)) } @@ -299,6 +320,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ + @Since("1.3.0") def logNormalRDD( sc: SparkContext, mean: Double, @@ -313,6 +335,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#logNormalRDD]]. */ + @Since("1.3.0") def logNormalJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -326,6 +349,7 @@ object RandomRDDs { /** * [[RandomRDDs#logNormalJavaRDD]] with the default seed. */ + @Since("1.3.0") def logNormalJavaRDD( jsc: JavaSparkContext, mean: Double, @@ -338,11 +362,12 @@ object RandomRDDs { /** * [[RandomRDDs#logNormalJavaRDD]] with the default number of partitions and the default seed. */ + @Since("1.3.0") def logNormalJavaRDD( - jsc: JavaSparkContext, - mean: Double, - std: Double, - size: Long): JavaDoubleRDD = { + jsc: JavaSparkContext, + mean: Double, + std: Double, + size: Long): JavaDoubleRDD = { JavaDoubleRDD.fromRDD(logNormalRDD(jsc.sc, mean, std, size)) } @@ -359,6 +384,7 @@ object RandomRDDs { * @return RDD[Double] comprised of `i.i.d.` samples produced by generator. */ @DeveloperApi + @Since("1.1.0") def randomRDD[T: ClassTag]( sc: SparkContext, generator: RandomDataGenerator[T], @@ -381,6 +407,7 @@ object RandomRDDs { * @param seed Seed for the RNG that generates the seed for the generator in each partition. * @return RDD[Vector] with vectors containing i.i.d samples ~ `U(0.0, 1.0)`. */ + @Since("1.1.0") def uniformVectorRDD( sc: SparkContext, numRows: Long, @@ -394,6 +421,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#uniformVectorRDD]]. */ + @Since("1.1.0") def uniformJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -406,6 +434,7 @@ object RandomRDDs { /** * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed. */ + @Since("1.1.0") def uniformJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -417,6 +446,7 @@ object RandomRDDs { /** * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def uniformJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -435,6 +465,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ `N(0.0, 1.0)`. */ + @Since("1.1.0") def normalVectorRDD( sc: SparkContext, numRows: Long, @@ -448,6 +479,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#normalVectorRDD]]. */ + @Since("1.1.0") def normalJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -460,6 +492,7 @@ object RandomRDDs { /** * [[RandomRDDs#normalJavaVectorRDD]] with the default seed. */ + @Since("1.1.0") def normalJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -471,6 +504,7 @@ object RandomRDDs { /** * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def normalJavaVectorRDD( jsc: JavaSparkContext, numRows: Long, @@ -491,6 +525,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples. */ + @Since("1.3.0") def logNormalVectorRDD( sc: SparkContext, mean: Double, @@ -507,6 +542,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#logNormalVectorRDD]]. */ + @Since("1.3.0") def logNormalJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -521,6 +557,7 @@ object RandomRDDs { /** * [[RandomRDDs#logNormalJavaVectorRDD]] with the default seed. */ + @Since("1.3.0") def logNormalJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -535,6 +572,7 @@ object RandomRDDs { * [[RandomRDDs#logNormalJavaVectorRDD]] with the default number of partitions and * the default seed. */ + @Since("1.3.0") def logNormalJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -556,6 +594,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Pois(mean). */ + @Since("1.1.0") def poissonVectorRDD( sc: SparkContext, mean: Double, @@ -570,6 +609,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#poissonVectorRDD]]. */ + @Since("1.1.0") def poissonJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -583,6 +623,7 @@ object RandomRDDs { /** * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed. */ + @Since("1.1.0") def poissonJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -595,6 +636,7 @@ object RandomRDDs { /** * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions and the default seed. */ + @Since("1.1.0") def poissonJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -615,6 +657,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Exp(mean). */ + @Since("1.3.0") def exponentialVectorRDD( sc: SparkContext, mean: Double, @@ -630,6 +673,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#exponentialVectorRDD]]. */ + @Since("1.3.0") def exponentialJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -643,6 +687,7 @@ object RandomRDDs { /** * [[RandomRDDs#exponentialJavaVectorRDD]] with the default seed. */ + @Since("1.3.0") def exponentialJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -656,6 +701,7 @@ object RandomRDDs { * [[RandomRDDs#exponentialJavaVectorRDD]] with the default number of partitions * and the default seed. */ + @Since("1.3.0") def exponentialJavaVectorRDD( jsc: JavaSparkContext, mean: Double, @@ -678,6 +724,7 @@ object RandomRDDs { * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Exp(mean). */ + @Since("1.3.0") def gammaVectorRDD( sc: SparkContext, shape: Double, @@ -693,6 +740,7 @@ object RandomRDDs { /** * Java-friendly version of [[RandomRDDs#gammaVectorRDD]]. */ + @Since("1.3.0") def gammaJavaVectorRDD( jsc: JavaSparkContext, shape: Double, @@ -707,6 +755,7 @@ object RandomRDDs { /** * [[RandomRDDs#gammaJavaVectorRDD]] with the default seed. */ + @Since("1.3.0") def gammaJavaVectorRDD( jsc: JavaSparkContext, shape: Double, @@ -720,6 +769,7 @@ object RandomRDDs { /** * [[RandomRDDs#gammaJavaVectorRDD]] with the default number of partitions and the default seed. */ + @Since("1.3.0") def gammaJavaVectorRDD( jsc: JavaSparkContext, shape: Double, @@ -744,6 +794,7 @@ object RandomRDDs { * @return RDD[Vector] with vectors containing `i.i.d.` samples produced by generator. */ @DeveloperApi + @Since("1.1.0") def randomVectorRDD(sc: SparkContext, generator: RandomDataGenerator[Double], numRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala index bd4d81390bfae..92a5af708d04b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala @@ -35,9 +35,9 @@ import org.apache.spark.mllib.util.MLUtils */ @Since("1.3.0") @DeveloperApi -class MultivariateGaussian ( - val mu: Vector, - val sigma: Matrix) extends Serializable { +class MultivariateGaussian @Since("1.3.0") ( + @Since("1.3.0") val mu: Vector, + @Since("1.3.0") val sigma: Matrix) extends Serializable { require(sigma.numCols == sigma.numRows, "Covariance matrix must be square") require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala index f44be13706695..d01b3707be944 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.stat.test -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} /** * :: Experimental :: @@ -25,28 +25,33 @@ import org.apache.spark.annotation.Experimental * @tparam DF Return type of `degreesOfFreedom`. */ @Experimental +@Since("1.1.0") trait TestResult[DF] { /** * The probability of obtaining a test statistic result at least as extreme as the one that was * actually observed, assuming that the null hypothesis is true. */ + @Since("1.1.0") def pValue: Double /** * Returns the degree(s) of freedom of the hypothesis test. * Return type should be Number(e.g. Int, Double) or tuples of Numbers for toString compatibility. */ + @Since("1.1.0") def degreesOfFreedom: DF /** * Test statistic. */ + @Since("1.1.0") def statistic: Double /** * Null hypothesis of the test. */ + @Since("1.1.0") def nullHypothesis: String /** @@ -78,11 +83,12 @@ trait TestResult[DF] { * Object containing the test results for the chi-squared hypothesis test. */ @Experimental +@Since("1.1.0") class ChiSqTestResult private[stat] (override val pValue: Double, - override val degreesOfFreedom: Int, - override val statistic: Double, - val method: String, - override val nullHypothesis: String) extends TestResult[Int] { + @Since("1.1.0") override val degreesOfFreedom: Int, + @Since("1.1.0") override val statistic: Double, + @Since("1.1.0") val method: String, + @Since("1.1.0") override val nullHypothesis: String) extends TestResult[Int] { override def toString: String = { "Chi squared test summary:\n" + @@ -96,11 +102,13 @@ class ChiSqTestResult private[stat] (override val pValue: Double, * Object containing the test results for the Kolmogorov-Smirnov test. */ @Experimental +@Since("1.5.0") class KolmogorovSmirnovTestResult private[stat] ( - override val pValue: Double, - override val statistic: Double, - override val nullHypothesis: String) extends TestResult[Int] { + @Since("1.5.0") override val pValue: Double, + @Since("1.5.0") override val statistic: Double, + @Since("1.5.0") override val nullHypothesis: String) extends TestResult[Int] { + @Since("1.5.0") override val degreesOfFreedom = 0 override def toString: String = { From d703372f86d6a59383ba8569fcd9d379849cffbf Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 22:33:48 -0700 Subject: [PATCH 096/232] [SPARK-10234] [MLLIB] update since version in mllib.clustering Same as #8421 but for `mllib.clustering`. cc feynmanliang yu-iskw Author: Xiangrui Meng Closes #8435 from mengxr/SPARK-10234. --- .../mllib/clustering/GaussianMixture.scala | 1 + .../clustering/GaussianMixtureModel.scala | 8 +++--- .../spark/mllib/clustering/KMeans.scala | 1 + .../spark/mllib/clustering/KMeansModel.scala | 4 +-- .../spark/mllib/clustering/LDAModel.scala | 28 ++++++++++++++----- .../clustering/PowerIterationClustering.scala | 10 +++++-- .../mllib/clustering/StreamingKMeans.scala | 15 +++++----- 7 files changed, 44 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index daa947e81d44d..f82bd82c20371 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -53,6 +53,7 @@ import org.apache.spark.util.Utils * @param maxIterations The maximum number of iterations to perform */ @Experimental +@Since("1.3.0") class GaussianMixture private ( private var k: Int, private var convergenceTol: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 1a10a8b624218..7f6163e04bf17 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -46,9 +46,9 @@ import org.apache.spark.sql.{SQLContext, Row} */ @Since("1.3.0") @Experimental -class GaussianMixtureModel( - val weights: Array[Double], - val gaussians: Array[MultivariateGaussian]) extends Serializable with Saveable { +class GaussianMixtureModel @Since("1.3.0") ( + @Since("1.3.0") val weights: Array[Double], + @Since("1.3.0") val gaussians: Array[MultivariateGaussian]) extends Serializable with Saveable { require(weights.length == gaussians.length, "Length of weight and Gaussian arrays must match") @@ -178,7 +178,7 @@ object GaussianMixtureModel extends Loader[GaussianMixtureModel] { (weight, new MultivariateGaussian(mu, sigma)) }.unzip - return new GaussianMixtureModel(weights.toArray, gaussians.toArray) + new GaussianMixtureModel(weights.toArray, gaussians.toArray) } } 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 3e9545a74bef3..46920fffe6e1a 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 @@ -37,6 +37,7 @@ import org.apache.spark.util.random.XORShiftRandom * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given * to it should be cached by the user. */ +@Since("0.8.0") class KMeans private ( private var k: Int, private var maxIterations: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index e425ecdd481c6..a741584982725 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -37,8 +37,8 @@ import org.apache.spark.sql.Row * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ @Since("0.8.0") -class KMeansModel ( - val clusterCenters: Array[Vector]) extends Saveable with Serializable with PMMLExportable { +class KMeansModel @Since("1.1.0") (@Since("1.0.0") val clusterCenters: Array[Vector]) + extends Saveable with Serializable with PMMLExportable { /** * A Java-friendly constructor that takes an Iterable of Vectors. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 432bbedc8d6f8..15129e0dd5a91 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -43,12 +43,15 @@ import org.apache.spark.util.BoundedPriorityQueue * including local and distributed data structures. */ @Experimental +@Since("1.3.0") abstract class LDAModel private[clustering] extends Saveable { /** Number of topics */ + @Since("1.3.0") def k: Int /** Vocabulary size (number of terms or terms in the vocabulary) */ + @Since("1.3.0") def vocabSize: Int /** @@ -57,6 +60,7 @@ abstract class LDAModel private[clustering] extends Saveable { * * This is the parameter to a Dirichlet distribution. */ + @Since("1.5.0") def docConcentration: Vector /** @@ -68,6 +72,7 @@ abstract class LDAModel private[clustering] extends Saveable { * Note: The topics' distributions over terms are called "beta" in the original LDA paper * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. */ + @Since("1.5.0") def topicConcentration: Double /** @@ -81,6 +86,7 @@ abstract class LDAModel private[clustering] extends Saveable { * This is a matrix of size vocabSize x k, where each column is a topic. * No guarantees are given about the ordering of the topics. */ + @Since("1.3.0") def topicsMatrix: Matrix /** @@ -91,6 +97,7 @@ abstract class LDAModel private[clustering] extends Saveable { * (term indices, term weights in topic). * Each topic's terms are sorted in order of decreasing weight. */ + @Since("1.3.0") def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] /** @@ -102,6 +109,7 @@ abstract class LDAModel private[clustering] extends Saveable { * (term indices, term weights in topic). * Each topic's terms are sorted in order of decreasing weight. */ + @Since("1.3.0") def describeTopics(): Array[(Array[Int], Array[Double])] = describeTopics(vocabSize) /* TODO (once LDA can be trained with Strings or given a dictionary) @@ -185,10 +193,11 @@ abstract class LDAModel private[clustering] extends Saveable { * @param topics Inferred topics (vocabSize x k matrix). */ @Experimental +@Since("1.3.0") class LocalLDAModel private[clustering] ( - val topics: Matrix, - override val docConcentration: Vector, - override val topicConcentration: Double, + @Since("1.3.0") val topics: Matrix, + @Since("1.5.0") override val docConcentration: Vector, + @Since("1.5.0") override val topicConcentration: Double, override protected[clustering] val gammaShape: Double = 100) extends LDAModel with Serializable { @@ -376,6 +385,7 @@ class LocalLDAModel private[clustering] ( } @Experimental +@Since("1.5.0") object LocalLDAModel extends Loader[LocalLDAModel] { private object SaveLoadV1_0 { @@ -479,13 +489,14 @@ object LocalLDAModel extends Loader[LocalLDAModel] { * than the [[LocalLDAModel]]. */ @Experimental +@Since("1.3.0") class DistributedLDAModel private[clustering] ( private[clustering] val graph: Graph[LDA.TopicCounts, LDA.TokenCount], private[clustering] val globalTopicTotals: LDA.TopicCounts, - val k: Int, - val vocabSize: Int, - override val docConcentration: Vector, - override val topicConcentration: Double, + @Since("1.3.0") val k: Int, + @Since("1.3.0") val vocabSize: Int, + @Since("1.5.0") override val docConcentration: Vector, + @Since("1.5.0") override val topicConcentration: Double, private[spark] val iterationTimes: Array[Double], override protected[clustering] val gammaShape: Double = 100) extends LDAModel { @@ -603,6 +614,7 @@ class DistributedLDAModel private[clustering] ( * (term indices, topic indices). Note that terms will be omitted if not present in * the document. */ + @Since("1.5.0") lazy val topicAssignments: RDD[(Long, Array[Int], Array[Int])] = { // For reference, compare the below code with the core part of EMLDAOptimizer.next(). val eta = topicConcentration @@ -634,6 +646,7 @@ class DistributedLDAModel private[clustering] ( } /** Java-friendly version of [[topicAssignments]] */ + @Since("1.5.0") lazy val javaTopicAssignments: JavaRDD[(java.lang.Long, Array[Int], Array[Int])] = { topicAssignments.asInstanceOf[RDD[(java.lang.Long, Array[Int], Array[Int])]].toJavaRDD() } @@ -770,6 +783,7 @@ class DistributedLDAModel private[clustering] ( @Experimental +@Since("1.5.0") object DistributedLDAModel extends Loader[DistributedLDAModel] { private object SaveLoadV1_0 { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 396b36f2f6454..da234bdbb29e6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -42,9 +42,10 @@ import org.apache.spark.{Logging, SparkContext, SparkException} */ @Since("1.3.0") @Experimental -class PowerIterationClusteringModel( - val k: Int, - val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { +class PowerIterationClusteringModel @Since("1.3.0") ( + @Since("1.3.0") val k: Int, + @Since("1.3.0") val assignments: RDD[PowerIterationClustering.Assignment]) + extends Saveable with Serializable { @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { @@ -56,6 +57,8 @@ class PowerIterationClusteringModel( @Since("1.4.0") object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { + + @Since("1.4.0") override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) } @@ -120,6 +123,7 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode * @see [[http://en.wikipedia.org/wiki/Spectral_clustering Spectral clustering (Wikipedia)]] */ @Experimental +@Since("1.3.0") class PowerIterationClustering private[clustering] ( private var k: Int, private var maxIterations: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 41f2668ec6a7d..1d50ffec96faf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -66,9 +66,10 @@ import org.apache.spark.util.random.XORShiftRandom */ @Since("1.2.0") @Experimental -class StreamingKMeansModel( - override val clusterCenters: Array[Vector], - val clusterWeights: Array[Double]) extends KMeansModel(clusterCenters) with Logging { +class StreamingKMeansModel @Since("1.2.0") ( + @Since("1.2.0") override val clusterCenters: Array[Vector], + @Since("1.2.0") val clusterWeights: Array[Double]) + extends KMeansModel(clusterCenters) with Logging { /** * Perform a k-means update on a batch of data. @@ -168,10 +169,10 @@ class StreamingKMeansModel( */ @Since("1.2.0") @Experimental -class StreamingKMeans( - var k: Int, - var decayFactor: Double, - var timeUnit: String) extends Logging with Serializable { +class StreamingKMeans @Since("1.2.0") ( + @Since("1.2.0") var k: Int, + @Since("1.2.0") var decayFactor: Double, + @Since("1.2.0") var timeUnit: String) extends Logging with Serializable { @Since("1.2.0") def this() = this(2, 1.0, StreamingKMeans.BATCHES) From fb7e12fe2e14af8de4c206ca8096b2e8113bfddc Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 22:35:49 -0700 Subject: [PATCH 097/232] [SPARK-10243] [MLLIB] update since versions in mllib.tree Same as #8421 but for `mllib.tree`. cc jkbradley Author: Xiangrui Meng Closes #8442 from mengxr/SPARK-10236. --- .../spark/mllib/tree/DecisionTree.scala | 3 +- .../mllib/tree/GradientBoostedTrees.scala | 2 +- .../spark/mllib/tree/configuration/Algo.scala | 2 ++ .../tree/configuration/BoostingStrategy.scala | 12 ++++---- .../tree/configuration/FeatureType.scala | 2 ++ .../tree/configuration/QuantileStrategy.scala | 2 ++ .../mllib/tree/configuration/Strategy.scala | 29 ++++++++++--------- .../mllib/tree/model/DecisionTreeModel.scala | 5 +++- .../apache/spark/mllib/tree/model/Node.scala | 18 ++++++------ .../spark/mllib/tree/model/Predict.scala | 6 ++-- .../apache/spark/mllib/tree/model/Split.scala | 8 ++--- .../mllib/tree/model/treeEnsembleModels.scala | 12 ++++---- 12 files changed, 57 insertions(+), 44 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 972841015d4f0..4a77d4adcd865 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -46,7 +46,8 @@ import org.apache.spark.util.random.XORShiftRandom */ @Since("1.0.0") @Experimental -class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { +class DecisionTree @Since("1.0.0") (private val strategy: Strategy) + extends Serializable with Logging { strategy.assertValid() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index e750408600c33..95ed48cea6716 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -51,7 +51,7 @@ import org.apache.spark.storage.StorageLevel */ @Since("1.2.0") @Experimental -class GradientBoostedTrees(private val boostingStrategy: BoostingStrategy) +class GradientBoostedTrees @Since("1.2.0") (private val boostingStrategy: BoostingStrategy) extends Serializable with Logging { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 8301ad160836b..853c7319ec44d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -26,7 +26,9 @@ import org.apache.spark.annotation.{Experimental, Since} @Since("1.0.0") @Experimental object Algo extends Enumeration { + @Since("1.0.0") type Algo = Value + @Since("1.0.0") val Classification, Regression = Value private[mllib] def fromString(name: String): Algo = name match { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 7c569981977b4..b5c72fba3ede1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -41,14 +41,14 @@ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} */ @Since("1.2.0") @Experimental -case class BoostingStrategy( +case class BoostingStrategy @Since("1.4.0") ( // Required boosting parameters - @BeanProperty var treeStrategy: Strategy, - @BeanProperty var loss: Loss, + @Since("1.2.0") @BeanProperty var treeStrategy: Strategy, + @Since("1.2.0") @BeanProperty var loss: Loss, // Optional boosting parameters - @BeanProperty var numIterations: Int = 100, - @BeanProperty var learningRate: Double = 0.1, - @BeanProperty var validationTol: Double = 1e-5) extends Serializable { + @Since("1.2.0") @BeanProperty var numIterations: Int = 100, + @Since("1.2.0") @BeanProperty var learningRate: Double = 0.1, + @Since("1.4.0") @BeanProperty var validationTol: Double = 1e-5) extends Serializable { /** * Check validity of parameters. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index bb7c7ee4f964f..4e0cd473def06 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -26,6 +26,8 @@ import org.apache.spark.annotation.{Experimental, Since} @Since("1.0.0") @Experimental object FeatureType extends Enumeration { + @Since("1.0.0") type FeatureType = Value + @Since("1.0.0") val Continuous, Categorical = Value } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index 904e42deebb5f..8262db8a4f111 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -26,6 +26,8 @@ import org.apache.spark.annotation.{Experimental, Since} @Since("1.0.0") @Experimental object QuantileStrategy extends Enumeration { + @Since("1.0.0") type QuantileStrategy = Value + @Since("1.0.0") val Sort, MinMax, ApproxHist = Value } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index b74e3f1f46523..89cc13b7c06cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -69,20 +69,20 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ */ @Since("1.0.0") @Experimental -class Strategy ( - @BeanProperty var algo: Algo, - @BeanProperty var impurity: Impurity, - @BeanProperty var maxDepth: Int, - @BeanProperty var numClasses: Int = 2, - @BeanProperty var maxBins: Int = 32, - @BeanProperty var quantileCalculationStrategy: QuantileStrategy = Sort, - @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - @BeanProperty var minInstancesPerNode: Int = 1, - @BeanProperty var minInfoGain: Double = 0.0, - @BeanProperty var maxMemoryInMB: Int = 256, - @BeanProperty var subsamplingRate: Double = 1, - @BeanProperty var useNodeIdCache: Boolean = false, - @BeanProperty var checkpointInterval: Int = 10) extends Serializable { +class Strategy @Since("1.3.0") ( + @Since("1.0.0") @BeanProperty var algo: Algo, + @Since("1.0.0") @BeanProperty var impurity: Impurity, + @Since("1.0.0") @BeanProperty var maxDepth: Int, + @Since("1.2.0") @BeanProperty var numClasses: Int = 2, + @Since("1.0.0") @BeanProperty var maxBins: Int = 32, + @Since("1.0.0") @BeanProperty var quantileCalculationStrategy: QuantileStrategy = Sort, + @Since("1.0.0") @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + @Since("1.2.0") @BeanProperty var minInstancesPerNode: Int = 1, + @Since("1.2.0") @BeanProperty var minInfoGain: Double = 0.0, + @Since("1.0.0") @BeanProperty var maxMemoryInMB: Int = 256, + @Since("1.2.0") @BeanProperty var subsamplingRate: Double = 1, + @Since("1.2.0") @BeanProperty var useNodeIdCache: Boolean = false, + @Since("1.2.0") @BeanProperty var checkpointInterval: Int = 10) extends Serializable { /** */ @@ -206,6 +206,7 @@ object Strategy { } @deprecated("Use Strategy.defaultStrategy instead.", "1.5.0") + @Since("1.2.0") def defaultStategy(algo: Algo): Strategy = defaultStrategy(algo) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 3eefd135f7836..e1bf23f4c34bb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -43,7 +43,9 @@ import org.apache.spark.util.Utils */ @Since("1.0.0") @Experimental -class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable with Saveable { +class DecisionTreeModel @Since("1.0.0") ( + @Since("1.0.0") val topNode: Node, + @Since("1.0.0") val algo: Algo) extends Serializable with Saveable { /** * Predict values for a single data point using the model trained. @@ -110,6 +112,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable /** * Print the full model to a string. */ + @Since("1.2.0") def toDebugString: String = { val header = toString + "\n" header + topNode.subtreeToString(2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 8c54c55107233..ea6e5aa5d94e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -41,15 +41,15 @@ import org.apache.spark.mllib.linalg.Vector */ @Since("1.0.0") @DeveloperApi -class Node ( - val id: Int, - var predict: Predict, - var impurity: Double, - var isLeaf: Boolean, - var split: Option[Split], - var leftNode: Option[Node], - var rightNode: Option[Node], - var stats: Option[InformationGainStats]) extends Serializable with Logging { +class Node @Since("1.2.0") ( + @Since("1.0.0") val id: Int, + @Since("1.0.0") var predict: Predict, + @Since("1.2.0") var impurity: Double, + @Since("1.0.0") var isLeaf: Boolean, + @Since("1.0.0") var split: Option[Split], + @Since("1.0.0") var leftNode: Option[Node], + @Since("1.0.0") var rightNode: Option[Node], + @Since("1.0.0") var stats: Option[InformationGainStats]) extends Serializable with Logging { override def toString: String = { s"id = $id, isLeaf = $isLeaf, predict = $predict, impurity = $impurity, " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 965784051ede5..06ceff19d8633 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -26,9 +26,9 @@ import org.apache.spark.annotation.{DeveloperApi, Since} */ @Since("1.2.0") @DeveloperApi -class Predict( - val predict: Double, - val prob: Double = 0.0) extends Serializable { +class Predict @Since("1.2.0") ( + @Since("1.2.0") val predict: Double, + @Since("1.2.0") val prob: Double = 0.0) extends Serializable { override def toString: String = s"$predict (prob = $prob)" diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 45db83ae3a1f3..b85a66c05a81d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -34,10 +34,10 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType @Since("1.0.0") @DeveloperApi case class Split( - feature: Int, - threshold: Double, - featureType: FeatureType, - categories: List[Double]) { + @Since("1.0.0") feature: Int, + @Since("1.0.0") threshold: Double, + @Since("1.0.0") featureType: FeatureType, + @Since("1.0.0") categories: List[Double]) { override def toString: String = { s"Feature = $feature, threshold = $threshold, featureType = $featureType, " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 19571447a2c56..df5b8feab5d5d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -48,7 +48,9 @@ import org.apache.spark.util.Utils */ @Since("1.2.0") @Experimental -class RandomForestModel(override val algo: Algo, override val trees: Array[DecisionTreeModel]) +class RandomForestModel @Since("1.2.0") ( + @Since("1.2.0") override val algo: Algo, + @Since("1.2.0") override val trees: Array[DecisionTreeModel]) extends TreeEnsembleModel(algo, trees, Array.fill(trees.length)(1.0), combiningStrategy = if (algo == Classification) Vote else Average) with Saveable { @@ -115,10 +117,10 @@ object RandomForestModel extends Loader[RandomForestModel] { */ @Since("1.2.0") @Experimental -class GradientBoostedTreesModel( - override val algo: Algo, - override val trees: Array[DecisionTreeModel], - override val treeWeights: Array[Double]) +class GradientBoostedTreesModel @Since("1.2.0") ( + @Since("1.2.0") override val algo: Algo, + @Since("1.2.0") override val trees: Array[DecisionTreeModel], + @Since("1.2.0") override val treeWeights: Array[Double]) extends TreeEnsembleModel(algo, trees, treeWeights, combiningStrategy = Sum) with Saveable { From 4657fa1f37d41dd4c7240a960342b68c7c591f48 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 22:49:33 -0700 Subject: [PATCH 098/232] [SPARK-10235] [MLLIB] update since versions in mllib.regression Same as #8421 but for `mllib.regression`. cc freeman-lab dbtsai Author: Xiangrui Meng Closes #8426 from mengxr/SPARK-10235 and squashes the following commits: 6cd28e4 [Xiangrui Meng] update since versions in mllib.regression --- .../regression/GeneralizedLinearAlgorithm.scala | 6 ++++-- .../mllib/regression/IsotonicRegression.scala | 16 +++++++++------- .../spark/mllib/regression/LabeledPoint.scala | 5 +++-- .../apache/spark/mllib/regression/Lasso.scala | 9 ++++++--- .../mllib/regression/LinearRegression.scala | 9 ++++++--- .../spark/mllib/regression/RidgeRegression.scala | 12 +++++++----- .../regression/StreamingLinearAlgorithm.scala | 8 +++----- .../StreamingLinearRegressionWithSGD.scala | 11 +++++++++-- 8 files changed, 47 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 509f6a2d169c4..7e3b4d5648fe3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -38,7 +38,9 @@ import org.apache.spark.storage.StorageLevel */ @Since("0.8.0") @DeveloperApi -abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) +abstract class GeneralizedLinearModel @Since("1.0.0") ( + @Since("1.0.0") val weights: Vector, + @Since("0.8.0") val intercept: Double) extends Serializable { /** @@ -107,7 +109,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * The optimizer to solve the problem. * */ - @Since("1.0.0") + @Since("0.8.0") def optimizer: Optimizer /** Whether to add intercept (default: false). */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 31ca7c2f207d9..877d31ba41303 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -50,10 +50,10 @@ import org.apache.spark.sql.SQLContext */ @Since("1.3.0") @Experimental -class IsotonicRegressionModel ( - val boundaries: Array[Double], - val predictions: Array[Double], - val isotonic: Boolean) extends Serializable with Saveable { +class IsotonicRegressionModel @Since("1.3.0") ( + @Since("1.3.0") val boundaries: Array[Double], + @Since("1.3.0") val predictions: Array[Double], + @Since("1.3.0") val isotonic: Boolean) extends Serializable with Saveable { private val predictionOrd = if (isotonic) Ordering[Double] else Ordering[Double].reverse @@ -63,7 +63,6 @@ class IsotonicRegressionModel ( /** * A Java-friendly constructor that takes two Iterable parameters and one Boolean parameter. - * */ @Since("1.4.0") def this(boundaries: java.lang.Iterable[Double], @@ -214,8 +213,6 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { } } - /** - */ @Since("1.4.0") override def load(sc: SparkContext, path: String): IsotonicRegressionModel = { implicit val formats = DefaultFormats @@ -256,6 +253,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { * @see [[http://en.wikipedia.org/wiki/Isotonic_regression Isotonic regression (Wikipedia)]] */ @Experimental +@Since("1.3.0") class IsotonicRegression private (private var isotonic: Boolean) extends Serializable { /** @@ -263,6 +261,7 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali * * @return New instance of IsotonicRegression. */ + @Since("1.3.0") def this() = this(true) /** @@ -271,6 +270,7 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali * @param isotonic Isotonic (increasing) or antitonic (decreasing) sequence. * @return This instance of IsotonicRegression. */ + @Since("1.3.0") def setIsotonic(isotonic: Boolean): this.type = { this.isotonic = isotonic this @@ -286,6 +286,7 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali * the algorithm is executed. * @return Isotonic regression model. */ + @Since("1.3.0") def run(input: RDD[(Double, Double, Double)]): IsotonicRegressionModel = { val preprocessedInput = if (isotonic) { input @@ -311,6 +312,7 @@ class IsotonicRegression private (private var isotonic: Boolean) extends Seriali * the algorithm is executed. * @return Isotonic regression model. */ + @Since("1.3.0") def run(input: JavaRDD[(JDouble, JDouble, JDouble)]): IsotonicRegressionModel = { run(input.rdd.retag.asInstanceOf[RDD[(Double, Double, Double)]]) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index f7fe1b7b21fca..c284ad2325374 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -29,11 +29,12 @@ import org.apache.spark.SparkException * * @param label Label for this data point. * @param features List of features for this data point. - * */ @Since("0.8.0") @BeanInfo -case class LabeledPoint(label: Double, features: Vector) { +case class LabeledPoint @Since("1.0.0") ( + @Since("0.8.0") label: Double, + @Since("1.0.0") features: Vector) { override def toString: String = { s"($label,$features)" } 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 556411a366bd2..a9aba173fa0e3 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 @@ -34,9 +34,9 @@ import org.apache.spark.rdd.RDD * */ @Since("0.8.0") -class LassoModel ( - override val weights: Vector, - override val intercept: Double) +class LassoModel @Since("1.1.0") ( + @Since("1.0.0") override val weights: Vector, + @Since("0.8.0") override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable with Saveable with PMMLExportable { @@ -84,6 +84,7 @@ object LassoModel extends Loader[LassoModel] { * its corresponding right hand side label y. * See also the documentation for the precise formulation. */ +@Since("0.8.0") class LassoWithSGD private ( private var stepSize: Double, private var numIterations: Int, @@ -93,6 +94,7 @@ class LassoWithSGD private ( private val gradient = new LeastSquaresGradient() private val updater = new L1Updater() + @Since("0.8.0") override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) @@ -103,6 +105,7 @@ class LassoWithSGD private ( * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 0.01, miniBatchFraction: 1.0}. */ + @Since("0.8.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { 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 00ab06e3ba738..4996ace5df85d 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 @@ -34,9 +34,9 @@ import org.apache.spark.rdd.RDD * */ @Since("0.8.0") -class LinearRegressionModel ( - override val weights: Vector, - override val intercept: Double) +class LinearRegressionModel @Since("1.1.0") ( + @Since("1.0.0") override val weights: Vector, + @Since("0.8.0") override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable with Saveable with PMMLExportable { @@ -85,6 +85,7 @@ object LinearRegressionModel extends Loader[LinearRegressionModel] { * its corresponding right hand side label y. * See also the documentation for the precise formulation. */ +@Since("0.8.0") class LinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -93,6 +94,7 @@ class LinearRegressionWithSGD private[mllib] ( private val gradient = new LeastSquaresGradient() private val updater = new SimpleUpdater() + @Since("0.8.0") override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) @@ -102,6 +104,7 @@ class LinearRegressionWithSGD private[mllib] ( * Construct a LinearRegression object with default parameters: {stepSize: 1.0, * numIterations: 100, miniBatchFraction: 1.0}. */ + @Since("0.8.0") def this() = this(1.0, 100, 1.0) override protected[mllib] def createModel(weights: Vector, intercept: Double) = { 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 21a791d98b2cb..0a44ff559d55b 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 @@ -35,9 +35,9 @@ import org.apache.spark.rdd.RDD * */ @Since("0.8.0") -class RidgeRegressionModel ( - override val weights: Vector, - override val intercept: Double) +class RidgeRegressionModel @Since("1.1.0") ( + @Since("1.0.0") override val weights: Vector, + @Since("0.8.0") override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable with Saveable with PMMLExportable { @@ -85,6 +85,7 @@ object RidgeRegressionModel extends Loader[RidgeRegressionModel] { * its corresponding right hand side label y. * See also the documentation for the precise formulation. */ +@Since("0.8.0") class RidgeRegressionWithSGD private ( private var stepSize: Double, private var numIterations: Int, @@ -94,7 +95,7 @@ class RidgeRegressionWithSGD private ( private val gradient = new LeastSquaresGradient() private val updater = new SquaredL2Updater() - + @Since("0.8.0") override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) @@ -105,6 +106,7 @@ class RidgeRegressionWithSGD private ( * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 0.01, miniBatchFraction: 1.0}. */ + @Since("0.8.0") def this() = this(1.0, 100, 0.01, 1.0) override protected def createModel(weights: Vector, intercept: Double) = { @@ -134,7 +136,7 @@ object RidgeRegressionWithSGD { * the number of features in the data. * */ - @Since("0.8.0") + @Since("1.0.0") def train( input: RDD[LabeledPoint], numIterations: Int, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index cd3ed8a1549db..73948b2d9851a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream @@ -83,9 +83,8 @@ abstract class StreamingLinearAlgorithm[ * batch of data from the stream. * * @param data DStream containing labeled data - * */ - @Since("1.3.0") + @Since("1.1.0") def trainOn(data: DStream[LabeledPoint]): Unit = { if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting training.") @@ -105,7 +104,6 @@ abstract class StreamingLinearAlgorithm[ /** * Java-friendly version of `trainOn`. - * */ @Since("1.3.0") def trainOn(data: JavaDStream[LabeledPoint]): Unit = trainOn(data.dstream) @@ -129,7 +127,7 @@ abstract class StreamingLinearAlgorithm[ * Java-friendly version of `predictOn`. * */ - @Since("1.1.0") + @Since("1.3.0") def predictOn(data: JavaDStream[Vector]): JavaDStream[java.lang.Double] = { JavaDStream.fromDStream(predictOn(data.dstream).asInstanceOf[DStream[java.lang.Double]]) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index 26654e4a06838..fe1d487cdd078 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.mllib.linalg.Vector /** @@ -41,6 +41,7 @@ import org.apache.spark.mllib.linalg.Vector * .trainOn(DStream) */ @Experimental +@Since("1.1.0") class StreamingLinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, @@ -54,8 +55,10 @@ class StreamingLinearRegressionWithSGD private[mllib] ( * Initial weights must be set before using trainOn or predictOn * (see `StreamingLinearAlgorithm`) */ + @Since("1.1.0") def this() = this(0.1, 50, 1.0) + @Since("1.1.0") val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) protected var model: Option[LinearRegressionModel] = None @@ -63,6 +66,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** * Set the step size for gradient descent. Default: 0.1. */ + @Since("1.1.0") def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) this @@ -71,6 +75,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** * Set the number of iterations of gradient descent to run per update. Default: 50. */ + @Since("1.1.0") def setNumIterations(numIterations: Int): this.type = { this.algorithm.optimizer.setNumIterations(numIterations) this @@ -79,6 +84,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** * Set the fraction of each batch to use for updates. Default: 1.0. */ + @Since("1.1.0") def setMiniBatchFraction(miniBatchFraction: Double): this.type = { this.algorithm.optimizer.setMiniBatchFraction(miniBatchFraction) this @@ -87,6 +93,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** * Set the initial weights. */ + @Since("1.1.0") def setInitialWeights(initialWeights: Vector): this.type = { this.model = Some(algorithm.createModel(initialWeights, 0.0)) this @@ -95,9 +102,9 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** * Set the convergence tolerance. Default: 0.001. */ + @Since("1.5.0") def setConvergenceTol(tolerance: Double): this.type = { this.algorithm.optimizer.setConvergenceTol(tolerance) this } - } From 321d7759691bed9867b1f0470f12eab2faa50aff Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Aug 2015 23:45:41 -0700 Subject: [PATCH 099/232] [SPARK-10236] [MLLIB] update since versions in mllib.feature Same as #8421 but for `mllib.feature`. cc dbtsai Author: Xiangrui Meng Closes #8449 from mengxr/SPARK-10236.feature and squashes the following commits: 0e8d658 [Xiangrui Meng] remove unnecessary comment ad70b03 [Xiangrui Meng] update since versions in mllib.feature --- .../mllib/clustering/PowerIterationClustering.scala | 2 -- .../apache/spark/mllib/feature/ChiSqSelector.scala | 4 ++-- .../spark/mllib/feature/ElementwiseProduct.scala | 3 ++- .../scala/org/apache/spark/mllib/feature/IDF.scala | 6 ++++-- .../org/apache/spark/mllib/feature/Normalizer.scala | 2 +- .../scala/org/apache/spark/mllib/feature/PCA.scala | 7 +++++-- .../apache/spark/mllib/feature/StandardScaler.scala | 12 ++++++------ .../org/apache/spark/mllib/feature/Word2Vec.scala | 1 + 8 files changed, 21 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index da234bdbb29e6..6c76e26fd1626 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -71,8 +71,6 @@ object PowerIterationClusteringModel extends Loader[PowerIterationClusteringMode private[clustering] val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" - /** - */ @Since("1.4.0") def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { val sqlContext = new SQLContext(sc) 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 fdd974d7a391e..4743cfd1a2c3f 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 @@ -33,7 +33,7 @@ import org.apache.spark.rdd.RDD */ @Since("1.3.0") @Experimental -class ChiSqSelectorModel ( +class ChiSqSelectorModel @Since("1.3.0") ( @Since("1.3.0") val selectedFeatures: Array[Int]) extends VectorTransformer { require(isSorted(selectedFeatures), "Array has to be sorted asc") @@ -112,7 +112,7 @@ class ChiSqSelectorModel ( */ @Since("1.3.0") @Experimental -class ChiSqSelector ( +class ChiSqSelector @Since("1.3.0") ( @Since("1.3.0") val numTopFeatures: Int) extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala index 33e2d17bb472e..d0a6cf61687a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala @@ -29,7 +29,8 @@ import org.apache.spark.mllib.linalg._ */ @Since("1.4.0") @Experimental -class ElementwiseProduct(val scalingVec: Vector) extends VectorTransformer { +class ElementwiseProduct @Since("1.4.0") ( + @Since("1.4.0") val scalingVec: Vector) extends VectorTransformer { /** * Does the hadamard product transformation. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index d5353ddd972e0..68078ccfa3d60 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -39,8 +39,9 @@ import org.apache.spark.rdd.RDD */ @Since("1.1.0") @Experimental -class IDF(val minDocFreq: Int) { +class IDF @Since("1.2.0") (@Since("1.2.0") val minDocFreq: Int) { + @Since("1.1.0") def this() = this(0) // TODO: Allow different IDF formulations. @@ -162,7 +163,8 @@ private object IDF { * Represents an IDF model that can transform term frequency vectors. */ @Experimental -class IDFModel private[spark] (val idf: Vector) extends Serializable { +@Since("1.1.0") +class IDFModel private[spark] (@Since("1.1.0") val idf: Vector) extends Serializable { /** * Transforms term frequency (TF) vectors to TF-IDF vectors. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index 0e070257d9fb2..8d5a22520d6b8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -33,7 +33,7 @@ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors */ @Since("1.1.0") @Experimental -class Normalizer(p: Double) extends VectorTransformer { +class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer { @Since("1.1.0") def this() = this(2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index a48b7bba665d7..ecb3c1e6c1c83 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD * @param k number of principal components */ @Since("1.4.0") -class PCA(val k: Int) { +class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { require(k >= 1, s"PCA requires a number of principal components k >= 1 but was given $k") /** @@ -74,7 +74,10 @@ class PCA(val k: Int) { * @param k number of principal components. * @param pc a principal components Matrix. Each column is one principal component. */ -class PCAModel private[spark] (val k: Int, val pc: DenseMatrix) extends VectorTransformer { +@Since("1.4.0") +class PCAModel private[spark] ( + @Since("1.4.0") val k: Int, + @Since("1.4.0") val pc: DenseMatrix) extends VectorTransformer { /** * Transform a vector by computed Principal Components. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index b95d5a899001e..f018b453bae7e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -34,7 +34,7 @@ import org.apache.spark.rdd.RDD */ @Since("1.1.0") @Experimental -class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { +class StandardScaler @Since("1.1.0") (withMean: Boolean, withStd: Boolean) extends Logging { @Since("1.1.0") def this() = this(false, true) @@ -74,11 +74,11 @@ class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { */ @Since("1.1.0") @Experimental -class StandardScalerModel ( - val std: Vector, - val mean: Vector, - var withStd: Boolean, - var withMean: Boolean) extends VectorTransformer { +class StandardScalerModel @Since("1.3.0") ( + @Since("1.3.0") val std: Vector, + @Since("1.1.0") val mean: Vector, + @Since("1.3.0") var withStd: Boolean, + @Since("1.3.0") var withMean: Boolean) extends VectorTransformer { /** */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index e6f45ae4b01d5..36b124c5d2966 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -436,6 +436,7 @@ class Word2Vec extends Serializable with Logging { * (i * vectorSize, i * vectorSize + vectorSize) */ @Experimental +@Since("1.1.0") class Word2VecModel private[mllib] ( private val wordIndex: Map[String, Int], private val wordVectors: Array[Float]) extends Serializable with Saveable { From 75d4773aa50e24972c533e8b48697fde586429eb Mon Sep 17 00:00:00 2001 From: felixcheung Date: Tue, 25 Aug 2015 23:48:16 -0700 Subject: [PATCH 100/232] [SPARK-9316] [SPARKR] Add support for filtering using `[` (synonym for filter / select) Add support for ``` df[df$name == "Smith", c(1,2)] df[df$age %in% c(19, 30), 1:2] ``` shivaram Author: felixcheung Closes #8394 from felixcheung/rsubset. --- R/pkg/R/DataFrame.R | 22 +++++++++++++++++++++- R/pkg/inst/tests/test_sparkSQL.R | 27 +++++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index ae1d912cf6da1..a5162de705f8f 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -985,9 +985,11 @@ setMethod("$<-", signature(x = "DataFrame"), x }) +setClassUnion("numericOrcharacter", c("numeric", "character")) + #' @rdname select #' @name [[ -setMethod("[[", signature(x = "DataFrame"), +setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"), function(x, i) { if (is.numeric(i)) { cols <- columns(x) @@ -1010,6 +1012,20 @@ setMethod("[", signature(x = "DataFrame", i = "missing"), select(x, j) }) +#' @rdname select +#' @name [ +setMethod("[", signature(x = "DataFrame", i = "Column"), + function(x, i, j, ...) { + # It could handle i as "character" but it seems confusing and not required + # https://stat.ethz.ch/R-manual/R-devel/library/base/html/Extract.data.frame.html + filtered <- filter(x, i) + if (!missing(j)) { + filtered[, j] + } else { + filtered + } + }) + #' Select #' #' Selects a set of columns with names or Column expressions. @@ -1028,8 +1044,12 @@ setMethod("[", signature(x = "DataFrame", i = "missing"), #' # Columns can also be selected using `[[` and `[` #' df[[2]] == df[["age"]] #' df[,2] == df[,"age"] +#' df[,c("name", "age")] #' # Similar to R data frames columns can also be selected using `$` #' df$age +#' # It can also be subset on rows and Columns +#' df[df$name == "Smith", c(1,2)] +#' df[df$age %in% c(19, 30), 1:2] #' } setMethod("select", signature(x = "DataFrame", col = "character"), function(x, col, ...) { diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 556b8c5447054..ee48a3dc0cc05 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -587,6 +587,33 @@ test_that("select with column", { expect_equal(collect(select(df3, "x"))[[1, 1]], "x") }) +test_that("subsetting", { + # jsonFile returns columns in random order + df <- select(jsonFile(sqlContext, jsonPath), "name", "age") + filtered <- df[df$age > 20,] + expect_equal(count(filtered), 1) + expect_equal(columns(filtered), c("name", "age")) + expect_equal(collect(filtered)$name, "Andy") + + df2 <- df[df$age == 19, 1] + expect_is(df2, "DataFrame") + expect_equal(count(df2), 1) + expect_equal(columns(df2), c("name")) + expect_equal(collect(df2)$name, "Justin") + + df3 <- df[df$age > 20, 2] + expect_equal(count(df3), 1) + expect_equal(columns(df3), c("age")) + + df4 <- df[df$age %in% c(19, 30), 1:2] + expect_equal(count(df4), 2) + expect_equal(columns(df4), c("name", "age")) + + df5 <- df[df$age %in% c(19), c(1,2)] + expect_equal(count(df5), 1) + expect_equal(columns(df5), c("name", "age")) +}) + test_that("selectExpr() on a DataFrame", { df <- jsonFile(sqlContext, jsonPath) selected <- selectExpr(df, "age * 2") From bb1640529725c6c38103b95af004f8bd90eeee5c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 26 Aug 2015 00:37:04 -0700 Subject: [PATCH 101/232] Closes #8443 From 6519fd06cc8175c9182ef16cf8a37d7f255eb846 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Aug 2015 11:47:05 -0700 Subject: [PATCH 102/232] [SPARK-9665] [MLLIB] audit MLlib API annotations I only found `ml.NaiveBayes` missing `Experimental` annotation. This PR doesn't cover Python APIs. cc jkbradley Author: Xiangrui Meng Closes #8452 from mengxr/SPARK-9665. --- .../apache/spark/ml/classification/NaiveBayes.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 97cbaf1fa8761..69cb88a7e6718 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -18,11 +18,11 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkException -import org.apache.spark.ml.{PredictorParams, PredictionModel, Predictor} -import org.apache.spark.ml.param.{ParamMap, ParamValidators, Param, DoubleParam} +import org.apache.spark.annotation.Experimental +import org.apache.spark.ml.PredictorParams +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.util.Identifiable -import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes} -import org.apache.spark.mllib.classification.{NaiveBayesModel => OldNaiveBayesModel} +import org.apache.spark.mllib.classification.{NaiveBayes => OldNaiveBayes, NaiveBayesModel => OldNaiveBayesModel} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -59,6 +59,7 @@ private[ml] trait NaiveBayesParams extends PredictorParams { } /** + * :: Experimental :: * Naive Bayes Classifiers. * It supports both Multinomial NB * ([[http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html]]) @@ -68,6 +69,7 @@ private[ml] trait NaiveBayesParams extends PredictorParams { * ([[http://nlp.stanford.edu/IR-book/html/htmledition/the-bernoulli-model-1.html]]). * The input feature values must be nonnegative. */ +@Experimental class NaiveBayes(override val uid: String) extends ProbabilisticClassifier[Vector, NaiveBayes, NaiveBayesModel] with NaiveBayesParams { @@ -101,11 +103,13 @@ class NaiveBayes(override val uid: String) } /** + * :: Experimental :: * Model produced by [[NaiveBayes]] * @param pi log of class priors, whose dimension is C (number of classes) * @param theta log of class conditional probabilities, whose dimension is C (number of classes) * by D (number of features) */ +@Experimental class NaiveBayesModel private[ml] ( override val uid: String, val pi: Vector, From de7209c256aaf79a0978cfcf6e98bb013267b93a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Aug 2015 12:19:36 -0700 Subject: [PATCH 103/232] HOTFIX: Increase PRB timeout --- dev/run-tests-jenkins | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index c4d39d95d5890..f144c053046c5 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -48,8 +48,8 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" # format: http://linux.die.net/man/1/timeout -# must be less than the timeout configured on Jenkins (currently 180m) -TESTS_TIMEOUT="175m" +# must be less than the timeout configured on Jenkins (currently 300m) +TESTS_TIMEOUT="250m" # Array to capture all tests to run on the pull request. These tests are held under the #+ dev/tests/ directory. From 086d4681df3ebfccfc04188262c10482f44553b0 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Aug 2015 14:02:19 -0700 Subject: [PATCH 104/232] [SPARK-10241] [MLLIB] update since versions in mllib.recommendation Same as #8421 but for `mllib.recommendation`. cc srowen coderxiang Author: Xiangrui Meng Closes #8432 from mengxr/SPARK-10241. --- .../spark/mllib/recommendation/ALS.scala | 22 ++++++++++++++++++- .../MatrixFactorizationModel.scala | 8 +++---- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index b27ef1b949e2e..33aaf853e599d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -28,7 +28,10 @@ import org.apache.spark.storage.StorageLevel * A more compact class to represent a rating than Tuple3[Int, Int, Double]. */ @Since("0.8.0") -case class Rating(user: Int, product: Int, rating: Double) +case class Rating @Since("0.8.0") ( + @Since("0.8.0") user: Int, + @Since("0.8.0") product: Int, + @Since("0.8.0") rating: Double) /** * Alternating Least Squares matrix factorization. @@ -59,6 +62,7 @@ case class Rating(user: Int, product: Int, rating: Double) * indicated user * preferences rather than explicit ratings given to items. */ +@Since("0.8.0") class ALS private ( private var numUserBlocks: Int, private var numProductBlocks: Int, @@ -74,6 +78,7 @@ class ALS private ( * Constructs an ALS instance with default parameters: {numBlocks: -1, rank: 10, iterations: 10, * lambda: 0.01, implicitPrefs: false, alpha: 1.0}. */ + @Since("0.8.0") def this() = this(-1, -1, 10, 10, 0.01, false, 1.0) /** If true, do alternating nonnegative least squares. */ @@ -90,6 +95,7 @@ class ALS private ( * Set the number of blocks for both user blocks and product blocks to parallelize the computation * into; pass -1 for an auto-configured number of blocks. Default: -1. */ + @Since("0.8.0") def setBlocks(numBlocks: Int): this.type = { this.numUserBlocks = numBlocks this.numProductBlocks = numBlocks @@ -99,6 +105,7 @@ class ALS private ( /** * Set the number of user blocks to parallelize the computation. */ + @Since("1.1.0") def setUserBlocks(numUserBlocks: Int): this.type = { this.numUserBlocks = numUserBlocks this @@ -107,30 +114,35 @@ class ALS private ( /** * Set the number of product blocks to parallelize the computation. */ + @Since("1.1.0") def setProductBlocks(numProductBlocks: Int): this.type = { this.numProductBlocks = numProductBlocks this } /** Set the rank of the feature matrices computed (number of features). Default: 10. */ + @Since("0.8.0") def setRank(rank: Int): this.type = { this.rank = rank this } /** Set the number of iterations to run. Default: 10. */ + @Since("0.8.0") def setIterations(iterations: Int): this.type = { this.iterations = iterations this } /** Set the regularization parameter, lambda. Default: 0.01. */ + @Since("0.8.0") def setLambda(lambda: Double): this.type = { this.lambda = lambda this } /** Sets whether to use implicit preference. Default: false. */ + @Since("0.8.1") def setImplicitPrefs(implicitPrefs: Boolean): this.type = { this.implicitPrefs = implicitPrefs this @@ -139,12 +151,14 @@ class ALS private ( /** * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ + @Since("0.8.1") def setAlpha(alpha: Double): this.type = { this.alpha = alpha this } /** Sets a random seed to have deterministic results. */ + @Since("1.0.0") def setSeed(seed: Long): this.type = { this.seed = seed this @@ -154,6 +168,7 @@ class ALS private ( * Set whether the least-squares problems solved at each iteration should have * nonnegativity constraints. */ + @Since("1.1.0") def setNonnegative(b: Boolean): this.type = { this.nonnegative = b this @@ -166,6 +181,7 @@ class ALS private ( * set `spark.rdd.compress` to `true` to reduce the space requirement, at the cost of speed. */ @DeveloperApi + @Since("1.1.0") def setIntermediateRDDStorageLevel(storageLevel: StorageLevel): this.type = { require(storageLevel != StorageLevel.NONE, "ALS is not designed to run without persisting intermediate RDDs.") @@ -181,6 +197,7 @@ class ALS private ( * at the cost of speed. */ @DeveloperApi + @Since("1.3.0") def setFinalRDDStorageLevel(storageLevel: StorageLevel): this.type = { this.finalRDDStorageLevel = storageLevel this @@ -194,6 +211,7 @@ class ALS private ( * this setting is ignored. */ @DeveloperApi + @Since("1.4.0") def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval this @@ -203,6 +221,7 @@ class ALS private ( * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. */ + @Since("0.8.0") def run(ratings: RDD[Rating]): MatrixFactorizationModel = { val sc = ratings.context @@ -250,6 +269,7 @@ class ALS private ( /** * Java-friendly version of [[ALS.run]]. */ + @Since("1.3.0") def run(ratings: JavaRDD[Rating]): MatrixFactorizationModel = run(ratings.rdd) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index ba4cfdcd9f1dd..46562eb2ad0f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -52,10 +52,10 @@ import org.apache.spark.storage.StorageLevel * and the features computed for this product. */ @Since("0.8.0") -class MatrixFactorizationModel( - val rank: Int, - val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) +class MatrixFactorizationModel @Since("0.8.0") ( + @Since("0.8.0") val rank: Int, + @Since("0.8.0") val userFeatures: RDD[(Int, Array[Double])], + @Since("0.8.0") val productFeatures: RDD[(Int, Array[Double])]) extends Saveable with Serializable with Logging { require(rank > 0) From d41d6c48207159490c1e1d9cc54015725cfa41b2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 26 Aug 2015 16:04:44 -0700 Subject: [PATCH 105/232] [SPARK-10305] [SQL] fix create DataFrame from Python class cc jkbradley Author: Davies Liu Closes #8470 from davies/fix_create_df. --- python/pyspark/sql/tests.py | 12 ++++++++++++ python/pyspark/sql/types.py | 6 ++++++ 2 files changed, 18 insertions(+) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index aacfb34c77618..cd32e26c64f22 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -145,6 +145,12 @@ class PythonOnlyPoint(ExamplePoint): __UDT__ = PythonOnlyUDT() +class MyObject(object): + def __init__(self, key, value): + self.key = key + self.value = value + + class DataTypeTests(unittest.TestCase): # regression test for SPARK-6055 def test_data_type_eq(self): @@ -383,6 +389,12 @@ def test_infer_nested_schema(self): df = self.sqlCtx.inferSchema(rdd) self.assertEquals(Row(field1=1, field2=u'row1'), df.first()) + def test_create_dataframe_from_objects(self): + data = [MyObject(1, "1"), MyObject(2, "2")] + df = self.sqlCtx.createDataFrame(data) + self.assertEqual(df.dtypes, [("key", "bigint"), ("value", "string")]) + self.assertEqual(df.first(), Row(key=1, value="1")) + def test_select_null_literal(self): df = self.sqlCtx.sql("select null as col") self.assertEquals(Row(col=None), df.first()) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index ed4e5b594bd61..94e581a78364c 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -537,6 +537,9 @@ def toInternal(self, obj): return tuple(f.toInternal(obj.get(n)) for n, f in zip(self.names, self.fields)) elif isinstance(obj, (tuple, list)): return tuple(f.toInternal(v) for f, v in zip(self.fields, obj)) + elif hasattr(obj, "__dict__"): + d = obj.__dict__ + return tuple(f.toInternal(d.get(n)) for n, f in zip(self.names, self.fields)) else: raise ValueError("Unexpected tuple %r with StructType" % obj) else: @@ -544,6 +547,9 @@ def toInternal(self, obj): return tuple(obj.get(n) for n in self.names) elif isinstance(obj, (list, tuple)): return tuple(obj) + elif hasattr(obj, "__dict__"): + d = obj.__dict__ + return tuple(d.get(n) for n in self.names) else: raise ValueError("Unexpected tuple %r with StructType" % obj) From ad7f0f160be096c0fdae6e6cf7e3b6ba4a606de7 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 26 Aug 2015 18:13:07 -0700 Subject: [PATCH 106/232] [SPARK-10308] [SPARKR] Add %in% to the exported namespace I also checked all the other functions defined in column.R, functions.R and DataFrame.R and everything else looked fine. cc yu-iskw Author: Shivaram Venkataraman Closes #8473 from shivaram/in-namespace. --- R/pkg/NAMESPACE | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 3e5c89d779b7b..5286c01986204 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -47,12 +47,12 @@ exportMethods("arrange", "join", "limit", "merge", + "mutate", + "na.omit", "names", "ncol", "nrow", "orderBy", - "mutate", - "names", "persist", "printSchema", "rbind", @@ -82,7 +82,8 @@ exportMethods("arrange", exportClasses("Column") -exportMethods("abs", +exportMethods("%in%", + "abs", "acos", "add_months", "alias", From 773ca037a43d464ce7f16fe693ca6034f09a35b7 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Wed, 26 Aug 2015 18:14:32 -0700 Subject: [PATCH 107/232] [MINOR] [SPARKR] Fix some validation problems in SparkR Getting rid of some validation problems in SparkR https://github.com/apache/spark/pull/7883 cc shivaram ``` inst/tests/test_Serde.R:26:1: style: Trailing whitespace is superfluous. ^~ inst/tests/test_Serde.R:34:1: style: Trailing whitespace is superfluous. ^~ inst/tests/test_Serde.R:37:38: style: Trailing whitespace is superfluous. expect_equal(class(x), "character") ^~ inst/tests/test_Serde.R:50:1: style: Trailing whitespace is superfluous. ^~ inst/tests/test_Serde.R:55:1: style: Trailing whitespace is superfluous. ^~ inst/tests/test_Serde.R:60:1: style: Trailing whitespace is superfluous. ^~ inst/tests/test_sparkSQL.R:611:1: style: Trailing whitespace is superfluous. ^~ R/DataFrame.R:664:1: style: Trailing whitespace is superfluous. ^~~~~~~~~~~~~~ R/DataFrame.R:670:55: style: Trailing whitespace is superfluous. df <- data.frame(row.names = 1 : nrow) ^~~~~~~~~~~~~~~~ R/DataFrame.R:672:1: style: Trailing whitespace is superfluous. ^~~~~~~~~~~~~~ R/DataFrame.R:686:49: style: Trailing whitespace is superfluous. df[[names[colIndex]]] <- vec ^~~~~~~~~~~~~~~~~~ ``` Author: Yu ISHIKAWA Closes #8474 from yu-iskw/minor-fix-sparkr. --- R/pkg/R/DataFrame.R | 8 ++++---- R/pkg/inst/tests/test_Serde.R | 12 ++++++------ R/pkg/inst/tests/test_sparkSQL.R | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index a5162de705f8f..dd8126aebf467 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -661,15 +661,15 @@ setMethod("collect", # listCols is a list of columns listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) stopifnot(length(listCols) == ncol) - + # An empty data.frame with 0 columns and number of rows as collected nrow <- length(listCols[[1]]) if (nrow <= 0) { df <- data.frame() } else { - df <- data.frame(row.names = 1 : nrow) + df <- data.frame(row.names = 1 : nrow) } - + # Append columns one by one for (colIndex in 1 : ncol) { # Note: appending a column of list type into a data.frame so that @@ -683,7 +683,7 @@ setMethod("collect", # TODO: more robust check on column of primitive types vec <- do.call(c, col) if (class(vec) != "list") { - df[[names[colIndex]]] <- vec + df[[names[colIndex]]] <- vec } else { # For columns of complex type, be careful to access them. # Get a column of complex type returns a list. diff --git a/R/pkg/inst/tests/test_Serde.R b/R/pkg/inst/tests/test_Serde.R index 009db85da2beb..dddce54d70443 100644 --- a/R/pkg/inst/tests/test_Serde.R +++ b/R/pkg/inst/tests/test_Serde.R @@ -23,7 +23,7 @@ test_that("SerDe of primitive types", { x <- callJStatic("SparkRHandler", "echo", 1L) expect_equal(x, 1L) expect_equal(class(x), "integer") - + x <- callJStatic("SparkRHandler", "echo", 1) expect_equal(x, 1) expect_equal(class(x), "numeric") @@ -31,10 +31,10 @@ test_that("SerDe of primitive types", { x <- callJStatic("SparkRHandler", "echo", TRUE) expect_true(x) expect_equal(class(x), "logical") - + x <- callJStatic("SparkRHandler", "echo", "abc") expect_equal(x, "abc") - expect_equal(class(x), "character") + expect_equal(class(x), "character") }) test_that("SerDe of list of primitive types", { @@ -47,17 +47,17 @@ test_that("SerDe of list of primitive types", { y <- callJStatic("SparkRHandler", "echo", x) expect_equal(x, y) expect_equal(class(y[[1]]), "numeric") - + x <- list(TRUE, FALSE) y <- callJStatic("SparkRHandler", "echo", x) expect_equal(x, y) expect_equal(class(y[[1]]), "logical") - + x <- list("a", "b", "c") y <- callJStatic("SparkRHandler", "echo", x) expect_equal(x, y) expect_equal(class(y[[1]]), "character") - + # Empty list x <- list() y <- callJStatic("SparkRHandler", "echo", x) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index ee48a3dc0cc05..8e22c56824b16 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -608,7 +608,7 @@ test_that("subsetting", { df4 <- df[df$age %in% c(19, 30), 1:2] expect_equal(count(df4), 2) expect_equal(columns(df4), c("name", "age")) - + df5 <- df[df$age %in% c(19), c(1,2)] expect_equal(count(df5), 1) expect_equal(columns(df5), c("name", "age")) From 0fac144f6bd835395059154532d72cdb5dc7ef8d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 26 Aug 2015 18:14:54 -0700 Subject: [PATCH 108/232] [SPARK-9424] [SQL] Parquet programming guide updates for 1.5 Author: Cheng Lian Closes #8467 from liancheng/spark-9424/parquet-docs-for-1.5. --- docs/sql-programming-guide.md | 45 ++++++++++++++++++++++++++++------- 1 file changed, 37 insertions(+), 8 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 33e7893d7bd0a..e64190b9b209d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1124,6 +1124,13 @@ a simple schema, and gradually add more columns to the schema as needed. In thi up with multiple Parquet files with different but mutually compatible schemas. The Parquet data source is now able to automatically detect this case and merge schemas of all these files. +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default starting from 1.5.0. You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the + examples below), or +2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. +
@@ -1143,7 +1150,7 @@ val df2 = sc.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") df2.write.parquet("data/test_table/key=2") // Read the partitioned table -val df3 = sqlContext.read.parquet("data/test_table") +val df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") df3.printSchema() // The final schema consists of all 3 columns in the Parquet files together @@ -1165,16 +1172,16 @@ df3.printSchema() # Create a simple DataFrame, stored into a partition directory df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ .map(lambda i: Row(single=i, double=i * 2))) -df1.save("data/test_table/key=1", "parquet") +df1.write.parquet("data/test_table/key=1") # Create another DataFrame in a new partition directory, # adding a new column and dropping an existing column df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) .map(lambda i: Row(single=i, triple=i * 3))) -df2.save("data/test_table/key=2", "parquet") +df2.write.parquet("data/test_table/key=2") # Read the partitioned table -df3 = sqlContext.load("data/test_table", "parquet") +df3 = sqlContext.read.option("mergeSchema", "true").parquet("data/test_table") df3.printSchema() # The final schema consists of all 3 columns in the Parquet files together @@ -1201,7 +1208,7 @@ saveDF(df1, "data/test_table/key=1", "parquet", "overwrite") saveDF(df2, "data/test_table/key=2", "parquet", "overwrite") # Read the partitioned table -df3 <- loadDF(sqlContext, "data/test_table", "parquet") +df3 <- loadDF(sqlContext, "data/test_table", "parquet", mergeSchema="true") printSchema(df3) # The final schema consists of all 3 columns in the Parquet files together @@ -1301,7 +1308,7 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext` spark.sql.parquet.binaryAsString false - Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do + Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. @@ -1310,8 +1317,7 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext` spark.sql.parquet.int96AsTimestamp true - Some Parquet-producing systems, in particular Impala, store Timestamp into INT96. Spark would also - store Timestamp as INT96 because we need to avoid precision lost of the nanoseconds field. This + Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. @@ -1355,6 +1361,9 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`

Note:

    +
  • + This option is automatically ignored if spark.speculation is turned on. +
  • This option must be set via Hadoop Configuration rather than Spark SQLConf. @@ -1371,6 +1380,26 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`

    + + spark.sql.parquet.mergeSchema + false + +

    + When true, the Parquet data source merges schemas collected from all data files, otherwise the + schema is picked from the summary file or a random data file if no summary file is available. +

    + + + + spark.sql.parquet.mergeSchema + false + +

    + When true, the Parquet data source merges schemas collected from all data files, otherwise the + schema is picked from the summary file or a random data file if no summary file is available. +

    + + ## JSON Datasets From ce97834dc0cc55eece0e909a4061ca6f2123f60d Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 26 Aug 2015 22:19:11 -0700 Subject: [PATCH 109/232] [SPARK-9964] [PYSPARK] [SQL] PySpark DataFrameReader accept RDD of String for JSON PySpark DataFrameReader should could accept an RDD of Strings (like the Scala version does) for JSON, rather than only taking a path. If this PR is merged, it should be duplicated to cover the other input types (not just JSON). Author: Yanbo Liang Closes #8444 from yanboliang/spark-9964. --- python/pyspark/sql/readwriter.py | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 78247c8fa7372..3fa6895880a97 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -15,8 +15,14 @@ # limitations under the License. # +import sys + +if sys.version >= '3': + basestring = unicode = str + from py4j.java_gateway import JavaClass +from pyspark import RDD from pyspark.sql import since from pyspark.sql.column import _to_seq from pyspark.sql.types import * @@ -125,23 +131,33 @@ def load(self, path=None, format=None, schema=None, **options): @since(1.4) def json(self, path, schema=None): """ - Loads a JSON file (one object per line) and returns the result as - a :class`DataFrame`. + Loads a JSON file (one object per line) or an RDD of Strings storing JSON objects + (one object per record) and returns the result as a :class`DataFrame`. If the ``schema`` parameter is not specified, this function goes through the input once to determine the input schema. - :param path: string, path to the JSON dataset. + :param path: string represents path to the JSON dataset, + or RDD of Strings storing JSON objects. :param schema: an optional :class:`StructType` for the input schema. - >>> df = sqlContext.read.json('python/test_support/sql/people.json') - >>> df.dtypes + >>> df1 = sqlContext.read.json('python/test_support/sql/people.json') + >>> df1.dtypes + [('age', 'bigint'), ('name', 'string')] + >>> rdd = sc.textFile('python/test_support/sql/people.json') + >>> df2 = sqlContext.read.json(rdd) + >>> df2.dtypes [('age', 'bigint'), ('name', 'string')] """ if schema is not None: self.schema(schema) - return self._df(self._jreader.json(path)) + if isinstance(path, basestring): + return self._df(self._jreader.json(path)) + elif isinstance(path, RDD): + return self._df(self._jreader.json(path._jrdd)) + else: + raise TypeError("path can be only string or RDD") @since(1.4) def table(self, tableName): From e936cf8088a06d6aefce44305f3904bbeb17b432 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 26 Aug 2015 22:27:31 -0700 Subject: [PATCH 110/232] [SPARK-10219] [SPARKR] Fix varargsToEnv and add test case cc sun-rui davies Author: Shivaram Venkataraman Closes #8475 from shivaram/varargs-fix. --- R/pkg/R/utils.R | 3 ++- R/pkg/inst/tests/test_sparkSQL.R | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 4f9f4d9cad2a8..3babcb519378e 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -314,7 +314,8 @@ convertEnvsToList <- function(keys, vals) { # Utility function to capture the varargs into environment object varargsToEnv <- function(...) { - pairs <- as.list(substitute(list(...)))[-1L] + # Based on http://stackoverflow.com/a/3057419/4577954 + pairs <- list(...) env <- new.env() for (name in names(pairs)) { env[[name]] <- pairs[[name]] diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 8e22c56824b16..4b672e115f924 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -1060,6 +1060,12 @@ test_that("parquetFile works with multiple input paths", { parquetDF <- parquetFile(sqlContext, parquetPath, parquetPath2) expect_is(parquetDF, "DataFrame") expect_equal(count(parquetDF), count(df) * 2) + + # Test if varargs works with variables + saveMode <- "overwrite" + mergeSchema <- "true" + parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") + write.df(df, parquetPath2, "parquet", mode = saveMode, mergeSchema = mergeSchema) }) test_that("describe() and summarize() on a DataFrame", { From de0278286cf6db8df53b0b68918ea114f2c77f1f Mon Sep 17 00:00:00 2001 From: Ram Sriharsha Date: Wed, 26 Aug 2015 23:12:55 -0700 Subject: [PATCH 111/232] =?UTF-8?q?[SPARK-10251]=20[CORE]=20some=20common?= =?UTF-8?q?=20types=20are=20not=20registered=20for=20Kryo=20Serializat?= =?UTF-8?q?=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …ion by default Author: Ram Sriharsha Closes #8465 from harsha2010/SPARK-10251. --- .../spark/serializer/KryoSerializer.scala | 35 ++++++++++++++++++- .../serializer/KryoSerializerSuite.scala | 30 ++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 048a938507277..b977711e7d5ad 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import javax.annotation.Nullable import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException} @@ -38,7 +39,7 @@ import org.apache.spark.network.nio.{GetBlock, GotBlock, PutBlock} import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ -import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf} +import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf} import org.apache.spark.util.collection.CompactBuffer /** @@ -131,6 +132,38 @@ class KryoSerializer(conf: SparkConf) // our code override the generic serializers in Chill for things like Seq new AllScalaRegistrar().apply(kryo) + // Register types missed by Chill. + // scalastyle:off + kryo.register(classOf[Array[Tuple1[Any]]]) + kryo.register(classOf[Array[Tuple2[Any, Any]]]) + kryo.register(classOf[Array[Tuple3[Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple4[Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple5[Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple6[Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple7[Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple8[Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple9[Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + kryo.register(classOf[Array[Tuple22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]]]) + + // scalastyle:on + + kryo.register(None.getClass) + kryo.register(Nil.getClass) + kryo.register(Utils.classForName("scala.collection.immutable.$colon$colon")) + kryo.register(classOf[ArrayBuffer[Any]]) + kryo.setClassLoader(classLoader) kryo } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 8d1c9d17e977e..e428414cf6e85 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -150,6 +150,36 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { mutable.HashMap(1->"one", 2->"two", 3->"three"))) } + test("Bug: SPARK-10251") { + val ser = new KryoSerializer(conf.clone.set("spark.kryo.registrationRequired", "true")) + .newInstance() + def check[T: ClassTag](t: T) { + assert(ser.deserialize[T](ser.serialize(t)) === t) + } + check((1, 3)) + check(Array((1, 3))) + check(List((1, 3))) + check(List[Int]()) + check(List[Int](1, 2, 3)) + check(List[String]()) + check(List[String]("x", "y", "z")) + check(None) + check(Some(1)) + check(Some("hi")) + check(1 -> 1) + check(mutable.ArrayBuffer(1, 2, 3)) + check(mutable.ArrayBuffer("1", "2", "3")) + check(mutable.Map()) + check(mutable.Map(1 -> "one", 2 -> "two")) + check(mutable.Map("one" -> 1, "two" -> 2)) + check(mutable.HashMap(1 -> "one", 2 -> "two")) + check(mutable.HashMap("one" -> 1, "two" -> 2)) + check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1->"one", 2->"two", 3->"three"))) + } + test("ranges") { val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { From 9625d13d575c97bbff264f6a94838aae72c9202d Mon Sep 17 00:00:00 2001 From: Moussa Taifi Date: Thu, 27 Aug 2015 10:34:47 +0100 Subject: [PATCH 112/232] [DOCS] [STREAMING] [KAFKA] Fix typo in exactly once semantics Fix Typo in exactly once semantics [Semantics of output operations] link Author: Moussa Taifi Closes #8468 from moutai/patch-3. --- docs/streaming-kafka-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 7571e22575efd..5db39ae54a274 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -82,7 +82,7 @@ This approach has the following advantages over the receiver-based approach (i.e - *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. -- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semanitcs of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). +- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). From 1650f6f56ed4b7f1a7f645c9e8d5ac533464bd78 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 10:44:44 +0100 Subject: [PATCH 113/232] [SPARK-10254] [ML] Removes Guava dependencies in spark.ml.feature JavaTests * Replaces `com.google.common` dependencies with `java.util.Arrays` * Small clean up in `JavaNormalizerSuite` Author: Feynman Liang Closes #8445 from feynmanliang/SPARK-10254. --- .../apache/spark/ml/feature/JavaBucketizerSuite.java | 5 +++-- .../org/apache/spark/ml/feature/JavaDCTSuite.java | 5 +++-- .../apache/spark/ml/feature/JavaHashingTFSuite.java | 5 +++-- .../apache/spark/ml/feature/JavaNormalizerSuite.java | 11 +++++------ .../org/apache/spark/ml/feature/JavaPCASuite.java | 4 ++-- .../ml/feature/JavaPolynomialExpansionSuite.java | 5 +++-- .../spark/ml/feature/JavaStandardScalerSuite.java | 4 ++-- .../apache/spark/ml/feature/JavaTokenizerSuite.java | 6 ++++-- .../spark/ml/feature/JavaVectorIndexerSuite.java | 5 ++--- .../spark/ml/feature/JavaVectorSlicerSuite.java | 4 ++-- .../apache/spark/ml/feature/JavaWord2VecSuite.java | 11 ++++++----- 11 files changed, 35 insertions(+), 30 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java index d5bd230a957a1..47d68de599da2 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaBucketizerSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -54,7 +55,7 @@ public void tearDown() { public void bucketizerTest() { double[] splits = {-0.5, 0.0, 0.5}; - JavaRDD data = jsc.parallelize(Lists.newArrayList( + JavaRDD data = jsc.parallelize(Arrays.asList( RowFactory.create(-0.5), RowFactory.create(-0.3), RowFactory.create(0.0), diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java index 845eed61c45c6..0f6ec64d97d36 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaDCTSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D; import org.junit.After; import org.junit.Assert; @@ -56,7 +57,7 @@ public void tearDown() { @Test public void javaCompatibilityTest() { double[] input = new double[] {1D, 2D, 3D, 4D}; - JavaRDD data = jsc.parallelize(Lists.newArrayList( + JavaRDD data = jsc.parallelize(Arrays.asList( RowFactory.create(Vectors.dense(input)) )); DataFrame dataset = jsql.createDataFrame(data, new StructType(new StructField[]{ diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java index 599e9cfd23ad4..03dd5369bddf7 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaHashingTFSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -54,7 +55,7 @@ public void tearDown() { @Test public void hashingTF() { - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + JavaRDD jrdd = jsc.parallelize(Arrays.asList( RowFactory.create(0.0, "Hi I heard about Spark"), RowFactory.create(0.0, "I wish Java could use case classes"), RowFactory.create(1.0, "Logistic regression models are neat") diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java index d82f3b7e8c076..e17d549c5059b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaNormalizerSuite.java @@ -17,15 +17,15 @@ package org.apache.spark.ml.feature; -import java.util.List; +import java.util.Arrays; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; @@ -48,13 +48,12 @@ public void tearDown() { @Test public void normalizer() { // The tests are to check Java compatibility. - List points = Lists.newArrayList( + JavaRDD points = jsc.parallelize(Arrays.asList( new VectorIndexerSuite.FeatureData(Vectors.dense(0.0, -2.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) - ); - DataFrame dataFrame = jsql.createDataFrame(jsc.parallelize(points, 2), - VectorIndexerSuite.FeatureData.class); + )); + DataFrame dataFrame = jsql.createDataFrame(points, VectorIndexerSuite.FeatureData.class); Normalizer normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normFeatures"); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 5cf43fec6f29e..e8f329f9cf29e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -18,11 +18,11 @@ package org.apache.spark.ml.feature; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import scala.Tuple2; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -78,7 +78,7 @@ public Vector getExpected() { @Test public void testPCA() { - List points = Lists.newArrayList( + List points = Arrays.asList( Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0}), Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0), Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java index 5e8211c2c5118..834fedbb59e1b 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPolynomialExpansionSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -59,7 +60,7 @@ public void polynomialExpansionTest() { .setOutputCol("polyFeatures") .setDegree(3); - JavaRDD data = jsc.parallelize(Lists.newArrayList( + JavaRDD data = jsc.parallelize(Arrays.asList( RowFactory.create( Vectors.dense(-2.0, 2.3), Vectors.dense(-2.0, 4.0, -8.0, 2.3, -4.6, 9.2, 5.29, -10.58, 12.17) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java index 74eb2733f06ef..ed74363f59e34 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStandardScalerSuite.java @@ -17,9 +17,9 @@ package org.apache.spark.ml.feature; +import java.util.Arrays; import java.util.List; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -48,7 +48,7 @@ public void tearDown() { @Test public void standardScaler() { // The tests are to check Java compatibility. - List points = Lists.newArrayList( + List points = Arrays.asList( new VectorIndexerSuite.FeatureData(Vectors.dense(0.0, -2.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 3.0)), new VectorIndexerSuite.FeatureData(Vectors.dense(1.0, 4.0)) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java index 3806f650025b2..02309ce63219a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaTokenizerSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -54,7 +55,8 @@ public void regexTokenizer() { .setGaps(true) .setMinTokenLength(3); - JavaRDD rdd = jsc.parallelize(Lists.newArrayList( + + JavaRDD rdd = jsc.parallelize(Arrays.asList( new TokenizerTestData("Test of tok.", new String[] {"Test", "tok."}), new TokenizerTestData("Te,st. punct", new String[] {"Te,st.", "punct"}) )); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java index c7ae5468b9429..bfcca62fa1c98 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.ml.feature; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import java.util.Map; @@ -26,8 +27,6 @@ import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Lists; - import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; import org.apache.spark.mllib.linalg.Vectors; @@ -52,7 +51,7 @@ public void tearDown() { @Test public void vectorIndexerAPI() { // The tests are to check Java compatibility. - List points = Lists.newArrayList( + List points = Arrays.asList( new FeatureData(Vectors.dense(0.0, -2.0)), new FeatureData(Vectors.dense(1.0, 3.0)), new FeatureData(Vectors.dense(1.0, 4.0)) diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index 56988b9fb29cb..f953361427586 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -17,7 +17,7 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; import org.junit.After; import org.junit.Assert; @@ -63,7 +63,7 @@ public void vectorSlice() { }; AttributeGroup group = new AttributeGroup("userFeatures", attrs); - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + JavaRDD jrdd = jsc.parallelize(Arrays.asList( RowFactory.create(Vectors.sparse(3, new int[]{0, 1}, new double[]{-2.0, 2.3})), RowFactory.create(Vectors.dense(-2.0, 2.3, 0.0)) )); diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java index 39c70157f83c0..70f5ad9432212 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaWord2VecSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.feature; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -50,10 +51,10 @@ public void tearDown() { @Test public void testJavaWord2Vec() { - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( - RowFactory.create(Lists.newArrayList("Hi I heard about Spark".split(" "))), - RowFactory.create(Lists.newArrayList("I wish Java could use case classes".split(" "))), - RowFactory.create(Lists.newArrayList("Logistic regression models are neat".split(" "))) + JavaRDD jrdd = jsc.parallelize(Arrays.asList( + RowFactory.create(Arrays.asList("Hi I heard about Spark".split(" "))), + RowFactory.create(Arrays.asList("I wish Java could use case classes".split(" "))), + RowFactory.create(Arrays.asList("Logistic regression models are neat".split(" "))) )); StructType schema = new StructType(new StructField[]{ new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) From 75d62307946283b03bec6aaf1bdd4f2b08c93915 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 10:45:35 +0100 Subject: [PATCH 114/232] [SPARK-10255] [ML] Removes Guava dependencies from spark.ml.param JavaTests Author: Feynman Liang Closes #8446 from feynmanliang/SPARK-10255. --- .../java/org/apache/spark/ml/param/JavaParamsSuite.java | 7 ++++--- .../java/org/apache/spark/ml/param/JavaTestParams.java | 5 ++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java index 9890155e9f865..fa777f3d42a9a 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaParamsSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.ml.param; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -61,7 +62,7 @@ public void testParamValidate() { ParamValidators.ltEq(1.0); ParamValidators.inRange(0, 1, true, false); ParamValidators.inRange(0, 1); - ParamValidators.inArray(Lists.newArrayList(0, 1, 3)); - ParamValidators.inArray(Lists.newArrayList("a", "b")); + ParamValidators.inArray(Arrays.asList(0, 1, 3)); + ParamValidators.inArray(Arrays.asList("a", "b")); } } diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index dc6ce8061f62b..65841182df9b4 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -17,10 +17,9 @@ package org.apache.spark.ml.param; +import java.util.Arrays; import java.util.List; -import com.google.common.collect.Lists; - import org.apache.spark.ml.util.Identifiable$; /** @@ -89,7 +88,7 @@ private void init() { myIntParam_ = new IntParam(this, "myIntParam", "this is an int param", ParamValidators.gt(0)); myDoubleParam_ = new DoubleParam(this, "myDoubleParam", "this is a double param", ParamValidators.inRange(0.0, 1.0)); - List validStrings = Lists.newArrayList("a", "b"); + List validStrings = Arrays.asList("a", "b"); myStringParam_ = new Param(this, "myStringParam", "this is a string param", ParamValidators.inArray(validStrings)); myDoubleArrayParam_ = From 1a446f75b6cac46caea0217a66abeb226946ac71 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 10:46:18 +0100 Subject: [PATCH 115/232] [SPARK-10256] [ML] Removes guava dependency from spark.ml.classification JavaTests Author: Feynman Liang Closes #8447 from feynmanliang/SPARK-10256. --- .../apache/spark/ml/classification/JavaNaiveBayesSuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java index a700c9cddb206..8fd7bf55a2e5d 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaNaiveBayesSuite.java @@ -18,8 +18,8 @@ package org.apache.spark.ml.classification; import java.io.Serializable; +import java.util.Arrays; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -74,7 +74,7 @@ public void naiveBayesDefaultParams() { @Test public void testNaiveBayes() { - JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + JavaRDD jrdd = jsc.parallelize(Arrays.asList( RowFactory.create(0.0, Vectors.dense(1.0, 0.0, 0.0)), RowFactory.create(0.0, Vectors.dense(2.0, 0.0, 0.0)), RowFactory.create(1.0, Vectors.dense(0.0, 1.0, 0.0)), From b02e8187225d1765f67ce38864dfaca487be8a44 Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Thu, 27 Aug 2015 11:07:37 +0100 Subject: [PATCH 116/232] [SPARK-9613] [HOTFIX] Fix usage of JavaConverters removed in Scala 2.11 Fix for [JavaConverters.asJavaListConverter](http://www.scala-lang.org/api/2.10.5/index.html#scala.collection.JavaConverters$) being removed in 2.11.7 and hence the build fails with the 2.11 profile enabled. Tested with the default 2.10 and 2.11 profiles. BUILD SUCCESS in both cases. Build for 2.10: ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.7.1 -DskipTests clean install and 2.11: ./dev/change-scala-version.sh 2.11 ./build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.7.1 -Dscala-2.11 -DskipTests clean install Author: Jacek Laskowski Closes #8479 from jaceklaskowski/SPARK-9613-hotfix. --- .../org/apache/spark/ml/classification/JavaOneVsRestSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index 2744e020e9e49..253cabf0133d0 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -55,7 +55,7 @@ public void setUp() { double[] xMean = {5.843, 3.057, 3.758, 1.199}; double[] xVariance = {0.6856, 0.1899, 3.116, 0.581}; - List points = JavaConverters.asJavaListConverter( + List points = JavaConverters.seqAsJavaListConverter( generateMultinomialLogisticInput(weights, xMean, xVariance, true, nPoints, 42) ).asJava(); datasetRDD = jsc.parallelize(points, 2); From e1f4de4a7d15d4ca4b5c64ff929ac3980f5d706f Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 18:46:41 +0100 Subject: [PATCH 117/232] [SPARK-10257] [MLLIB] Removes Guava from all spark.mllib Java tests * Replaces instances of `Lists.newArrayList` with `Arrays.asList` * Replaces `commons.lang.StringUtils` over `com.google.collections.Strings` * Replaces `List` interface over `ArrayList` implementations This PR along with #8445 #8446 #8447 completely removes all `com.google.collections.Lists` dependencies within mllib's Java tests. Author: Feynman Liang Closes #8451 from feynmanliang/SPARK-10257. --- .../JavaStreamingLogisticRegressionSuite.java | 10 +++---- .../clustering/JavaGaussianMixtureSuite.java | 4 +-- .../mllib/clustering/JavaKMeansSuite.java | 9 +++---- .../clustering/JavaStreamingKMeansSuite.java | 10 +++---- .../spark/mllib/feature/JavaTfIdfSuite.java | 19 +++++++------ .../mllib/feature/JavaWord2VecSuite.java | 6 ++--- .../mllib/fpm/JavaAssociationRulesSuite.java | 5 ++-- .../spark/mllib/fpm/JavaFPGrowthSuite.java | 17 ++++++------ .../spark/mllib/linalg/JavaVectorsSuite.java | 5 ++-- .../mllib/random/JavaRandomRDDsSuite.java | 27 ++++++++++--------- .../mllib/recommendation/JavaALSSuite.java | 5 ++-- .../JavaIsotonicRegressionSuite.java | 7 ++--- .../JavaStreamingLinearRegressionSuite.java | 10 +++---- .../spark/mllib/stat/JavaStatisticsSuite.java | 11 ++++---- 14 files changed, 71 insertions(+), 74 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index 55787f8606d48..c9e5ee22f3273 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -18,11 +18,11 @@ package org.apache.spark.mllib.classification; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import scala.Tuple2; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -60,16 +60,16 @@ public void tearDown() { @Test @SuppressWarnings("unchecked") public void javaAPI() { - List trainingBatch = Lists.newArrayList( + List trainingBatch = Arrays.asList( new LabeledPoint(1.0, Vectors.dense(1.0)), new LabeledPoint(0.0, Vectors.dense(0.0))); JavaDStream training = - attachTestInputStream(ssc, Lists.newArrayList(trainingBatch, trainingBatch), 2); - List> testBatch = Lists.newArrayList( + attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); + List> testBatch = Arrays.asList( new Tuple2(10, Vectors.dense(1.0)), new Tuple2(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( - attachTestInputStream(ssc, Lists.newArrayList(testBatch, testBatch), 2)); + attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingLogisticRegressionWithSGD slr = new StreamingLogisticRegressionWithSGD() .setNumIterations(2) .setInitialWeights(Vectors.dense(0.0)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java index 467a7a69e8f30..123f78da54e34 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaGaussianMixtureSuite.java @@ -18,9 +18,9 @@ package org.apache.spark.mllib.clustering; import java.io.Serializable; +import java.util.Arrays; import java.util.List; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -48,7 +48,7 @@ public void tearDown() { @Test public void runGaussianMixture() { - List points = Lists.newArrayList( + List points = Arrays.asList( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.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 31676e64025d0..ad06676c72ac6 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.clustering; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import org.junit.After; @@ -25,8 +26,6 @@ import org.junit.Test; import static org.junit.Assert.*; -import com.google.common.collect.Lists; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.linalg.Vector; @@ -48,7 +47,7 @@ public void tearDown() { @Test public void runKMeansUsingStaticMethods() { - List points = Lists.newArrayList( + List points = Arrays.asList( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) @@ -67,7 +66,7 @@ public void runKMeansUsingStaticMethods() { @Test public void runKMeansUsingConstructor() { - List points = Lists.newArrayList( + List points = Arrays.asList( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) @@ -90,7 +89,7 @@ public void runKMeansUsingConstructor() { @Test public void testPredictJavaRDD() { - List points = Lists.newArrayList( + List points = Arrays.asList( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), Vectors.dense(1.0, 4.0, 6.0) diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index 3b0e879eec77f..d644766d1e54d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -18,11 +18,11 @@ package org.apache.spark.mllib.clustering; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import scala.Tuple2; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -60,16 +60,16 @@ public void tearDown() { @Test @SuppressWarnings("unchecked") public void javaAPI() { - List trainingBatch = Lists.newArrayList( + List trainingBatch = Arrays.asList( Vectors.dense(1.0), Vectors.dense(0.0)); JavaDStream training = - attachTestInputStream(ssc, Lists.newArrayList(trainingBatch, trainingBatch), 2); - List> testBatch = Lists.newArrayList( + attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); + List> testBatch = Arrays.asList( new Tuple2(10, Vectors.dense(1.0)), new Tuple2(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( - attachTestInputStream(ssc, Lists.newArrayList(testBatch, testBatch), 2)); + attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingKMeans skmeans = new StreamingKMeans() .setK(1) .setDecayFactor(1.0) diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index fbc26167ce66f..8a320afa4b13d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -18,14 +18,13 @@ package org.apache.spark.mllib.feature; import java.io.Serializable; -import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Lists; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -50,10 +49,10 @@ public void tfIdf() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Lists.newArrayList( - Lists.newArrayList("this is a sentence".split(" ")), - Lists.newArrayList("this is another sentence".split(" ")), - Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD> documents = sc.parallelize(Arrays.asList( + Arrays.asList("this is a sentence".split(" ")), + Arrays.asList("this is another sentence".split(" ")), + Arrays.asList("this is still a sentence".split(" "))), 2); JavaRDD termFreqs = tf.transform(documents); termFreqs.collect(); IDF idf = new IDF(); @@ -70,10 +69,10 @@ public void tfIdfMinimumDocumentFrequency() { // The tests are to check Java compatibility. HashingTF tf = new HashingTF(); @SuppressWarnings("unchecked") - JavaRDD> documents = sc.parallelize(Lists.newArrayList( - Lists.newArrayList("this is a sentence".split(" ")), - Lists.newArrayList("this is another sentence".split(" ")), - Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD> documents = sc.parallelize(Arrays.asList( + Arrays.asList("this is a sentence".split(" ")), + Arrays.asList("this is another sentence".split(" ")), + Arrays.asList("this is still a sentence".split(" "))), 2); JavaRDD termFreqs = tf.transform(documents); termFreqs.collect(); IDF idf = new IDF(2); diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java index fb7afe8c6434b..e13ed07e283dd 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -18,11 +18,11 @@ package org.apache.spark.mllib.feature; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import scala.Tuple2; -import com.google.common.collect.Lists; import com.google.common.base.Strings; import org.junit.After; import org.junit.Assert; @@ -51,8 +51,8 @@ public void tearDown() { public void word2Vec() { // The tests are to check Java compatibility. String sentence = Strings.repeat("a b ", 100) + Strings.repeat("a c ", 10); - List words = Lists.newArrayList(sentence.split(" ")); - List> localDoc = Lists.newArrayList(words, words); + List words = Arrays.asList(sentence.split(" ")); + List> localDoc = Arrays.asList(words, words); JavaRDD> doc = sc.parallelize(localDoc); Word2Vec word2vec = new Word2Vec() .setVectorSize(10) diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index d7c2cb3ae2067..2bef7a8609757 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -17,17 +17,16 @@ package org.apache.spark.mllib.fpm; import java.io.Serializable; +import java.util.Arrays; import org.junit.After; import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Lists; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset; - public class JavaAssociationRulesSuite implements Serializable { private transient JavaSparkContext sc; @@ -46,7 +45,7 @@ public void tearDown() { public void runAssociationRules() { @SuppressWarnings("unchecked") - JavaRDD> freqItemsets = sc.parallelize(Lists.newArrayList( + JavaRDD> freqItemsets = sc.parallelize(Arrays.asList( new FreqItemset(new String[] {"a"}, 15L), new FreqItemset(new String[] {"b"}, 35L), new FreqItemset(new String[] {"a", "b"}, 12L) diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java index 9ce2c52dca8b6..154f75d75e4a6 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java @@ -18,13 +18,12 @@ package org.apache.spark.mllib.fpm; import java.io.Serializable; -import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Test; -import com.google.common.collect.Lists; import static org.junit.Assert.*; import org.apache.spark.api.java.JavaRDD; @@ -48,13 +47,13 @@ public void tearDown() { public void runFPGrowth() { @SuppressWarnings("unchecked") - JavaRDD> rdd = sc.parallelize(Lists.newArrayList( - Lists.newArrayList("r z h k p".split(" ")), - Lists.newArrayList("z y x w v u t s".split(" ")), - Lists.newArrayList("s x o n r".split(" ")), - Lists.newArrayList("x z y m t s q e".split(" ")), - Lists.newArrayList("z".split(" ")), - Lists.newArrayList("x z y r q t p".split(" "))), 2); + JavaRDD> rdd = sc.parallelize(Arrays.asList( + Arrays.asList("r z h k p".split(" ")), + Arrays.asList("z y x w v u t s".split(" ")), + Arrays.asList("s x o n r".split(" ")), + Arrays.asList("x z y m t s q e".split(" ")), + Arrays.asList("z".split(" ")), + Arrays.asList("x z y r q t p".split(" "))), 2); FPGrowthModel model = new FPGrowth() .setMinSupport(0.5) diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 1421067dc61ed..77c8c6274f374 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -18,11 +18,10 @@ package org.apache.spark.mllib.linalg; import java.io.Serializable; +import java.util.Arrays; import scala.Tuple2; -import com.google.common.collect.Lists; - import org.junit.Test; import static org.junit.Assert.*; @@ -37,7 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { @SuppressWarnings("unchecked") - Vector v = Vectors.sparse(3, Lists.>newArrayList( + Vector v = Vectors.sparse(3, Arrays.asList( new Tuple2(0, 2.0), new Tuple2(2, 3.0))); assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java index fcc13c00cbdc5..33d81b1e9592b 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -17,7 +17,8 @@ package org.apache.spark.mllib.random; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.apache.spark.api.java.JavaRDD; import org.junit.Assert; import org.junit.After; @@ -51,7 +52,7 @@ public void testUniformRDD() { JavaDoubleRDD rdd1 = uniformJavaRDD(sc, m); JavaDoubleRDD rdd2 = uniformJavaRDD(sc, m, p); JavaDoubleRDD rdd3 = uniformJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -64,7 +65,7 @@ public void testNormalRDD() { JavaDoubleRDD rdd1 = normalJavaRDD(sc, m); JavaDoubleRDD rdd2 = normalJavaRDD(sc, m, p); JavaDoubleRDD rdd3 = normalJavaRDD(sc, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -79,7 +80,7 @@ public void testLNormalRDD() { JavaDoubleRDD rdd1 = logNormalJavaRDD(sc, mean, std, m); JavaDoubleRDD rdd2 = logNormalJavaRDD(sc, mean, std, m, p); JavaDoubleRDD rdd3 = logNormalJavaRDD(sc, mean, std, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -93,7 +94,7 @@ public void testPoissonRDD() { JavaDoubleRDD rdd1 = poissonJavaRDD(sc, mean, m); JavaDoubleRDD rdd2 = poissonJavaRDD(sc, mean, m, p); JavaDoubleRDD rdd3 = poissonJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -107,7 +108,7 @@ public void testExponentialRDD() { JavaDoubleRDD rdd1 = exponentialJavaRDD(sc, mean, m); JavaDoubleRDD rdd2 = exponentialJavaRDD(sc, mean, m, p); JavaDoubleRDD rdd3 = exponentialJavaRDD(sc, mean, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -122,7 +123,7 @@ public void testGammaRDD() { JavaDoubleRDD rdd1 = gammaJavaRDD(sc, shape, scale, m); JavaDoubleRDD rdd2 = gammaJavaRDD(sc, shape, scale, m, p); JavaDoubleRDD rdd3 = gammaJavaRDD(sc, shape, scale, m, p, seed); - for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaDoubleRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); } } @@ -138,7 +139,7 @@ public void testUniformVectorRDD() { JavaRDD rdd1 = uniformJavaVectorRDD(sc, m, n); JavaRDD rdd2 = uniformJavaVectorRDD(sc, m, n, p); JavaRDD rdd3 = uniformJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -154,7 +155,7 @@ public void testNormalVectorRDD() { JavaRDD rdd1 = normalJavaVectorRDD(sc, m, n); JavaRDD rdd2 = normalJavaVectorRDD(sc, m, n, p); JavaRDD rdd3 = normalJavaVectorRDD(sc, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -172,7 +173,7 @@ public void testLogNormalVectorRDD() { JavaRDD rdd1 = logNormalJavaVectorRDD(sc, mean, std, m, n); JavaRDD rdd2 = logNormalJavaVectorRDD(sc, mean, std, m, n, p); JavaRDD rdd3 = logNormalJavaVectorRDD(sc, mean, std, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -189,7 +190,7 @@ public void testPoissonVectorRDD() { JavaRDD rdd1 = poissonJavaVectorRDD(sc, mean, m, n); JavaRDD rdd2 = poissonJavaVectorRDD(sc, mean, m, n, p); JavaRDD rdd3 = poissonJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -206,7 +207,7 @@ public void testExponentialVectorRDD() { JavaRDD rdd1 = exponentialJavaVectorRDD(sc, mean, m, n); JavaRDD rdd2 = exponentialJavaVectorRDD(sc, mean, m, n, p); JavaRDD rdd3 = exponentialJavaVectorRDD(sc, mean, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } @@ -224,7 +225,7 @@ public void testGammaVectorRDD() { JavaRDD rdd1 = gammaJavaVectorRDD(sc, shape, scale, m, n); JavaRDD rdd2 = gammaJavaVectorRDD(sc, shape, scale, m, n, p); JavaRDD rdd3 = gammaJavaVectorRDD(sc, shape, scale, m, n, p, seed); - for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + for (JavaRDD rdd: Arrays.asList(rdd1, rdd2, rdd3)) { Assert.assertEquals(m, rdd.count()); Assert.assertEquals(n, rdd.first().size()); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index af688c504cf1e..271dda4662e0d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -18,12 +18,12 @@ package org.apache.spark.mllib.recommendation; import java.io.Serializable; +import java.util.ArrayList; import java.util.List; import scala.Tuple2; import scala.Tuple3; -import com.google.common.collect.Lists; import org.jblas.DoubleMatrix; import org.junit.After; import org.junit.Assert; @@ -56,8 +56,7 @@ void validatePrediction( double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) { - List> localUsersProducts = - Lists.newArrayListWithCapacity(users * products); + List> localUsersProducts = new ArrayList(users * products); for (int u=0; u < users; ++u) { for (int p=0; p < products; ++p) { localUsersProducts.add(new Tuple2(u, p)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java index d38fc91ace3cf..32c2f4f3395b7 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaIsotonicRegressionSuite.java @@ -18,11 +18,12 @@ package org.apache.spark.mllib.regression; import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import scala.Tuple3; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -36,7 +37,7 @@ public class JavaIsotonicRegressionSuite implements Serializable { private transient JavaSparkContext sc; private List> generateIsotonicInput(double[] labels) { - List> input = Lists.newArrayList(); + ArrayList> input = new ArrayList(labels.length); for (int i = 1; i <= labels.length; i++) { input.add(new Tuple3(labels[i-1], (double) i, 1d)); @@ -77,7 +78,7 @@ public void testIsotonicRegressionPredictionsJavaRDD() { IsotonicRegressionModel model = runIsotonicRegression(new double[]{1, 2, 3, 3, 1, 6, 7, 8, 11, 9, 10, 12}); - JavaDoubleRDD testRDD = sc.parallelizeDoubles(Lists.newArrayList(0.0, 1.0, 9.5, 12.0, 13.0)); + JavaDoubleRDD testRDD = sc.parallelizeDoubles(Arrays.asList(0.0, 1.0, 9.5, 12.0, 13.0)); List predictions = model.predict(testRDD).collect(); Assert.assertTrue(predictions.get(0) == 1d); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index 899c4ea607869..dbf6488d41085 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -18,11 +18,11 @@ package org.apache.spark.mllib.regression; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import scala.Tuple2; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -59,16 +59,16 @@ public void tearDown() { @Test @SuppressWarnings("unchecked") public void javaAPI() { - List trainingBatch = Lists.newArrayList( + List trainingBatch = Arrays.asList( new LabeledPoint(1.0, Vectors.dense(1.0)), new LabeledPoint(0.0, Vectors.dense(0.0))); JavaDStream training = - attachTestInputStream(ssc, Lists.newArrayList(trainingBatch, trainingBatch), 2); - List> testBatch = Lists.newArrayList( + attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2); + List> testBatch = Arrays.asList( new Tuple2(10, Vectors.dense(1.0)), new Tuple2(11, Vectors.dense(0.0))); JavaPairDStream test = JavaPairDStream.fromJavaDStream( - attachTestInputStream(ssc, Lists.newArrayList(testBatch, testBatch), 2)); + attachTestInputStream(ssc, Arrays.asList(testBatch, testBatch), 2)); StreamingLinearRegressionWithSGD slr = new StreamingLinearRegressionWithSGD() .setNumIterations(2) .setInitialWeights(Vectors.dense(0.0)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index eb4e3698624bc..4795809e47a46 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -19,7 +19,8 @@ import java.io.Serializable; -import com.google.common.collect.Lists; +import java.util.Arrays; + import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -50,8 +51,8 @@ public void tearDown() { @Test public void testCorr() { - JavaRDD x = sc.parallelize(Lists.newArrayList(1.0, 2.0, 3.0, 4.0)); - JavaRDD y = sc.parallelize(Lists.newArrayList(1.1, 2.2, 3.1, 4.3)); + JavaRDD x = sc.parallelize(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + JavaRDD y = sc.parallelize(Arrays.asList(1.1, 2.2, 3.1, 4.3)); Double corr1 = Statistics.corr(x, y); Double corr2 = Statistics.corr(x, y, "pearson"); @@ -61,7 +62,7 @@ public void testCorr() { @Test public void kolmogorovSmirnovTest() { - JavaDoubleRDD data = sc.parallelizeDoubles(Lists.newArrayList(0.2, 1.0, -1.0, 2.0)); + JavaDoubleRDD data = sc.parallelizeDoubles(Arrays.asList(0.2, 1.0, -1.0, 2.0)); KolmogorovSmirnovTestResult testResult1 = Statistics.kolmogorovSmirnovTest(data, "norm"); KolmogorovSmirnovTestResult testResult2 = Statistics.kolmogorovSmirnovTest( data, "norm", 0.0, 1.0); @@ -69,7 +70,7 @@ public void kolmogorovSmirnovTest() { @Test public void chiSqTest() { - JavaRDD data = sc.parallelize(Lists.newArrayList( + JavaRDD data = sc.parallelize(Arrays.asList( new LabeledPoint(0.0, Vectors.dense(0.1, 2.3)), new LabeledPoint(1.0, Vectors.dense(1.5, 5.1)), new LabeledPoint(0.0, Vectors.dense(2.4, 8.1)))); From fdd466bed7a7151dd066d732ef98d225f4acda4a Mon Sep 17 00:00:00 2001 From: Vyacheslav Baranov Date: Thu, 27 Aug 2015 18:56:18 +0100 Subject: [PATCH 118/232] [SPARK-10182] [MLLIB] GeneralizedLinearModel doesn't unpersist cached data `GeneralizedLinearModel` creates a cached RDD when building a model. It's inconvenient, since these RDDs flood the memory when building several models in a row, so useful data might get evicted from the cache. The proposed solution is to always cache the dataset & remove the warning. There's a caveat though: input dataset gets evaluated twice, in line 270 when fitting `StandardScaler` for the first time, and when running optimizer for the second time. So, it might worth to return removed warning. Another possible solution is to disable caching entirely & return removed warning. I don't really know what approach is better. Author: Vyacheslav Baranov Closes #8395 from SlavikBaranov/SPARK-10182. --- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 7e3b4d5648fe3..8f657bfb9c730 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -359,6 +359,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] + " parent RDDs are also uncached.") } + // Unpersist cached data + if (data.getStorageLevel != StorageLevel.NONE) { + data.unpersist(false) + } + createModel(weights, intercept) } } From dc86a227e4fc8a9d8c3e8c68da8dff9298447fd0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Aug 2015 11:45:15 -0700 Subject: [PATCH 119/232] [SPARK-9148] [SPARK-10252] [SQL] Update SQL Programming Guide Author: Michael Armbrust Closes #8441 from marmbrus/documentation. --- docs/sql-programming-guide.md | 92 +++++++++++++++++++++++++++-------- 1 file changed, 73 insertions(+), 19 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e64190b9b209d..99fec6c7785af 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -11,7 +11,7 @@ title: Spark SQL and DataFrames Spark SQL is a Spark module for structured data processing. It provides a programming abstraction called DataFrames and can also act as distributed SQL query engine. -For how to enable Hive support, please refer to the [Hive Tables](#hive-tables) section. +Spark SQL can also be used to read data from an existing Hive installation. For more on how to configure this feature, please refer to the [Hive Tables](#hive-tables) section. # DataFrames @@ -213,6 +213,11 @@ df.groupBy("age").count().show() // 30 1 {% endhighlight %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.DataFrame). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.DataFrame). + +
@@ -263,6 +268,10 @@ df.groupBy("age").count().show(); // 30 1 {% endhighlight %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/java/org/apache/spark/sql/DataFrame.html). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +
@@ -320,6 +329,10 @@ df.groupBy("age").count().show() {% endhighlight %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). +
@@ -370,10 +383,13 @@ showDF(count(groupBy(df, "age"))) {% endhighlight %} -
+For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/index.html).
+ ## Running SQL Queries Programmatically @@ -870,12 +886,11 @@ saveDF(select(df, "name", "age"), "namesAndAges.parquet", "parquet") Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Thus, it is not safe to have multiple writers attempting to write to the same location. -Additionally, when performing a `Overwrite`, the data will be deleted before writing out the +atomic. Additionally, when performing a `Overwrite`, the data will be deleted before writing out the new data. - + @@ -1671,12 +1686,12 @@ results <- collect(sql(sqlContext, "FROM src SELECT key, value")) ### Interacting with Different Versions of Hive Metastore One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, -which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. +which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary +build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. +Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL +will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). -Internally, Spark SQL uses two Hive clients, one for executing native Hive commands like `SET` -and `DESCRIBE`, the other dedicated for communicating with Hive metastore. The former uses Hive -jars of version 0.13.1, which are bundled with Spark 1.4.0. The latter uses Hive jars of the -version specified by users. An isolated classloader is used here to avoid dependency conflicts. +The following options can be used to configure the version of Hive that is used to retrieve metadata:
Scala/JavaPythonMeaning
Scala/JavaAny LanguageMeaning
SaveMode.ErrorIfExists (default) "error" (default)
@@ -1685,7 +1700,7 @@ version specified by users. An isolated classloader is used here to avoid depend @@ -1696,12 +1711,16 @@ version specified by users. An isolated classloader is used here to avoid depend property can be one of three options:
  1. builtin
  2. - Use Hive 0.13.1, which is bundled with the Spark assembly jar when -Phive is + Use Hive 1.2.1, which is bundled with the Spark assembly jar when -Phive is enabled. When this option is chosen, spark.sql.hive.metastore.version must be - either 0.13.1 or not defined. + either 1.2.1 or not defined.
  3. maven
  4. - Use Hive jars of specified version downloaded from Maven repositories. -
  5. A classpath in the standard format for both Hive and Hadoop.
  6. + Use Hive jars of specified version downloaded from Maven repositories. This configuration + is not generally recommended for production deployments. +
  7. A classpath in the standard format for the JVM. This classpath must include all of Hive + and its dependencies, including the correct version of Hadoop. These jars only need to be + present on the driver, but if you are running in yarn cluster mode then you must ensure + they are packaged with you application.
@@ -2017,6 +2036,28 @@ options. # Migration Guide +## Upgrading From Spark SQL 1.4 to 1.5 + + - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false. + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + - Timestamps are now stored at a precision of 1us, rather than 1ns + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + - The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM). + - It has been determined that using the DirectOutputCommitter when speculation is enabled is unsafe + and thus this output committer will not be used when speculation is on, independent of configuration. + ## Upgrading from Spark SQL 1.3 to 1.4 #### DataFrame data reader/writer interface @@ -2038,7 +2079,8 @@ See the API docs for `SQLContext.read` ( #### DataFrame.groupBy retains grouping columns -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`.
@@ -2175,7 +2217,7 @@ Python UDF registration is unchanged. When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of referencing a singleton. -## Migration Guide for Shark User +## Migration Guide for Shark Users ### Scheduling To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, @@ -2251,6 +2293,7 @@ Spark SQL supports the vast majority of Hive features, such as: * User defined functions (UDF) * User defined aggregation functions (UDAF) * User defined serialization formats (SerDes) +* Window functions * Joins * `JOIN` * `{LEFT|RIGHT|FULL} OUTER JOIN` @@ -2261,7 +2304,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `SELECT col FROM ( SELECT a + b AS col from t1) t2` * Sampling * Explain -* Partitioned tables +* Partitioned tables including dynamic partition insertion * View * All Hive DDL Functions, including: * `CREATE TABLE` @@ -2323,8 +2366,9 @@ releases of Spark SQL. Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that. +# Reference -# Data Types +## Data Types Spark SQL and DataFrames support the following data types: @@ -2937,3 +2981,13 @@ from pyspark.sql.types import *
+## NaN Semantics + +There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that +does not exactly match standard floating point semantics. +Specifically: + + - NaN = NaN returns true. + - In aggregations all NaN values are grouped together. + - NaN is treated as a normal value in join keys. + - NaN values go last when in ascending order, larger than any other numeric value. From 84baa5e9b5edc8c55871fbed5057324450bf097f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 27 Aug 2015 20:19:09 +0100 Subject: [PATCH 120/232] [SPARK-10315] remove document on spark.akka.failure-detector.threshold https://issues.apache.org/jira/browse/SPARK-10315 this parameter is not used any longer and there is some mistake in the current document , should be 'akka.remote.watch-failure-detector.threshold' Author: CodingCat Closes #8483 from CodingCat/SPARK_10315. --- docs/configuration.md | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4a6e4dd05b661..77c5cbc7b3196 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -906,16 +906,6 @@ Apart from these, the following properties are also available, and may be useful #### Networking
Property NameDefaultMeaning
0.13.1 Version of the Hive metastore. Available - options are 0.12.0 and 0.13.1. Support for more versions is coming in the future. + options are 0.12.0 through 1.2.1.
- - - - - From 6185cdd2afcd492b77ff225b477b3624e3bc7bb2 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 27 Aug 2015 13:57:20 -0700 Subject: [PATCH 121/232] [SPARK-9901] User guide for RowMatrix Tall-and-skinny QR jira: https://issues.apache.org/jira/browse/SPARK-9901 The jira covers only the document update. I can further provide example code for QR (like the ones for SVD and PCA) in a separate PR. Author: Yuhao Yang Closes #8462 from hhbyyh/qrDoc. --- docs/mllib-data-types.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index f0e8d5495675d..065bf4727624f 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -337,7 +337,10 @@ limited by the integer range but it should be much smaller in practice.
A [`RowMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be -created from an `RDD[Vector]` instance. Then we can compute its column summary statistics. +created from an `RDD[Vector]` instance. Then we can compute its column summary statistics and decompositions. +[QR decomposition](https://en.wikipedia.org/wiki/QR_decomposition) is of the form A = QR where Q is an orthogonal matrix and R is an upper triangular matrix. +For [singular value decomposition (SVD)](https://en.wikipedia.org/wiki/Singular_value_decomposition) and [principal component analysis (PCA)](https://en.wikipedia.org/wiki/Principal_component_analysis), please refer to [Dimensionality reduction](mllib-dimensionality-reduction.html). + {% highlight scala %} import org.apache.spark.mllib.linalg.Vector @@ -350,6 +353,9 @@ val mat: RowMatrix = new RowMatrix(rows) // Get its size. val m = mat.numRows() val n = mat.numCols() + +// QR decomposition +val qrResult = mat.tallSkinnyQR(true) {% endhighlight %}
@@ -370,6 +376,9 @@ RowMatrix mat = new RowMatrix(rows.rdd()); // Get its size. long m = mat.numRows(); long n = mat.numCols(); + +// QR decomposition +QRDecomposition result = mat.tallSkinnyQR(true); {% endhighlight %} From c94ecdfc5b3c0fe6c38a170dc2af9259354dc9e3 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 27 Aug 2015 15:33:43 -0700 Subject: [PATCH 122/232] [SPARK-9906] [ML] User guide for LogisticRegressionSummary User guide for LogisticRegression summaries Author: MechCoder Author: Manoj Kumar Author: Feynman Liang Closes #8197 from MechCoder/log_summary_user_guide. --- docs/ml-linear-methods.md | 149 ++++++++++++++++++++++++++++++++++---- 1 file changed, 133 insertions(+), 16 deletions(-) diff --git a/docs/ml-linear-methods.md b/docs/ml-linear-methods.md index 1ac83d94c9e81..2761aeb789621 100644 --- a/docs/ml-linear-methods.md +++ b/docs/ml-linear-methods.md @@ -23,20 +23,41 @@ displayTitle: ML - Linear Methods \]` -In MLlib, we implement popular linear methods such as logistic regression and linear least squares with L1 or L2 regularization. Refer to [the linear methods in mllib](mllib-linear-methods.html) for details. In `spark.ml`, we also include Pipelines API for [Elastic net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid of L1 and L2 regularization proposed in [this paper](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). Mathematically it is defined as a linear combination of the L1-norm and the L2-norm: +In MLlib, we implement popular linear methods such as logistic +regression and linear least squares with $L_1$ or $L_2$ regularization. +Refer to [the linear methods in mllib](mllib-linear-methods.html) for +details. In `spark.ml`, we also include Pipelines API for [Elastic +net](http://en.wikipedia.org/wiki/Elastic_net_regularization), a hybrid +of $L_1$ and $L_2$ regularization proposed in [Zou et al, Regularization +and variable selection via the elastic +net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). +Mathematically, it is defined as a convex combination of the $L_1$ and +the $L_2$ regularization terms: `\[ -\alpha \|\wv\|_1 + (1-\alpha) \frac{1}{2}\|\wv\|_2^2, \alpha \in [0, 1]. +\alpha~\lambda \|\wv\|_1 + (1-\alpha) \frac{\lambda}{2}\|\wv\|_2^2, \alpha \in [0, 1], \lambda \geq 0. \]` -By setting $\alpha$ properly, it contains both L1 and L2 regularization as special cases. For example, if a [linear regression](https://en.wikipedia.org/wiki/Linear_regression) model is trained with the elastic net parameter $\alpha$ set to $1$, it is equivalent to a [Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. On the other hand, if $\alpha$ is set to $0$, the trained model reduces to a [ridge regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. We implement Pipelines API for both linear regression and logistic regression with elastic net regularization. - -**Examples** +By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ +regularization as special cases. For example, if a [linear +regression](https://en.wikipedia.org/wiki/Linear_regression) model is +trained with the elastic net parameter $\alpha$ set to $1$, it is +equivalent to a +[Lasso](http://en.wikipedia.org/wiki/Least_squares#Lasso_method) model. +On the other hand, if $\alpha$ is set to $0$, the trained model reduces +to a [ridge +regression](http://en.wikipedia.org/wiki/Tikhonov_regularization) model. +We implement Pipelines API for both linear regression and logistic +regression with elastic net regularization. + +## Example: Logistic Regression + +The following example shows how to train a logistic regression model +with elastic net regularization. `elasticNetParam` corresponds to +$\alpha$ and `regParam` corresponds to $\lambda$.
- {% highlight scala %} - import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.mllib.util.MLUtils @@ -53,15 +74,11 @@ val lrModel = lr.fit(training) // Print the weights and intercept for logistic regression println(s"Weights: ${lrModel.weights} Intercept: ${lrModel.intercept}") - {% endhighlight %} -
- {% highlight java %} - import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.mllib.regression.LabeledPoint; @@ -99,9 +116,7 @@ public class LogisticRegressionWithElasticNetExample {
- {% highlight python %} - from pyspark.ml.classification import LogisticRegression from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.util import MLUtils @@ -118,12 +133,114 @@ lrModel = lr.fit(training) print("Weights: " + str(lrModel.weights)) print("Intercept: " + str(lrModel.intercept)) {% endhighlight %} +
+The `spark.ml` implementation of logistic regression also supports +extracting a summary of the model over the training set. Note that the +predictions and metrics which are stored as `Dataframe` in +`BinaryLogisticRegressionSummary` are annotated `@transient` and hence +only available on the driver. + +
+ +
+ +[`LogisticRegressionTrainingSummary`](api/scala/index.html#org.apache.spark.ml.classification.LogisticRegressionTrainingSummary) +provides a summary for a +[`LogisticRegressionModel`](api/scala/index.html#org.apache.spark.ml.classification.LogisticRegressionModel). +Currently, only binary classification is supported and the +summary must be explicitly cast to +[`BinaryLogisticRegressionTrainingSummary`](api/scala/index.html#org.apache.spark.ml.classification.BinaryLogisticRegressionTrainingSummary). +This will likely change when multiclass classification is supported. + +Continuing the earlier example: + +{% highlight scala %} +// Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example +val trainingSummary = lrModel.summary + +// Obtain the loss per iteration. +val objectiveHistory = trainingSummary.objectiveHistory +objectiveHistory.foreach(loss => println(loss)) + +// Obtain the metrics useful to judge performance on test data. +// We cast the summary to a BinaryLogisticRegressionSummary since the problem is a +// binary classification problem. +val binarySummary = trainingSummary.asInstanceOf[BinaryLogisticRegressionSummary] + +// Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. +val roc = binarySummary.roc +roc.show() +roc.select("FPR").show() +println(binarySummary.areaUnderROC) + +// Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with +// this selected threshold. +val fMeasure = binarySummary.fMeasureByThreshold +val maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0) +val bestThreshold = fMeasure.where($"F-Measure" === maxFMeasure). + select("threshold").head().getDouble(0) +logReg.setThreshold(bestThreshold) +logReg.fit(logRegDataFrame) +{% endhighlight %}
-### Optimization +
+[`LogisticRegressionTrainingSummary`](api/java/org/apache/spark/ml/classification/LogisticRegressionTrainingSummary.html) +provides a summary for a +[`LogisticRegressionModel`](api/java/org/apache/spark/ml/classification/LogisticRegressionModel.html). +Currently, only binary classification is supported and the +summary must be explicitly cast to +[`BinaryLogisticRegressionTrainingSummary`](api/java/org/apache/spark/ml/classification/BinaryLogisticRegressionTrainingSummary.html). +This will likely change when multiclass classification is supported. + +Continuing the earlier example: + +{% highlight java %} +// Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example +LogisticRegressionTrainingSummary trainingSummary = logRegModel.summary(); + +// Obtain the loss per iteration. +double[] objectiveHistory = trainingSummary.objectiveHistory(); +for (double lossPerIteration : objectiveHistory) { + System.out.println(lossPerIteration); +} + +// Obtain the metrics useful to judge performance on test data. +// We cast the summary to a BinaryLogisticRegressionSummary since the problem is a +// binary classification problem. +BinaryLogisticRegressionSummary binarySummary = (BinaryLogisticRegressionSummary) trainingSummary; + +// Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. +DataFrame roc = binarySummary.roc(); +roc.show(); +roc.select("FPR").show(); +System.out.println(binarySummary.areaUnderROC()); + +// Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with +// this selected threshold. +DataFrame fMeasure = binarySummary.fMeasureByThreshold(); +double maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0); +double bestThreshold = fMeasure.where(fMeasure.col("F-Measure").equalTo(maxFMeasure)). + select("threshold").head().getDouble(0); +logReg.setThreshold(bestThreshold); +logReg.fit(logRegDataFrame); +{% endhighlight %} +
+ +
+Logistic regression model summary is not yet supported in Python. +
+ +
+ +# Optimization + +The optimization algorithm underlying the implementation is called +[Orthant-Wise Limited-memory +QuasiNewton](http://research-srv.microsoft.com/en-us/um/people/jfgao/paper/icml07scalable.pdf) +(OWL-QN). It is an extension of L-BFGS that can effectively handle L1 +regularization and elastic net. -The optimization algorithm underlies the implementation is called [Orthant-Wise Limited-memory QuasiNewton](http://research-srv.microsoft.com/en-us/um/people/jfgao/paper/icml07scalable.pdf) -(OWL-QN). It is an extension of L-BFGS that can effectively handle L1 regularization and elastic net. From 5bfe9e1111d9862084586549a7dc79476f67bab9 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 16:10:37 -0700 Subject: [PATCH 123/232] [SPARK-9680] [MLLIB] [DOC] StopWordsRemovers user guide and Java compatibility test * Adds user guide for ml.feature.StopWordsRemovers, ran code examples on my machine * Cleans up scaladocs for public methods * Adds test for Java compatibility * Follow up Python user guide code example is tracked by SPARK-10249 Author: Feynman Liang Closes #8436 from feynmanliang/SPARK-10230. --- docs/ml-features.md | 102 +++++++++++++++++- .../ml/feature/JavaStopWordsRemoverSuite.java | 72 +++++++++++++ 2 files changed, 171 insertions(+), 3 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java diff --git a/docs/ml-features.md b/docs/ml-features.md index 62de4838981cb..89a9bad570446 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -306,15 +306,111 @@ regexTokenizer = RegexTokenizer(inputCol="sentence", outputCol="words", pattern= +## StopWordsRemover +[Stop words](https://en.wikipedia.org/wiki/Stop_words) are words which +should be excluded from the input, typically because the words appear +frequently and don't carry as much meaning. + +`StopWordsRemover` takes as input a sequence of strings (e.g. the output +of a [Tokenizer](ml-features.html#tokenizer)) and drops all the stop +words from the input sequences. The list of stopwords is specified by +the `stopWords` parameter. We provide [a list of stop +words](http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words) by +default, accessible by calling `getStopWords` on a newly instantiated +`StopWordsRemover` instance. -## $n$-gram +**Examples** -An [n-gram](https://en.wikipedia.org/wiki/N-gram) is a sequence of $n$ tokens (typically words) for some integer $n$. The `NGram` class can be used to transform input features into $n$-grams. +Assume that we have the following DataFrame with columns `id` and `raw`: -`NGram` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer). The parameter `n` is used to determine the number of terms in each $n$-gram. The output will consist of a sequence of $n$-grams where each $n$-gram is represented by a space-delimited string of $n$ consecutive words. If the input sequence contains fewer than `n` strings, no output is produced. +~~~~ + id | raw +----|---------- + 0 | [I, saw, the, red, baloon] + 1 | [Mary, had, a, little, lamb] +~~~~ + +Applying `StopWordsRemover` with `raw` as the input column and `filtered` as the output +column, we should get the following: + +~~~~ + id | raw | filtered +----|-----------------------------|-------------------- + 0 | [I, saw, the, red, baloon] | [saw, red, baloon] + 1 | [Mary, had, a, little, lamb]|[Mary, little, lamb] +~~~~ + +In `filtered`, the stop words "I", "the", "had", and "a" have been +filtered out.
+
+ +[`StopWordsRemover`](api/scala/index.html#org.apache.spark.ml.feature.StopWordsRemover) +takes an input column name, an output column name, a list of stop words, +and a boolean indicating if the matches should be case sensitive (false +by default). + +{% highlight scala %} +import org.apache.spark.ml.feature.StopWordsRemover + +val remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered") +val dataSet = sqlContext.createDataFrame(Seq( + (0, Seq("I", "saw", "the", "red", "baloon")), + (1, Seq("Mary", "had", "a", "little", "lamb")) +)).toDF("id", "raw") + +remover.transform(dataSet).show() +{% endhighlight %} +
+ +
+ +[`StopWordsRemover`](api/java/org/apache/spark/ml/feature/StopWordsRemover.html) +takes an input column name, an output column name, a list of stop words, +and a boolean indicating if the matches should be case sensitive (false +by default). + +{% highlight java %} +import java.util.Arrays; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.StopWordsRemover; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +StopWordsRemover remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered"); + +JavaRDD rdd = jsc.parallelize(Arrays.asList( + RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), + RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) +)); +StructType schema = new StructType(new StructField[] { + new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) +}); +DataFrame dataset = jsql.createDataFrame(rdd, schema); + +remover.transform(dataset).show(); +{% endhighlight %} +
+
+ +## $n$-gram + +An [n-gram](https://en.wikipedia.org/wiki/N-gram) is a sequence of $n$ tokens (typically words) for some integer $n$. The `NGram` class can be used to transform input features into $n$-grams. + +`NGram` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer)). The parameter `n` is used to determine the number of terms in each $n$-gram. The output will consist of a sequence of $n$-grams where each $n$-gram is represented by a space-delimited string of $n$ consecutive words. If the input sequence contains fewer than `n` strings, no output is produced. +
diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java new file mode 100644 index 0000000000000..76cdd0fae84ab --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -0,0 +1,72 @@ +/* + * 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.ml.feature; + +import java.util.Arrays; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + + +public class JavaStopWordsRemoverSuite { + + private transient JavaSparkContext jsc; + private transient SQLContext jsql; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaStopWordsRemoverSuite"); + jsql = new SQLContext(jsc); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void javaCompatibilityTest() { + StopWordsRemover remover = new StopWordsRemover() + .setInputCol("raw") + .setOutputCol("filtered"); + + JavaRDD rdd = jsc.parallelize(Arrays.asList( + RowFactory.create(Arrays.asList("I", "saw", "the", "red", "baloon")), + RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) + )); + StructType schema = new StructType(new StructField[] { + new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) + }); + DataFrame dataset = jsql.createDataFrame(rdd, schema); + + remover.transform(dataset).collect(); + } +} From b3dd569ad40905f8861a547a1e25ed3ca8e1d272 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 27 Aug 2015 16:11:25 -0700 Subject: [PATCH 124/232] [SPARK-10287] [SQL] Fixes JSONRelation refreshing on read path https://issues.apache.org/jira/browse/SPARK-10287 After porting json to HadoopFsRelation, it seems hard to keep the behavior of picking up new files automatically for JSON. This PR removes this behavior, so JSON is consistent with others (ORC and Parquet). Author: Yin Huai Closes #8469 from yhuai/jsonRefresh. --- docs/sql-programming-guide.md | 6 ++++++ .../execution/datasources/json/JSONRelation.scala | 9 --------- .../org/apache/spark/sql/sources/interfaces.scala | 2 +- .../apache/spark/sql/sources/InsertSuite.scala | 15 --------------- 4 files changed, 7 insertions(+), 25 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 99fec6c7785af..e8eb88488ee24 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2057,6 +2057,12 @@ options. - The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM). - It has been determined that using the DirectOutputCommitter when speculation is enabled is unsafe and thus this output committer will not be used when speculation is on, independent of configuration. + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. ## Upgrading from Spark SQL 1.3 to 1.4 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index 114c8b211891e..ab8ca5f748f24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -111,15 +111,6 @@ private[sql] class JSONRelation( jsonSchema } - override private[sql] def buildScan( - requiredColumns: Array[String], - filters: Array[Filter], - inputPaths: Array[String], - broadcastedConf: Broadcast[SerializableConfiguration]): RDD[Row] = { - refresh() - super.buildScan(requiredColumns, filters, inputPaths, broadcastedConf) - } - override def buildScan( requiredColumns: Array[String], filters: Array[Filter], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index b3b326fe612c7..dff726b33fc74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -562,7 +562,7 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio }) } - private[sql] def buildScan( + final private[sql] def buildScan( requiredColumns: Array[String], filters: Array[Filter], inputPaths: Array[String], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 78bd3e5582964..084d83f6e9bff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -167,21 +167,6 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { ) } - test("save directly to the path of a JSON table") { - caseInsensitiveContext.table("jt").selectExpr("a * 5 as a", "b") - .write.mode(SaveMode.Overwrite).json(path.toString) - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - (1 to 10).map(i => Row(i * 5, s"str$i")) - ) - - caseInsensitiveContext.table("jt").write.mode(SaveMode.Overwrite).json(path.toString) - checkAnswer( - sql("SELECT a, b FROM jsonTable"), - (1 to 10).map(i => Row(i, s"str$i")) - ) - } - test("it is not allowed to write to a table while querying it.") { val message = intercept[AnalysisException] { sql( From 54cda0deb6bebf1470f16ba5bcc6c4fb842bdac1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 27 Aug 2015 16:38:00 -0700 Subject: [PATCH 125/232] [SPARK-10321] sizeInBytes in HadoopFsRelation Having sizeInBytes in HadoopFsRelation to enable broadcast join. cc marmbrus Author: Davies Liu Closes #8490 from davies/sizeInByte. --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index dff726b33fc74..7b030b7d73bd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -518,6 +518,8 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio override def inputFiles: Array[String] = cachedLeafStatuses().map(_.getPath.toString).toArray + override def sizeInBytes: Long = cachedLeafStatuses().map(_.getLen).sum + /** * Partition columns. Can be either defined by [[userDefinedPartitionColumns]] or automatically * discovered. Note that they should always be nullable. From 1f90c5e2198bcf49e115d97ec300c17c1be4dcb4 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Thu, 27 Aug 2015 19:38:53 -0700 Subject: [PATCH 126/232] [SPARK-8505] [SPARKR] Add settings to kick `lint-r` from `./dev/run-test.py` JoshRosen we'd like to check the SparkR source code with the `dev/lint-r` script on the Jenkins. I tried to incorporate the script into `dev/run-test.py`. Could you review it when you have time? shivaram I modified `dev/lint-r` and `dev/lint-r.R` to install lintr package into a local directory(`R/lib/`) and to exit with a lint status. Could you review it? - [[SPARK-8505] Add settings to kick `lint-r` from `./dev/run-test.py` - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-8505) Author: Yu ISHIKAWA Closes #7883 from yu-iskw/SPARK-8505. --- dev/lint-r | 11 +++++++++++ dev/lint-r.R | 12 +++++++----- dev/run-tests-codes.sh | 13 +++++++------ dev/run-tests-jenkins | 2 ++ dev/run-tests.py | 21 ++++++++++++++++++++- 5 files changed, 47 insertions(+), 12 deletions(-) diff --git a/dev/lint-r b/dev/lint-r index 7d5f4cd31153d..c15d57aad86da 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -28,3 +28,14 @@ if ! type "Rscript" > /dev/null; then fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" + +NUM_LINES=`wc -l < "$LINT_R_REPORT_FILE_NAME"` +if [ "$NUM_LINES" = "0" ] ; then + lint_status=0 + echo "lintr checks passed." +else + lint_status=1 + echo "lintr checks failed." +fi + +exit "$lint_status" diff --git a/dev/lint-r.R b/dev/lint-r.R index 48bd6246096ae..999eef571b824 100644 --- a/dev/lint-r.R +++ b/dev/lint-r.R @@ -17,8 +17,14 @@ argv <- commandArgs(TRUE) SPARK_ROOT_DIR <- as.character(argv[1]) +LOCAL_LIB_LOC <- file.path(SPARK_ROOT_DIR, "R", "lib") -# Installs lintr from Github. +# Checks if SparkR is installed in a local directory. +if (! library(SparkR, lib.loc = LOCAL_LIB_LOC, logical.return = TRUE)) { + stop("You should install SparkR in a local directory with `R/install-dev.sh`.") +} + +# Installs lintr from Github in a local directory. # NOTE: The CRAN's version is too old to adapt to our rules. if ("lintr" %in% row.names(installed.packages()) == FALSE) { devtools::install_github("jimhester/lintr") @@ -27,9 +33,5 @@ if ("lintr" %in% row.names(installed.packages()) == FALSE) { library(lintr) library(methods) library(testthat) -if (! library(SparkR, lib.loc = file.path(SPARK_ROOT_DIR, "R", "lib"), logical.return = TRUE)) { - stop("You should install SparkR in a local directory with `R/install-dev.sh`.") -} - path.to.package <- file.path(SPARK_ROOT_DIR, "R", "pkg") lint_package(path.to.package, cache = FALSE) diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh index f4b238e1b78a7..1f16790522e76 100644 --- a/dev/run-tests-codes.sh +++ b/dev/run-tests-codes.sh @@ -21,9 +21,10 @@ readonly BLOCK_GENERAL=10 readonly BLOCK_RAT=11 readonly BLOCK_SCALA_STYLE=12 readonly BLOCK_PYTHON_STYLE=13 -readonly BLOCK_DOCUMENTATION=14 -readonly BLOCK_BUILD=15 -readonly BLOCK_MIMA=16 -readonly BLOCK_SPARK_UNIT_TESTS=17 -readonly BLOCK_PYSPARK_UNIT_TESTS=18 -readonly BLOCK_SPARKR_UNIT_TESTS=19 +readonly BLOCK_R_STYLE=14 +readonly BLOCK_DOCUMENTATION=15 +readonly BLOCK_BUILD=16 +readonly BLOCK_MIMA=17 +readonly BLOCK_SPARK_UNIT_TESTS=18 +readonly BLOCK_PYSPARK_UNIT_TESTS=19 +readonly BLOCK_SPARKR_UNIT_TESTS=20 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f144c053046c5..39cf54f78104c 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -210,6 +210,8 @@ done failing_test="Scala style tests" elif [ "$test_result" -eq "$BLOCK_PYTHON_STYLE" ]; then failing_test="Python style tests" + elif [ "$test_result" -eq "$BLOCK_R_STYLE" ]; then + failing_test="R style tests" elif [ "$test_result" -eq "$BLOCK_DOCUMENTATION" ]; then failing_test="to generate documentation" elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then diff --git a/dev/run-tests.py b/dev/run-tests.py index f689425ee40b6..4fd703a7c219f 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -209,6 +209,18 @@ def run_python_style_checks(): run_cmd([os.path.join(SPARK_HOME, "dev", "lint-python")]) +def run_sparkr_style_checks(): + set_title_and_block("Running R style checks", "BLOCK_R_STYLE") + + if which("R"): + # R style check should be executed after `install-dev.sh`. + # Since warnings about `no visible global function definition` appear + # without the installation. SEE ALSO: SPARK-9121. + run_cmd([os.path.join(SPARK_HOME, "dev", "lint-r")]) + else: + print("Ignoring SparkR style check as R was not found in PATH") + + def build_spark_documentation(): set_title_and_block("Building Spark Documentation", "BLOCK_DOCUMENTATION") os.environ["PRODUCTION"] = "1 jekyll build" @@ -387,7 +399,6 @@ def run_sparkr_tests(): set_title_and_block("Running SparkR tests", "BLOCK_SPARKR_UNIT_TESTS") if which("R"): - run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) run_cmd([os.path.join(SPARK_HOME, "R", "run-tests.sh")]) else: print("Ignoring SparkR tests as R was not found in PATH") @@ -438,6 +449,12 @@ def main(): if java_version.minor < 8: print("[warn] Java 8 tests will not run because JDK version is < 1.8.") + # install SparkR + if which("R"): + run_cmd([os.path.join(SPARK_HOME, "R", "install-dev.sh")]) + else: + print("Can't install SparkR as R is was not found in PATH") + if os.environ.get("AMPLAB_JENKINS"): # if we're on the Amplab Jenkins build servers setup variables # to reflect the environment settings @@ -485,6 +502,8 @@ def main(): run_scala_style_checks() if not changed_files or any(f.endswith(".py") for f in changed_files): run_python_style_checks() + if not changed_files or any(f.endswith(".R") for f in changed_files): + run_sparkr_style_checks() # determine if docs were changed and if we're inside the amplab environment # note - the below commented out until *all* Jenkins workers can get `jekyll` installed From 30734d45fbbb269437c062241a9161e198805a76 Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 27 Aug 2015 21:44:06 -0700 Subject: [PATCH 127/232] [SPARK-9911] [DOC] [ML] Update Userguide for Evaluator I added a small note about the different types of evaluator and the metrics used. Author: MechCoder Closes #8304 from MechCoder/multiclass_evaluator. --- docs/ml-guide.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index de8fead3529e4..01bf5ee18e328 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -643,6 +643,13 @@ An important task in ML is *model selection*, or using data to find the best mod Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) class, which takes an `Estimator`, a set of `ParamMap`s, and an [`Evaluator`](api/scala/index.html#org.apache.spark.ml.Evaluator). `CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets; e.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. `CrossValidator` iterates through the set of `ParamMap`s. For each `ParamMap`, it trains the given `Estimator` and evaluates it using the given `Evaluator`. + +The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.RegressionEvaluator) +for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.BinaryClassificationEvaluator) +for binary data or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.MultiClassClassificationEvaluator) +for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the setMetric +method in each of these evaluators. + The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. `CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. @@ -708,9 +715,12 @@ val pipeline = new Pipeline() // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. // This will allow us to jointly choose parameters for all Pipeline stages. // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +// Note that the evaluator here is a BinaryClassificationEvaluator and the default metric +// used is areaUnderROC. val crossval = new CrossValidator() .setEstimator(pipeline) .setEvaluator(new BinaryClassificationEvaluator) + // We use a ParamGridBuilder to construct a grid of parameters to search over. // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. @@ -831,9 +841,12 @@ Pipeline pipeline = new Pipeline() // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. // This will allow us to jointly choose parameters for all Pipeline stages. // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +// Note that the evaluator here is a BinaryClassificationEvaluator and the default metric +// used is areaUnderROC. CrossValidator crossval = new CrossValidator() .setEstimator(pipeline) .setEvaluator(new BinaryClassificationEvaluator()); + // We use a ParamGridBuilder to construct a grid of parameters to search over. // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. From af0e1249b1c881c0fa7a921fd21fd2c27214b980 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Thu, 27 Aug 2015 21:55:20 -0700 Subject: [PATCH 128/232] [SPARK-9905] [ML] [DOC] Adds LinearRegressionSummary user guide * Adds user guide for `LinearRegressionSummary` * Fixes unresolved issues in #8197 CC jkbradley mengxr Author: Feynman Liang Closes #8491 from feynmanliang/SPARK-9905. --- docs/ml-linear-methods.md | 140 ++++++++++++++++++++++++++++++++++---- 1 file changed, 127 insertions(+), 13 deletions(-) diff --git a/docs/ml-linear-methods.md b/docs/ml-linear-methods.md index 2761aeb789621..cdd9d4999fa1b 100644 --- a/docs/ml-linear-methods.md +++ b/docs/ml-linear-methods.md @@ -34,7 +34,7 @@ net](http://users.stat.umn.edu/~zouxx019/Papers/elasticnet.pdf). Mathematically, it is defined as a convex combination of the $L_1$ and the $L_2$ regularization terms: `\[ -\alpha~\lambda \|\wv\|_1 + (1-\alpha) \frac{\lambda}{2}\|\wv\|_2^2, \alpha \in [0, 1], \lambda \geq 0. +\alpha \left( \lambda \|\wv\|_1 \right) + (1-\alpha) \left( \frac{\lambda}{2}\|\wv\|_2^2 \right) , \alpha \in [0, 1], \lambda \geq 0 \]` By setting $\alpha$ properly, elastic net contains both $L_1$ and $L_2$ regularization as special cases. For example, if a [linear @@ -95,7 +95,7 @@ public class LogisticRegressionWithElasticNetExample { SparkContext sc = new SparkContext(conf); SQLContext sql = new SQLContext(sc); - String path = "sample_libsvm_data.txt"; + String path = "data/mllib/sample_libsvm_data.txt"; // Load training data DataFrame training = sql.createDataFrame(MLUtils.loadLibSVMFile(sc, path).toJavaRDD(), LabeledPoint.class); @@ -103,7 +103,7 @@ public class LogisticRegressionWithElasticNetExample { LogisticRegression lr = new LogisticRegression() .setMaxIter(10) .setRegParam(0.3) - .setElasticNetParam(0.8) + .setElasticNetParam(0.8); // Fit the model LogisticRegressionModel lrModel = lr.fit(training); @@ -158,10 +158,12 @@ This will likely change when multiclass classification is supported. Continuing the earlier example: {% highlight scala %} +import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary + // Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example val trainingSummary = lrModel.summary -// Obtain the loss per iteration. +// Obtain the objective per iteration. val objectiveHistory = trainingSummary.objectiveHistory objectiveHistory.foreach(loss => println(loss)) @@ -173,17 +175,14 @@ val binarySummary = trainingSummary.asInstanceOf[BinaryLogisticRegressionSummary // Obtain the receiver-operating characteristic as a dataframe and areaUnderROC. val roc = binarySummary.roc roc.show() -roc.select("FPR").show() println(binarySummary.areaUnderROC) -// Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with -// this selected threshold. +// Set the model threshold to maximize F-Measure val fMeasure = binarySummary.fMeasureByThreshold val maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0) val bestThreshold = fMeasure.where($"F-Measure" === maxFMeasure). select("threshold").head().getDouble(0) -logReg.setThreshold(bestThreshold) -logReg.fit(logRegDataFrame) +lrModel.setThreshold(bestThreshold) {% endhighlight %}
@@ -199,8 +198,12 @@ This will likely change when multiclass classification is supported. Continuing the earlier example: {% highlight java %} +import org.apache.spark.ml.classification.LogisticRegressionTrainingSummary; +import org.apache.spark.ml.classification.BinaryLogisticRegressionSummary; +import org.apache.spark.sql.functions; + // Extract the summary from the returned LogisticRegressionModel instance trained in the earlier example -LogisticRegressionTrainingSummary trainingSummary = logRegModel.summary(); +LogisticRegressionTrainingSummary trainingSummary = lrModel.summary(); // Obtain the loss per iteration. double[] objectiveHistory = trainingSummary.objectiveHistory(); @@ -222,20 +225,131 @@ System.out.println(binarySummary.areaUnderROC()); // Get the threshold corresponding to the maximum F-Measure and rerun LogisticRegression with // this selected threshold. DataFrame fMeasure = binarySummary.fMeasureByThreshold(); -double maxFMeasure = fMeasure.select(max("F-Measure")).head().getDouble(0); +double maxFMeasure = fMeasure.select(functions.max("F-Measure")).head().getDouble(0); double bestThreshold = fMeasure.where(fMeasure.col("F-Measure").equalTo(maxFMeasure)). select("threshold").head().getDouble(0); -logReg.setThreshold(bestThreshold); -logReg.fit(logRegDataFrame); +lrModel.setThreshold(bestThreshold); {% endhighlight %}
+
Logistic regression model summary is not yet supported in Python.
+## Example: Linear Regression + +The interface for working with linear regression models and model +summaries is similar to the logistic regression case. The following +example demonstrates training an elastic net regularized linear +regression model and extracting model summary statistics. + +
+ +
+{% highlight scala %} +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.mllib.util.MLUtils + +// Load training data +val training = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +val lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + +// Fit the model +val lrModel = lr.fit(training) + +// Print the weights and intercept for linear regression +println(s"Weights: ${lrModel.weights} Intercept: ${lrModel.intercept}") + +// Summarize the model over the training set and print out some metrics +val trainingSummary = lrModel.summary +println(s"numIterations: ${trainingSummary.totalIterations}") +println(s"objectiveHistory: ${trainingSummary.objectiveHistory.toList}") +trainingSummary.residuals.show() +println(s"RMSE: ${trainingSummary.rootMeanSquaredError}") +println(s"r2: ${trainingSummary.r2}") +{% endhighlight %} +
+ +
+{% highlight java %} +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.regression.LinearRegressionModel; +import org.apache.spark.ml.regression.LinearRegressionTrainingSummary; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; + +public class LinearRegressionWithElasticNetExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf() + .setAppName("Linear Regression with Elastic Net Example"); + + SparkContext sc = new SparkContext(conf); + SQLContext sql = new SQLContext(sc); + String path = "data/mllib/sample_libsvm_data.txt"; + + // Load training data + DataFrame training = sql.createDataFrame(MLUtils.loadLibSVMFile(sc, path).toJavaRDD(), LabeledPoint.class); + + LinearRegression lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8); + + // Fit the model + LinearRegressionModel lrModel = lr.fit(training); + + // Print the weights and intercept for linear regression + System.out.println("Weights: " + lrModel.weights() + " Intercept: " + lrModel.intercept()); + + // Summarize the model over the training set and print out some metrics + LinearRegressionTrainingSummary trainingSummary = lrModel.summary(); + System.out.println("numIterations: " + trainingSummary.totalIterations()); + System.out.println("objectiveHistory: " + Vectors.dense(trainingSummary.objectiveHistory())); + trainingSummary.residuals().show(); + System.out.println("RMSE: " + trainingSummary.rootMeanSquaredError()); + System.out.println("r2: " + trainingSummary.r2()); + } +} +{% endhighlight %} +
+ +
+ +{% highlight python %} +from pyspark.ml.regression import LinearRegression +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.util import MLUtils + +# Load training data +training = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + +lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) + +# Fit the model +lrModel = lr.fit(training) + +# Print the weights and intercept for linear regression +print("Weights: " + str(lrModel.weights)) +print("Intercept: " + str(lrModel.intercept)) + +# Linear regression model summary is not yet supported in Python. +{% endhighlight %} +
+ +
+ # Optimization The optimization algorithm underlying the implementation is called From 89b943438512fcfb239c268b43431397de46cbcf Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 27 Aug 2015 22:30:01 -0700 Subject: [PATCH 129/232] [SPARK-SQL] [MINOR] Fixes some typos in HiveContext Author: Cheng Lian Closes #8481 from liancheng/hive-context-typo. --- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 8 ++++---- .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) 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 index c0a458fa9ab8d..2e791cea96b41 100644 --- 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 @@ -171,11 +171,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { * Overrides default Hive configurations to avoid breaking changes to Spark SQL users. * - allow SQL11 keywords to be used as identifiers */ - private[sql] def defaultOverides() = { + private[sql] def defaultOverrides() = { setConf(ConfVars.HIVE_SUPPORT_SQL11_RESERVED_KEYWORDS.varname, "false") } - defaultOverides() + defaultOverrides() /** * The copy of the Hive client that is used to retrieve metadata from the Hive MetaStore. @@ -190,8 +190,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) with Logging { // into the isolated client loader val metadataConf = new HiveConf() - val defaltWarehouseLocation = metadataConf.get("hive.metastore.warehouse.dir") - logInfo("defalt warehouse location is " + defaltWarehouseLocation) + val defaultWarehouseLocation = metadataConf.get("hive.metastore.warehouse.dir") + logInfo("default warehouse location is " + defaultWarehouseLocation) // `configure` goes second to override other settings. val allConfig = metadataConf.asScala.map(e => e.getKey -> e.getValue).toMap ++ configure diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 572eaebe81ac2..57fea5d8db343 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -434,7 +434,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { case (k, v) => metadataHive.runSqlHive(s"SET $k=$v") } - defaultOverides() + defaultOverrides() runSqlHive("USE default") From 7583681e6b0824d7eed471dc4d8fa0b2addf9ffc Mon Sep 17 00:00:00 2001 From: noelsmith Date: Thu, 27 Aug 2015 23:59:30 -0700 Subject: [PATCH 130/232] [SPARK-10188] [PYSPARK] Pyspark CrossValidator with RMSE selects incorrect model * Added isLargerBetter() method to Pyspark Evaluator to match the Scala version. * JavaEvaluator delegates isLargerBetter() to underlying Scala object. * Added check for isLargerBetter() in CrossValidator to determine whether to use argmin or argmax. * Added test cases for where smaller is better (RMSE) and larger is better (R-Squared). (This contribution is my original work and that I license the work to the project under Sparks' open source license) Author: noelsmith Closes #8399 from noel-smith/pyspark-rmse-xval-fix. --- python/pyspark/ml/evaluation.py | 12 +++++ python/pyspark/ml/tests.py | 87 +++++++++++++++++++++++++++++++++ python/pyspark/ml/tuning.py | 6 ++- 3 files changed, 104 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/evaluation.py b/python/pyspark/ml/evaluation.py index 6b0a9ffde9f42..cb3b07947e488 100644 --- a/python/pyspark/ml/evaluation.py +++ b/python/pyspark/ml/evaluation.py @@ -66,6 +66,14 @@ def evaluate(self, dataset, params=None): else: raise ValueError("Params must be a param map but got %s." % type(params)) + def isLargerBetter(self): + """ + Indicates whether the metric returned by :py:meth:`evaluate` should be maximized + (True, default) or minimized (False). + A given evaluator may support multiple metrics which may be maximized or minimized. + """ + return True + @inherit_doc class JavaEvaluator(Evaluator, JavaWrapper): @@ -85,6 +93,10 @@ def _evaluate(self, dataset): self._transfer_params_to_java() return self._java_obj.evaluate(dataset._jdf) + def isLargerBetter(self): + self._transfer_params_to_java() + return self._java_obj.isLargerBetter() + @inherit_doc class BinaryClassificationEvaluator(JavaEvaluator, HasLabelCol, HasRawPredictionCol): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index c151d21fd661a..60e4237293adc 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -32,11 +32,14 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame, SQLContext +from pyspark.sql.functions import rand +from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.param import Param, Params from pyspark.ml.param.shared import HasMaxIter, HasInputCol, HasSeed from pyspark.ml.util import keyword_only from pyspark.ml import Estimator, Model, Pipeline, Transformer from pyspark.ml.feature import * +from pyspark.ml.tuning import ParamGridBuilder, CrossValidator, CrossValidatorModel from pyspark.mllib.linalg import DenseVector @@ -264,5 +267,89 @@ def test_ngram(self): self.assertEquals(transformedDF.head().output, ["a b c d", "b c d e"]) +class HasInducedError(Params): + + def __init__(self): + super(HasInducedError, self).__init__() + self.inducedError = Param(self, "inducedError", + "Uniformly-distributed error added to feature") + + def getInducedError(self): + return self.getOrDefault(self.inducedError) + + +class InducedErrorModel(Model, HasInducedError): + + def __init__(self): + super(InducedErrorModel, self).__init__() + + def _transform(self, dataset): + return dataset.withColumn("prediction", + dataset.feature + (rand(0) * self.getInducedError())) + + +class InducedErrorEstimator(Estimator, HasInducedError): + + def __init__(self, inducedError=1.0): + super(InducedErrorEstimator, self).__init__() + self._set(inducedError=inducedError) + + def _fit(self, dataset): + model = InducedErrorModel() + self._copyValues(model) + return model + + +class CrossValidatorTests(PySparkTestCase): + + def test_fit_minimize_metric(self): + sqlContext = SQLContext(self.sc) + dataset = sqlContext.createDataFrame([ + (10, 10.0), + (50, 50.0), + (100, 100.0), + (500, 500.0)] * 10, + ["feature", "label"]) + + iee = InducedErrorEstimator() + evaluator = RegressionEvaluator(metricName="rmse") + + grid = (ParamGridBuilder() + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) + .build()) + cv = CrossValidator(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) + cvModel = cv.fit(dataset) + bestModel = cvModel.bestModel + bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + + self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), + "Best model should have zero induced error") + self.assertEqual(0.0, bestModelMetric, "Best model has RMSE of 0") + + def test_fit_maximize_metric(self): + sqlContext = SQLContext(self.sc) + dataset = sqlContext.createDataFrame([ + (10, 10.0), + (50, 50.0), + (100, 100.0), + (500, 500.0)] * 10, + ["feature", "label"]) + + iee = InducedErrorEstimator() + evaluator = RegressionEvaluator(metricName="r2") + + grid = (ParamGridBuilder() + .addGrid(iee.inducedError, [100.0, 0.0, 10000.0]) + .build()) + cv = CrossValidator(estimator=iee, estimatorParamMaps=grid, evaluator=evaluator) + cvModel = cv.fit(dataset) + bestModel = cvModel.bestModel + bestModelMetric = evaluator.evaluate(bestModel.transform(dataset)) + + self.assertEqual(0.0, bestModel.getOrDefault('inducedError'), + "Best model should have zero induced error") + self.assertEqual(1.0, bestModelMetric, "Best model has R-squared of 1") + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index dcfee6a3170ab..cae778869e9c5 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -223,7 +223,11 @@ def _fit(self, dataset): # TODO: duplicate evaluator to take extra params from input metric = eva.evaluate(model.transform(validation, epm[j])) metrics[j] += metric - bestIndex = np.argmax(metrics) + + if eva.isLargerBetter(): + bestIndex = np.argmax(metrics) + else: + bestIndex = np.argmin(metrics) bestModel = est.fit(dataset, epm[bestIndex]) return CrossValidatorModel(bestModel) From 2f99c37273c1d82e2ba39476e4429ea4aaba7ec6 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 28 Aug 2015 00:37:50 -0700 Subject: [PATCH 131/232] [SPARK-10328] [SPARKR] Fix generic for na.omit S3 function is at https://stat.ethz.ch/R-manual/R-patched/library/stats/html/na.fail.html Author: Shivaram Venkataraman Author: Shivaram Venkataraman Author: Yu ISHIKAWA Closes #8495 from shivaram/na-omit-fix. --- R/pkg/R/DataFrame.R | 6 +++--- R/pkg/R/generics.R | 2 +- R/pkg/inst/tests/test_sparkSQL.R | 23 ++++++++++++++++++++++- dev/lint-r | 2 +- 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index dd8126aebf467..74de7c81e35a6 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -1699,9 +1699,9 @@ setMethod("dropna", #' @name na.omit #' @export setMethod("na.omit", - signature(x = "DataFrame"), - function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { - dropna(x, how, minNonNulls, cols) + signature(object = "DataFrame"), + function(object, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { + dropna(object, how, minNonNulls, cols) }) #' fillna diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index a829d46c1894c..b578b8789d2c5 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -413,7 +413,7 @@ setGeneric("dropna", #' @rdname nafunctions #' @export setGeneric("na.omit", - function(x, how = c("any", "all"), minNonNulls = NULL, cols = NULL) { + function(object, ...) { standardGeneric("na.omit") }) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 4b672e115f924..933b11c8ee7e2 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -1083,7 +1083,7 @@ test_that("describe() and summarize() on a DataFrame", { expect_equal(collect(stats2)[5, "age"], "30") }) -test_that("dropna() on a DataFrame", { +test_that("dropna() and na.omit() on a DataFrame", { df <- jsonFile(sqlContext, jsonPathNa) rows <- collect(df) @@ -1092,6 +1092,8 @@ test_that("dropna() on a DataFrame", { expected <- rows[!is.na(rows$name),] actual <- collect(dropna(df, cols = "name")) expect_identical(expected, actual) + actual <- collect(na.omit(df, cols = "name")) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age),] actual <- collect(dropna(df, cols = "age")) @@ -1101,48 +1103,67 @@ test_that("dropna() on a DataFrame", { expect_identical(expected$age, actual$age) expect_identical(expected$height, actual$height) expect_identical(expected$name, actual$name) + actual <- collect(na.omit(df, cols = "age")) expected <- rows[!is.na(rows$age) & !is.na(rows$height),] actual <- collect(dropna(df, cols = c("age", "height"))) expect_identical(expected, actual) + actual <- collect(na.omit(df, cols = c("age", "height"))) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] actual <- collect(dropna(df)) expect_identical(expected, actual) + actual <- collect(na.omit(df)) + expect_identical(expected, actual) # drop with how expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] actual <- collect(dropna(df)) expect_identical(expected, actual) + actual <- collect(na.omit(df)) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age) | !is.na(rows$height) | !is.na(rows$name),] actual <- collect(dropna(df, "all")) expect_identical(expected, actual) + actual <- collect(na.omit(df, "all")) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] actual <- collect(dropna(df, "any")) expect_identical(expected, actual) + actual <- collect(na.omit(df, "any")) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age) & !is.na(rows$height),] actual <- collect(dropna(df, "any", cols = c("age", "height"))) expect_identical(expected, actual) + actual <- collect(na.omit(df, "any", cols = c("age", "height"))) + expect_identical(expected, actual) expected <- rows[!is.na(rows$age) | !is.na(rows$height),] actual <- collect(dropna(df, "all", cols = c("age", "height"))) expect_identical(expected, actual) + actual <- collect(na.omit(df, "all", cols = c("age", "height"))) + expect_identical(expected, actual) # drop with threshold expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) >= 2,] actual <- collect(dropna(df, minNonNulls = 2, cols = c("age", "height"))) expect_identical(expected, actual) + actual <- collect(na.omit(df, minNonNulls = 2, cols = c("age", "height"))) + expect_identical(expected, actual) expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) + as.integer(!is.na(rows$name)) >= 3,] actual <- collect(dropna(df, minNonNulls = 3, cols = c("name", "age", "height"))) expect_identical(expected, actual) + actual <- collect(na.omit(df, minNonNulls = 3, cols = c("name", "age", "height"))) + expect_identical(expected, actual) }) test_that("fillna() on a DataFrame", { diff --git a/dev/lint-r b/dev/lint-r index c15d57aad86da..bfda0bca15eb7 100755 --- a/dev/lint-r +++ b/dev/lint-r @@ -29,7 +29,7 @@ fi `which Rscript` --vanilla "$SPARK_ROOT_DIR/dev/lint-r.R" "$SPARK_ROOT_DIR" | tee "$LINT_R_REPORT_FILE_NAME" -NUM_LINES=`wc -l < "$LINT_R_REPORT_FILE_NAME"` +NUM_LINES=`wc -l < "$LINT_R_REPORT_FILE_NAME" | awk '{print $1}'` if [ "$NUM_LINES" = "0" ] ; then lint_status=0 echo "lintr checks passed." From 4eeda8d472498acd40ef54723d1be9924a273d76 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Fri, 28 Aug 2015 00:50:26 -0700 Subject: [PATCH 132/232] [SPARK-10260] [ML] Add @Since annotation to ml.clustering ### JIRA [[SPARK-10260] Add Since annotation to ml.clustering - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-10260) Author: Yu ISHIKAWA Closes #8455 from yu-iskw/SPARK-10260. --- .../apache/spark/ml/clustering/KMeans.scala | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 47a18cdb31b53..f40ab71fb22a6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.clustering -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{Since, Experimental} import org.apache.spark.ml.param.{Param, Params, IntParam, ParamMap} import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util.{Identifiable, SchemaUtils} @@ -39,9 +39,11 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * Set the number of clusters to create (k). Must be > 1. Default: 2. * @group param */ + @Since("1.5.0") final val k = new IntParam(this, "k", "number of clusters to create", (x: Int) => x > 1) /** @group getParam */ + @Since("1.5.0") def getK: Int = $(k) /** @@ -50,10 +52,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. * @group expertParam */ + @Since("1.5.0") final val initMode = new Param[String](this, "initMode", "initialization algorithm", (value: String) => MLlibKMeans.validateInitMode(value)) /** @group expertGetParam */ + @Since("1.5.0") def getInitMode: String = $(initMode) /** @@ -61,10 +65,12 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * setting -- the default of 5 is almost always enough. Must be > 0. Default: 5. * @group expertParam */ + @Since("1.5.0") final val initSteps = new IntParam(this, "initSteps", "number of steps for k-means||", (value: Int) => value > 0) /** @group expertGetParam */ + @Since("1.5.0") def getInitSteps: Int = $(initSteps) /** @@ -84,27 +90,32 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe * * @param parentModel a model trained by spark.mllib.clustering.KMeans. */ +@Since("1.5.0") @Experimental class KMeansModel private[ml] ( - override val uid: String, + @Since("1.5.0") override val uid: String, private val parentModel: MLlibKMeansModel) extends Model[KMeansModel] with KMeansParams { + @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { val copied = new KMeansModel(uid, parentModel) copyValues(copied, extra) } + @Since("1.5.0") override def transform(dataset: DataFrame): DataFrame = { val predictUDF = udf((vector: Vector) => predict(vector)) dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } private[clustering] def predict(features: Vector): Int = parentModel.predict(features) + @Since("1.5.0") def clusterCenters: Array[Vector] = parentModel.clusterCenters } @@ -114,8 +125,11 @@ class KMeansModel private[ml] ( * * @see [[http://dx.doi.org/10.14778/2180912.2180915 Bahmani et al., Scalable k-means++.]] */ +@Since("1.5.0") @Experimental -class KMeans(override val uid: String) extends Estimator[KMeansModel] with KMeansParams { +class KMeans @Since("1.5.0") ( + @Since("1.5.0") override val uid: String) + extends Estimator[KMeansModel] with KMeansParams { setDefault( k -> 2, @@ -124,34 +138,45 @@ class KMeans(override val uid: String) extends Estimator[KMeansModel] with KMean initSteps -> 5, tol -> 1e-4) + @Since("1.5.0") override def copy(extra: ParamMap): KMeans = defaultCopy(extra) + @Since("1.5.0") def this() = this(Identifiable.randomUID("kmeans")) /** @group setParam */ + @Since("1.5.0") def setFeaturesCol(value: String): this.type = set(featuresCol, value) /** @group setParam */ + @Since("1.5.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) /** @group setParam */ + @Since("1.5.0") def setK(value: Int): this.type = set(k, value) /** @group expertSetParam */ + @Since("1.5.0") def setInitMode(value: String): this.type = set(initMode, value) /** @group expertSetParam */ + @Since("1.5.0") def setInitSteps(value: Int): this.type = set(initSteps, value) /** @group setParam */ + @Since("1.5.0") def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ + @Since("1.5.0") def setTol(value: Double): this.type = set(tol, value) /** @group setParam */ + @Since("1.5.0") def setSeed(value: Long): this.type = set(seed, value) + @Since("1.5.0") override def fit(dataset: DataFrame): KMeansModel = { val rdd = dataset.select(col($(featuresCol))).map { case Row(point: Vector) => point } @@ -167,6 +192,7 @@ class KMeans(override val uid: String) extends Estimator[KMeansModel] with KMean copyValues(model) } + @Since("1.5.0") override def transformSchema(schema: StructType): StructType = { validateAndTransformSchema(schema) } From cc39803062119c1d14611dc227b9ed0ed1284d38 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 28 Aug 2015 09:32:23 +0100 Subject: [PATCH 133/232] [SPARK-10295] [CORE] Dynamic allocation in Mesos does not release when RDDs are cached Remove obsolete warning about dynamic allocation not working with cached RDDs See discussion in https://issues.apache.org/jira/browse/SPARK-10295 Author: Sean Owen Closes #8489 from srowen/SPARK-10295. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f3da04a7f55d0..738887076b0d1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1590,11 +1590,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Register an RDD to be persisted in memory and/or disk storage */ private[spark] def persistRDD(rdd: RDD[_]) { - _executorAllocationManager.foreach { _ => - logWarning( - s"Dynamic allocation currently does not support cached RDDs. Cached data for RDD " + - s"${rdd.id} will be lost when executors are removed.") - } persistentRdds(rdd.id) = rdd } From 18294cd8710427076caa86bfac596de67089d57e Mon Sep 17 00:00:00 2001 From: Keiji Yoshida Date: Fri, 28 Aug 2015 09:36:50 +0100 Subject: [PATCH 134/232] Fix DynamodDB/DynamoDB typo in Kinesis Integration doc Fix DynamodDB/DynamoDB typo in Kinesis Integration doc Author: Keiji Yoshida Closes #8501 from yosssi/patch-1. --- docs/streaming-kinesis-integration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index a7bcaec6fcd84..238a911a9199f 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -91,7 +91,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m - Kinesis data processing is ordered per partition and occurs at-least once per message. - - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamodDB. + - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamoDB. - A single Kinesis stream shard is processed by one input DStream at a time. From 71a077f6c16c8816eae13341f645ba50d997f63d Mon Sep 17 00:00:00 2001 From: Dharmesh Kakadia Date: Fri, 28 Aug 2015 09:38:35 +0100 Subject: [PATCH 135/232] typo in comment Author: Dharmesh Kakadia Closes #8497 from dharmeshkakadia/patch-2. --- .../apache/spark/network/shuffle/protocol/RegisterExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java index cca8b17c4f129..167ef33104227 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java @@ -27,7 +27,7 @@ /** * Initial registration message between an executor and its local shuffle server. - * Returns nothing (empty bye array). + * Returns nothing (empty byte array). */ public class RegisterExecutor extends BlockTransferMessage { public final String appId; From 1502a0f6c5d2f85a331b29d3bf50002911ea393e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Aug 2015 09:32:52 -0500 Subject: [PATCH 136/232] [YARN] [MINOR] Avoid hard code port number in YarnShuffleService test Current port number is fixed as default (7337) in test, this will introduce port contention exception, better to change to a random number in unit test. squito , seems you're author of this unit test, mind taking a look at this fix? Thanks a lot. ``` [info] - executor state kept across NM restart *** FAILED *** (597 milliseconds) [info] org.apache.hadoop.service.ServiceStateException: java.net.BindException: Address already in use [info] at org.apache.hadoop.service.ServiceStateException.convert(ServiceStateException.java:59) [info] at org.apache.hadoop.service.AbstractService.init(AbstractService.java:172) [info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply$mcV$sp(YarnShuffleServiceSuite.scala:72) [info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply(YarnShuffleServiceSuite.scala:70) [info] at org.apache.spark.network.yarn.YarnShuffleServiceSuite$$anonfun$1.apply(YarnShuffleServiceSuite.scala:70) [info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) [info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42) ... ``` Author: jerryshao Closes #8502 from jerryshao/avoid-hardcode-port. --- .../org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 2f22cbdbeac37..6aa8c814cd4f0 100644 --- a/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -37,6 +37,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), classOf[YarnShuffleService].getCanonicalName) + yarnConfig.setInt("spark.shuffle.service.port", 0) yarnConfig.get("yarn.nodemanager.local-dirs").split(",").foreach { dir => val d = new File(dir) From e2a843090cb031f6aa774f6d9c031a7f0f732ee1 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Fri, 28 Aug 2015 08:00:44 -0700 Subject: [PATCH 137/232] [SPARK-9890] [DOC] [ML] User guide for CountVectorizer jira: https://issues.apache.org/jira/browse/SPARK-9890 document with Scala and java examples Author: Yuhao Yang Closes #8487 from hhbyyh/cvDoc. --- docs/ml-features.md | 109 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 109 insertions(+) diff --git a/docs/ml-features.md b/docs/ml-features.md index 89a9bad570446..90654d1e5a248 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -211,6 +211,115 @@ for feature in result.select("result").take(3): +## CountVectorizer + +`CountVectorizer` and `CountVectorizerModel` aim to help convert a collection of text documents + to vectors of token counts. When an a-priori dictionary is not available, `CountVectorizer` can + be used as an `Estimator` to extract the vocabulary and generates a `CountVectorizerModel`. The + model produces sparse representations for the documents over the vocabulary, which can then be + passed to other algorithms like LDA. + + During the fitting process, `CountVectorizer` will select the top `vocabSize` words ordered by + term frequency across the corpus. An optional parameter "minDF" also affect the fitting process + by specifying the minimum number (or fraction if < 1.0) of documents a term must appear in to be + included in the vocabulary. + +**Examples** + +Assume that we have the following DataFrame with columns `id` and `texts`: + +~~~~ + id | texts +----|---------- + 0 | Array("a", "b", "c") + 1 | Array("a", "b", "b", "c", "a") +~~~~ + +each row in`texts` is a document of type Array[String]. +Invoking fit of `CountVectorizer` produces a `CountVectorizerModel` with vocabulary (a, b, c), +then the output column "vector" after transformation contains: + +~~~~ + id | texts | vector +----|---------------------------------|--------------- + 0 | Array("a", "b", "c") | (3,[0,1,2],[1.0,1.0,1.0]) + 1 | Array("a", "b", "b", "c", "a") | (3,[0,1,2],[2.0,2.0,1.0]) +~~~~ + +each vector represents the token counts of the document over the vocabulary. + +
+
+More details can be found in the API docs for +[CountVectorizer](api/scala/index.html#org.apache.spark.ml.feature.CountVectorizer) and +[CountVectorizerModel](api/scala/index.html#org.apache.spark.ml.feature.CountVectorizerModel). +{% highlight scala %} +import org.apache.spark.ml.feature.CountVectorizer +import org.apache.spark.mllib.util.CountVectorizerModel + +val df = sqlContext.createDataFrame(Seq( + (0, Array("a", "b", "c")), + (1, Array("a", "b", "b", "c", "a")) +)).toDF("id", "words") + +// fit a CountVectorizerModel from the corpus +val cvModel: CountVectorizerModel = new CountVectorizer() + .setInputCol("words") + .setOutputCol("features") + .setVocabSize(3) + .setMinDF(2) // a term must appear in more or equal to 2 documents to be included in the vocabulary + .fit(df) + +// alternatively, define CountVectorizerModel with a-priori vocabulary +val cvm = new CountVectorizerModel(Array("a", "b", "c")) + .setInputCol("words") + .setOutputCol("features") + +cvModel.transform(df).select("features").show() +{% endhighlight %} +
+ +
+More details can be found in the API docs for +[CountVectorizer](api/java/org/apache/spark/ml/feature/CountVectorizer.html) and +[CountVectorizerModel](api/java/org/apache/spark/ml/feature/CountVectorizerModel.html). +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.CountVectorizer; +import org.apache.spark.ml.feature.CountVectorizerModel; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.*; + +// Input data: Each row is a bag of words from a sentence or document. +JavaRDD jrdd = jsc.parallelize(Arrays.asList( + RowFactory.create(Arrays.asList("a", "b", "c")), + RowFactory.create(Arrays.asList("a", "b", "b", "c", "a")) +)); +StructType schema = new StructType(new StructField [] { + new StructField("text", new ArrayType(DataTypes.StringType, true), false, Metadata.empty()) +}); +DataFrame df = sqlContext.createDataFrame(jrdd, schema); + +// fit a CountVectorizerModel from the corpus +CountVectorizerModel cvModel = new CountVectorizer() + .setInputCol("text") + .setOutputCol("feature") + .setVocabSize(3) + .setMinDF(2) // a term must appear in more or equal to 2 documents to be included in the vocabulary + .fit(df); + +// alternatively, define CountVectorizerModel with a-priori vocabulary +CountVectorizerModel cvm = new CountVectorizerModel(new String[]{"a", "b", "c"}) + .setInputCol("text") + .setOutputCol("feature"); + +cvModel.transform(df).show(); +{% endhighlight %} +
+
+ # Feature Transformers ## Tokenizer From 499e8e154bdcc9d7b2f685b159e0ddb4eae48fe4 Mon Sep 17 00:00:00 2001 From: Luciano Resende Date: Fri, 28 Aug 2015 09:13:21 -0700 Subject: [PATCH 138/232] [SPARK-8952] [SPARKR] - Wrap normalizePath calls with suppressWarnings This is based on davies comment on SPARK-8952 which suggests to only call normalizePath() when path starts with '~' Author: Luciano Resende Closes #8343 from lresende/SPARK-8952. --- R/pkg/R/SQLContext.R | 4 ++-- R/pkg/R/sparkR.R | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 110117a18ccbc..1bc6445311473 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -201,7 +201,7 @@ setMethod("toDF", signature(x = "RDD"), jsonFile <- function(sqlContext, path) { # Allow the user to have a more flexible definiton of the text file path - path <- normalizePath(path) + path <- suppressWarnings(normalizePath(path)) # Convert a string vector of paths to a string containing comma separated paths path <- paste(path, collapse = ",") sdf <- callJMethod(sqlContext, "jsonFile", path) @@ -251,7 +251,7 @@ jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { # TODO: Implement saveasParquetFile and write examples for both parquetFile <- function(sqlContext, ...) { # Allow the user to have a more flexible definiton of the text file path - paths <- lapply(list(...), normalizePath) + paths <- lapply(list(...), function(x) suppressWarnings(normalizePath(x))) sdf <- callJMethod(sqlContext, "parquetFile", paths) dataFrame(sdf) } diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index e83104f116422..3c57a44db257d 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -160,7 +160,7 @@ sparkR.init <- function( }) if (nchar(sparkHome) != 0) { - sparkHome <- normalizePath(sparkHome) + sparkHome <- suppressWarnings(normalizePath(sparkHome)) } sparkEnvirMap <- new.env() From d3f87dc39480f075170817bbd00142967a938078 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 28 Aug 2015 11:51:42 -0700 Subject: [PATCH 139/232] [SPARK-10325] Override hashCode() for public Row This commit fixes an issue where the public SQL `Row` class did not override `hashCode`, causing it to violate the hashCode() + equals() contract. To fix this, I simply ported the `hashCode` implementation from the 1.4.x version of `Row`. Author: Josh Rosen Closes #8500 from JoshRosen/SPARK-10325 and squashes the following commits: 51ffea1 [Josh Rosen] Override hashCode() for public Row. --- .../src/main/scala/org/apache/spark/sql/Row.scala | 13 +++++++++++++ .../test/scala/org/apache/spark/sql/RowSuite.scala | 9 +++++++++ 2 files changed, 22 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index cfd9cb0e62598..ed2fdf9f2f7cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ +import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericRow @@ -410,6 +411,18 @@ trait Row extends Serializable { true } + override def hashCode: Int = { + // Using Scala's Seq hash code implementation. + var n = 0 + var h = MurmurHash3.seqSeed + val len = length + while (n < len) { + h = MurmurHash3.mix(h, apply(n).##) + n += 1 + } + MurmurHash3.finalizeHash(h, n) + } + /* ---------------------- utility methods for Scala ---------------------- */ /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 795d4e983f27e..77ccd6f775e50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -85,4 +85,13 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { val r2 = Row(Double.NaN) assert(r1 === r2) } + + test("equals and hashCode") { + val r1 = Row("Hello") + val r2 = Row("Hello") + assert(r1 === r2) + assert(r1.hashCode() === r2.hashCode()) + val r3 = Row("World") + assert(r3.hashCode() != r1.hashCode()) + } } From c53c902fa9c458200245f919067b41dde9cd9418 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 28 Aug 2015 12:33:40 -0700 Subject: [PATCH 140/232] [SPARK-9284] [TESTS] Allow all tests to run without an assembly. This change aims at speeding up the dev cycle a little bit, by making sure that all tests behave the same w.r.t. where the code to be tested is loaded from. Namely, that means that tests don't rely on the assembly anymore, rather loading all needed classes from the build directories. The main change is to make sure all build directories (classes and test-classes) are added to the classpath of child processes when running tests. YarnClusterSuite required some custom code since the executors are run differently (i.e. not through the launcher library, like standalone and Mesos do). I also found a couple of tests that could leak a SparkContext on failure, and added code to handle those. With this patch, it's possible to run the following command from a clean source directory and have all tests pass: mvn -Pyarn -Phadoop-2.4 -Phive-thriftserver install Author: Marcelo Vanzin Closes #7629 from vanzin/SPARK-9284. --- bin/spark-class | 16 ++++---- .../spark/launcher/SparkLauncherSuite.java | 0 .../spark/broadcast/BroadcastSuite.scala | 10 ++++- .../launcher/AbstractCommandBuilder.java | 28 ++++++++------ pom.xml | 8 ++++ project/SparkBuild.scala | 2 + .../deploy/yarn/BaseYarnClusterSuite.scala | 37 +++++++++++++------ .../spark/deploy/yarn/YarnClusterSuite.scala | 20 ++++++++-- .../yarn/YarnShuffleIntegrationSuite.scala | 2 +- .../spark/launcher/TestClasspathBuilder.scala | 36 ++++++++++++++++++ 10 files changed, 122 insertions(+), 37 deletions(-) rename {launcher => core}/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java (100%) create mode 100644 yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala diff --git a/bin/spark-class b/bin/spark-class index 2b59e5df5736f..e38e08dec40e4 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -43,17 +43,19 @@ else fi num_jars="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" | wc -l)" -if [ "$num_jars" -eq "0" -a -z "$SPARK_ASSEMBLY_JAR" ]; then +if [ "$num_jars" -eq "0" -a -z "$SPARK_ASSEMBLY_JAR" -a "$SPARK_PREPEND_CLASSES" != "1" ]; then echo "Failed to find Spark assembly in $ASSEMBLY_DIR." 1>&2 echo "You need to build Spark before running this program." 1>&2 exit 1 fi -ASSEMBLY_JARS="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" || true)" -if [ "$num_jars" -gt "1" ]; then - echo "Found multiple Spark assembly jars in $ASSEMBLY_DIR:" 1>&2 - echo "$ASSEMBLY_JARS" 1>&2 - echo "Please remove all but one jar." 1>&2 - exit 1 +if [ -d "$ASSEMBLY_DIR" ]; then + ASSEMBLY_JARS="$(ls -1 "$ASSEMBLY_DIR" | grep "^spark-assembly.*hadoop.*\.jar$" || true)" + if [ "$num_jars" -gt "1" ]; then + echo "Found multiple Spark assembly jars in $ASSEMBLY_DIR:" 1>&2 + echo "$ASSEMBLY_JARS" 1>&2 + echo "Please remove all but one jar." 1>&2 + exit 1 + fi fi SPARK_ASSEMBLY_JAR="${ASSEMBLY_DIR}/${ASSEMBLY_JARS}" diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java similarity index 100% rename from launcher/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java rename to core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 48e74f06f79b1..fb7a8ae3f9d41 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -310,8 +310,14 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext { val _sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", broadcastConf) // Wait until all salves are up - _sc.jobProgressListener.waitUntilExecutorsUp(numSlaves, 10000) - _sc + try { + _sc.jobProgressListener.waitUntilExecutorsUp(numSlaves, 10000) + _sc + } catch { + case e: Throwable => + _sc.stop() + throw e + } } else { new SparkContext("local", "test", broadcastConf) } diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 5e793a5c48775..0a237ee73b670 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -169,9 +169,11 @@ List buildClassPath(String appClassPath) throws IOException { "streaming", "tools", "sql/catalyst", "sql/core", "sql/hive", "sql/hive-thriftserver", "yarn", "launcher"); if (prependClasses) { - System.err.println( - "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + - "assembly."); + if (!isTesting) { + System.err.println( + "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark classes ahead of " + + "assembly."); + } for (String project : projects) { addToClassPath(cp, String.format("%s/%s/target/scala-%s/classes", sparkHome, project, scala)); @@ -200,7 +202,7 @@ List buildClassPath(String appClassPath) throws IOException { // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. // That duplicates some of the code in the shell scripts that look for the assembly, though. String assembly = getenv(ENV_SPARK_ASSEMBLY); - if (assembly == null && isEmpty(getenv("SPARK_TESTING"))) { + if (assembly == null && !isTesting) { assembly = findAssembly(); } addToClassPath(cp, assembly); @@ -215,12 +217,14 @@ List buildClassPath(String appClassPath) throws IOException { libdir = new File(sparkHome, "lib_managed/jars"); } - checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", - libdir.getAbsolutePath()); - for (File jar : libdir.listFiles()) { - if (jar.getName().startsWith("datanucleus-")) { - addToClassPath(cp, jar.getAbsolutePath()); + if (libdir.isDirectory()) { + for (File jar : libdir.listFiles()) { + if (jar.getName().startsWith("datanucleus-")) { + addToClassPath(cp, jar.getAbsolutePath()); + } } + } else { + checkState(isTesting, "Library directory '%s' does not exist.", libdir.getAbsolutePath()); } addToClassPath(cp, getenv("HADOOP_CONF_DIR")); @@ -256,15 +260,15 @@ String getScalaVersion() { return scala; } String sparkHome = getSparkHome(); - File scala210 = new File(sparkHome, "assembly/target/scala-2.10"); - File scala211 = new File(sparkHome, "assembly/target/scala-2.11"); + File scala210 = new File(sparkHome, "launcher/target/scala-2.10"); + File scala211 = new File(sparkHome, "launcher/target/scala-2.11"); checkState(!scala210.isDirectory() || !scala211.isDirectory(), "Presence of build for both scala versions (2.10 and 2.11) detected.\n" + "Either clean one of them or set SPARK_SCALA_VERSION in your environment."); if (scala210.isDirectory()) { return "2.10"; } else { - checkState(scala211.isDirectory(), "Cannot find any assembly build directories."); + checkState(scala211.isDirectory(), "Cannot find any build directories."); return "2.11"; } } diff --git a/pom.xml b/pom.xml index 0716016523ee1..88ebceca769e9 100644 --- a/pom.xml +++ b/pom.xml @@ -1421,6 +1421,10 @@ org.apache.thrift libthrift + + org.mortbay.jetty + servlet-api + com.google.guava guava @@ -1892,6 +1896,8 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + 1 + 1 ${test.java.home} @@ -1929,6 +1935,8 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + 1 + 1 ${test.java.home} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ea52bfd67944a..901cfa538d23e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -547,6 +547,8 @@ object TestSettings { envVars in Test ++= Map( "SPARK_DIST_CLASSPATH" -> (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), + "SPARK_PREPEND_CLASSES" -> "1", + "SPARK_TESTING" -> "1", "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += s"-Djava.io.tmpdir=$testTempDir", javaOptions in Test += "-Dspark.test.home=" + sparkHome, diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index b4f8049bff577..17c59ff06e0c1 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.server.MiniYARNCluster import org.scalatest.{BeforeAndAfterAll, Matchers} import org.apache.spark._ +import org.apache.spark.launcher.TestClasspathBuilder import org.apache.spark.util.Utils abstract class BaseYarnClusterSuite @@ -43,6 +44,9 @@ abstract class BaseYarnClusterSuite |log4j.appender.console.target=System.err |log4j.appender.console.layout=org.apache.log4j.PatternLayout |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + |log4j.logger.org.apache.hadoop=WARN + |log4j.logger.org.eclipse.jetty=WARN + |log4j.logger.org.spark-project.jetty=WARN """.stripMargin private var yarnCluster: MiniYARNCluster = _ @@ -51,8 +55,7 @@ abstract class BaseYarnClusterSuite private var hadoopConfDir: File = _ private var logConfDir: File = _ - - def yarnConfig: YarnConfiguration + def newYarnConfig(): YarnConfiguration override def beforeAll() { super.beforeAll() @@ -65,8 +68,14 @@ abstract class BaseYarnClusterSuite val logConfFile = new File(logConfDir, "log4j.properties") Files.write(LOG4J_CONF, logConfFile, UTF_8) + // Disable the disk utilization check to avoid the test hanging when people's disks are + // getting full. + val yarnConf = newYarnConfig() + yarnConf.set("yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage", + "100.0") + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) - yarnCluster.init(yarnConfig) + yarnCluster.init(yarnConf) yarnCluster.start() // There's a race in MiniYARNCluster in which start() may return before the RM has updated @@ -114,19 +123,23 @@ abstract class BaseYarnClusterSuite sparkArgs: Seq[String] = Nil, extraClassPath: Seq[String] = Nil, extraJars: Seq[String] = Nil, - extraConf: Map[String, String] = Map()): Unit = { + extraConf: Map[String, String] = Map(), + extraEnv: Map[String, String] = Map()): Unit = { val master = if (clientMode) "yarn-client" else "yarn-cluster" val props = new Properties() props.setProperty("spark.yarn.jar", "local:" + fakeSparkJar.getAbsolutePath()) - val childClasspath = logConfDir.getAbsolutePath() + - File.pathSeparator + - sys.props("java.class.path") + - File.pathSeparator + - extraClassPath.mkString(File.pathSeparator) - props.setProperty("spark.driver.extraClassPath", childClasspath) - props.setProperty("spark.executor.extraClassPath", childClasspath) + val testClasspath = new TestClasspathBuilder() + .buildClassPath( + logConfDir.getAbsolutePath() + + File.pathSeparator + + extraClassPath.mkString(File.pathSeparator)) + .asScala + .mkString(File.pathSeparator) + + props.setProperty("spark.driver.extraClassPath", testClasspath) + props.setProperty("spark.executor.extraClassPath", testClasspath) // SPARK-4267: make sure java options are propagated correctly. props.setProperty("spark.driver.extraJavaOptions", "-Dfoo=\"one two three\"") @@ -168,7 +181,7 @@ abstract class BaseYarnClusterSuite appArgs Utils.executeAndGetOutput(argv, - extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath())) + extraEnvironment = Map("YARN_CONF_DIR" -> hadoopConfDir.getAbsolutePath()) ++ extraEnv) } /** diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 5a4ea2ea2f4ff..b5a42fd6afd98 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -28,7 +28,9 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} +import org.apache.spark.launcher.TestClasspathBuilder +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, + SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -39,7 +41,7 @@ import org.apache.spark.util.Utils */ class YarnClusterSuite extends BaseYarnClusterSuite { - override def yarnConfig: YarnConfiguration = new YarnConfiguration() + override def newYarnConfig(): YarnConfiguration = new YarnConfiguration() private val TEST_PYFILE = """ |import mod1, mod2 @@ -111,6 +113,17 @@ class YarnClusterSuite extends BaseYarnClusterSuite { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) + // When running tests, let's not assume the user has built the assembly module, which also + // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the + // needed locations. + val sparkHome = sys.props("spark.test.home"); + val pythonPath = Seq( + s"$sparkHome/python/lib/py4j-0.8.2.1-src.zip", + s"$sparkHome/python") + val extraEnv = Map( + "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), + "PYTHONPATH" -> pythonPath.mkString(File.pathSeparator)) + val moduleDir = if (clientMode) { // In client-mode, .py files added with --py-files are not visible in the driver. @@ -130,7 +143,8 @@ class YarnClusterSuite extends BaseYarnClusterSuite { runSpark(clientMode, primaryPyFile.getAbsolutePath(), sparkArgs = Seq("--py-files", pyFiles), - appArgs = Seq(result.getAbsolutePath())) + appArgs = Seq(result.getAbsolutePath()), + extraEnv = extraEnv) checkResult(result) } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 5e8238822b90a..8d9c9b3004eda 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} */ class YarnShuffleIntegrationSuite extends BaseYarnClusterSuite { - override def yarnConfig: YarnConfiguration = { + override def newYarnConfig(): YarnConfiguration = { val yarnConfig = new YarnConfiguration() yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), diff --git a/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala b/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala new file mode 100644 index 0000000000000..da9e8e21a26ae --- /dev/null +++ b/yarn/src/test/scala/org/apache/spark/launcher/TestClasspathBuilder.scala @@ -0,0 +1,36 @@ +/* + * 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.launcher + +import java.util.{List => JList, Map => JMap} + +/** + * Exposes AbstractCommandBuilder to the YARN tests, so that they can build classpaths the same + * way other cluster managers do. + */ +private[spark] class TestClasspathBuilder extends AbstractCommandBuilder { + + childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, sys.props("spark.test.home")) + + override def buildClassPath(extraCp: String): JList[String] = super.buildClassPath(extraCp) + + /** Not used by the YARN tests. */ + override def buildCommand(env: JMap[String, String]): JList[String] = + throw new UnsupportedOperationException() + +} From 45723214e694b9a440723e9504c562e6393709f3 Mon Sep 17 00:00:00 2001 From: Shuo Xiang Date: Fri, 28 Aug 2015 13:09:13 -0700 Subject: [PATCH 141/232] [SPARK-10336][example] fix not being able to set intercept in LR example `fitIntercept` is a command line option but not set in the main program. dbtsai Author: Shuo Xiang Closes #8510 from coderxiang/intercept and squashes the following commits: 57c9b7d [Shuo Xiang] fix not being able to set intercept in LR example --- .../org/apache/spark/examples/ml/LogisticRegressionExample.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index 7682557127b51..8e3760ddb50a9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -136,6 +136,7 @@ object LogisticRegressionExample { .setElasticNetParam(params.elasticNetParam) .setMaxIter(params.maxIter) .setTol(params.tol) + .setFitIntercept(params.fitIntercept) stages += lor val pipeline = new Pipeline().setStages(stages.toArray) From 88032ecaf0455886aed7a66b30af80dae7f6cff7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 28 Aug 2015 13:53:31 -0700 Subject: [PATCH 142/232] [SPARK-9671] [MLLIB] re-org user guide and add migration guide This PR updates the MLlib user guide and adds migration guide for 1.4->1.5. * merge migration guide for `spark.mllib` and `spark.ml` packages * remove dependency section from `spark.ml` guide * move the paragraph about `spark.mllib` and `spark.ml` to the top and recommend `spark.ml` * move Sam's talk to footnote to make the section focus on dependencies Minor changes to code examples and other wording will be in a separate PR. jkbradley srowen feynmanliang Author: Xiangrui Meng Closes #8498 from mengxr/SPARK-9671. --- docs/ml-guide.md | 52 ++------------ docs/mllib-guide.md | 119 ++++++++++++++++----------------- docs/mllib-migration-guides.md | 30 +++++++++ 3 files changed, 95 insertions(+), 106 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 01bf5ee18e328..ce53400b6ee56 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -21,19 +21,11 @@ title: Spark ML Programming Guide \]` -Spark 1.2 introduced a new package called `spark.ml`, which aims to provide a uniform set of -high-level APIs that help users create and tune practical machine learning pipelines. - -*Graduated from Alpha!* The Pipelines API is no longer an alpha component, although many elements of it are still `Experimental` or `DeveloperApi`. - -Note that we will keep supporting and adding features to `spark.mllib` along with the -development of `spark.ml`. -Users should be comfortable using `spark.mllib` features and expect more features coming. -Developers should contribute new algorithms to `spark.mllib` and can optionally contribute -to `spark.ml`. - -See the [Algorithm Guides section](#algorithm-guides) below for guides on sub-packages of `spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. - +The `spark.ml` package aims to provide a uniform set of high-level APIs built on top of +[DataFrames](sql-programming-guide.html#dataframes) that help users create and tune practical +machine learning pipelines. +See the [Algorithm Guides section](#algorithm-guides) below for guides on sub-packages of +`spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. **Table of Contents** @@ -171,7 +163,7 @@ This is useful if there are two algorithms with the `maxIter` parameter in a `Pi # Algorithm Guides -There are now several algorithms in the Pipelines API which are not in the lower-level MLlib API, so we link to documentation for them here. These algorithms are mostly feature transformers, which fit naturally into the `Transformer` abstraction in Pipelines, and ensembles, which fit naturally into the `Estimator` abstraction in the Pipelines. +There are now several algorithms in the Pipelines API which are not in the `spark.mllib` API, so we link to documentation for them here. These algorithms are mostly feature transformers, which fit naturally into the `Transformer` abstraction in Pipelines, and ensembles, which fit naturally into the `Estimator` abstraction in the Pipelines. **Pipelines API Algorithm Guides** @@ -880,35 +872,3 @@ jsc.stop(); - -# Dependencies - -Spark ML currently depends on MLlib and has the same dependencies. -Please see the [MLlib Dependencies guide](mllib-guide.html#dependencies) for more info. - -Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. - -# Migration Guide - -## From 1.3 to 1.4 - -Several major API changes occurred, including: -* `Param` and other APIs for specifying parameters -* `uid` unique IDs for Pipeline components -* Reorganization of certain classes -Since the `spark.ml` API was an Alpha Component in Spark 1.3, we do not list all changes here. - -However, now that `spark.ml` is no longer an Alpha Component, we will provide details on any API changes for future releases. - -## From 1.2 to 1.3 - -The main API changes are from Spark SQL. We list the most important changes here: - -* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame. -* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. -* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. - -Other changes were in `LogisticRegression`: - -* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). -* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 6330c977552d1..876dcfd40ed7b 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -5,21 +5,28 @@ displayTitle: Machine Learning Library (MLlib) Guide description: MLlib machine learning library overview for Spark SPARK_VERSION_SHORT --- -MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, -including classification, regression, clustering, collaborative -filtering, dimensionality reduction, as well as underlying optimization primitives. -Guides for individual algorithms are listed below. +MLlib is Spark's machine learning (ML) library. +Its goal is to make practical machine learning scalable and easy. +It consists of common learning algorithms and utilities, including classification, regression, +clustering, collaborative filtering, dimensionality reduction, as well as lower-level optimization +primitives and higher-level pipeline APIs. -The API is divided into 2 parts: +It divides into two packages: -* [The original `spark.mllib` API](mllib-guide.html#mllib-types-algorithms-and-utilities) is the primary API. -* [The "Pipelines" `spark.ml` API](mllib-guide.html#sparkml-high-level-apis-for-ml-pipelines) is a higher-level API for constructing ML workflows. +* [`spark.mllib`](mllib-guide.html#mllib-types-algorithms-and-utilities) contains the original API + built on top of RDDs. +* [`spark.ml`](mllib-guide.html#sparkml-high-level-apis-for-ml-pipelines) provides higher-level API + built on top of DataFrames for constructing ML pipelines. -We list major functionality from both below, with links to detailed guides. +Using `spark.ml` is recommended because with DataFrames the API is more versatile and flexible. +But we will keep supporting `spark.mllib` along with the development of `spark.ml`. +Users should be comfortable using `spark.mllib` features and expect more features coming. +Developers should contribute new algorithms to `spark.ml` if they fit the ML pipeline concept well, +e.g., feature extractors and transformers. -# MLlib types, algorithms and utilities +We list major functionality from both below, with links to detailed guides. -This lists functionality included in `spark.mllib`, the main MLlib API. +# spark.mllib: data types, algorithms, and utilities * [Data types](mllib-data-types.html) * [Basic statistics](mllib-statistics.html) @@ -56,71 +63,63 @@ This lists functionality included in `spark.mllib`, the main MLlib API. * [limited-memory BFGS (L-BFGS)](mllib-optimization.html#limited-memory-bfgs-l-bfgs) * [PMML model export](mllib-pmml-model-export.html) -MLlib is under active development. -The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and the migration guide below will explain all changes between releases. - # spark.ml: high-level APIs for ML pipelines -Spark 1.2 introduced a new package called `spark.ml`, which aims to provide a uniform set of -high-level APIs that help users create and tune practical machine learning pipelines. - -*Graduated from Alpha!* The Pipelines API is no longer an alpha component, although many elements of it are still `Experimental` or `DeveloperApi`. - -Note that we will keep supporting and adding features to `spark.mllib` along with the -development of `spark.ml`. -Users should be comfortable using `spark.mllib` features and expect more features coming. -Developers should contribute new algorithms to `spark.mllib` and can optionally contribute -to `spark.ml`. - -Guides for `spark.ml` include: +**[spark.ml programming guide](ml-guide.html)** provides an overview of the Pipelines API and major +concepts. It also contains sections on using algorithms within the Pipelines API, for example: -* **[spark.ml programming guide](ml-guide.html)**: overview of the Pipelines API and major concepts -* Guides on using algorithms within the Pipelines API: - * [Feature transformers](ml-features.html), including a few not in the lower-level `spark.mllib` API - * [Decision trees](ml-decision-tree.html) - * [Ensembles](ml-ensembles.html) - * [Linear methods](ml-linear-methods.html) +* [Feature Extraction, Transformation, and Selection](ml-features.html) +* [Decision Trees for Classification and Regression](ml-decision-tree.html) +* [Ensembles](ml-ensembles.html) +* [Linear methods with elastic net regularization](ml-linear-methods.html) +* [Multilayer perceptron classifier](ml-ann.html) # Dependencies -MLlib uses the linear algebra package -[Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java) for optimised -numerical processing. If natives are not available at runtime, you -will see a warning message and a pure JVM implementation will be used -instead. +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java) for optimised numerical processing. +If natives libraries[^1] are not available at runtime, you will see a warning message and a pure JVM +implementation will be used instead. -To learn more about the benefits and background of system optimised -natives, you may wish to watch Sam Halliday's ScalaX talk on -[High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/)). +Due to licensing issues with runtime proprietary binaries, we do not include `netlib-java`'s native +proxies by default. +To configure `netlib-java` / Breeze to use system optimised binaries, include +`com.github.fommil.netlib:all:1.1.2` (or build Spark with `-Pnetlib-lgpl`) as a dependency of your +project and read the [netlib-java](https://github.com/fommil/netlib-java) documentation for your +platform's additional installation instructions. -Due to licensing issues with runtime proprietary binaries, we do not -include `netlib-java`'s native proxies by default. To configure -`netlib-java` / Breeze to use system optimised binaries, include -`com.github.fommil.netlib:all:1.1.2` (or build Spark with -`-Pnetlib-lgpl`) as a dependency of your project and read the -[netlib-java](https://github.com/fommil/netlib-java) documentation for -your platform's additional installation instructions. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) -version 1.4 or newer. +[^1]: To learn more about the benefits and background of system optimised natives, you may wish to + watch Sam Halliday's ScalaX talk on [High Performance Linear Algebra in Scala](http://fommil.github.io/scalax14/#/). ---- +# Migration guide -# Migration Guide +MLlib is under active development. +The APIs marked `Experimental`/`DeveloperApi` may change in future releases, +and the migration guide below will explain all changes between releases. + +## From 1.4 to 1.5 -For the `spark.ml` package, please see the [spark.ml Migration Guide](ml-guide.html#migration-guide). +In the `spark.mllib` package, there are no break API changes but several behavior changes: -## From 1.3 to 1.4 +* [SPARK-9005](https://issues.apache.org/jira/browse/SPARK-9005): + `RegressionMetrics.explainedVariance` returns the average regression sum of squares. +* [SPARK-8600](https://issues.apache.org/jira/browse/SPARK-8600): `NaiveBayesModel.labels` become + sorted. +* [SPARK-3382](https://issues.apache.org/jira/browse/SPARK-3382): `GradientDescent` has a default + convergence tolerance `1e-3`, and hence iterations might end earlier than 1.4. -In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: +In the `spark.ml` package, there exists one break API change and one behavior change: -* Gradient-Boosted Trees - * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. - * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. -* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. +* [SPARK-9268](https://issues.apache.org/jira/browse/SPARK-9268): Java's varargs support is removed + from `Params.setDefault` due to a + [Scala compiler bug](https://issues.scala-lang.org/browse/SI-9013). +* [SPARK-10097](https://issues.apache.org/jira/browse/SPARK-10097): `Evaluator.isLargerBetter` is + added to indicate metric ordering. Metrics like RMSE no longer flip signs as in 1.4. -## Previous Spark Versions +## Previous Spark versions Earlier migration guides are archived [on this page](mllib-migration-guides.html). + +--- diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md index 8df68d81f3c78..774b85d1f773a 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/mllib-migration-guides.md @@ -7,6 +7,25 @@ description: MLlib migration guides from before Spark SPARK_VERSION_SHORT The migration guide for the current Spark version is kept on the [MLlib Programming Guide main page](mllib-guide.html#migration-guide). +## From 1.3 to 1.4 + +In the `spark.mllib` package, there were several breaking changes, but all in `DeveloperApi` or `Experimental` APIs: + +* Gradient-Boosted Trees + * *(Breaking change)* The signature of the [`Loss.gradient`](api/scala/index.html#org.apache.spark.mllib.tree.loss.Loss) method was changed. This is only an issues for users who wrote their own losses for GBTs. + * *(Breaking change)* The `apply` and `copy` methods for the case class [`BoostingStrategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.BoostingStrategy) have been changed because of a modification to the case class fields. This could be an issue for users who use `BoostingStrategy` to set GBT parameters. +* *(Breaking change)* The return value of [`LDA.run`](api/scala/index.html#org.apache.spark.mllib.clustering.LDA) has changed. It now returns an abstract class `LDAModel` instead of the concrete class `DistributedLDAModel`. The object of type `LDAModel` can still be cast to the appropriate concrete type, which depends on the optimization algorithm. + +In the `spark.ml` package, several major API changes occurred, including: + +* `Param` and other APIs for specifying parameters +* `uid` unique IDs for Pipeline components +* Reorganization of certain classes + +Since the `spark.ml` API was an alpha component in Spark 1.3, we do not list all changes here. +However, since 1.4 `spark.ml` is no longer an alpha component, we will provide details on any API +changes for future releases. + ## From 1.2 to 1.3 In the `spark.mllib` package, there were several breaking changes. The first change (in `ALS`) is the only one in a component not marked as Alpha or Experimental. @@ -23,6 +42,17 @@ In the `spark.mllib` package, there were several breaking changes. The first ch * In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. +In the `spark.ml` package, the main API changes are from Spark SQL. We list the most important changes here: + +* The old [SchemaRDD](http://spark.apache.org/docs/1.2.1/api/scala/index.html#org.apache.spark.sql.SchemaRDD) has been replaced with [DataFrame](api/scala/index.html#org.apache.spark.sql.DataFrame) with a somewhat modified API. All algorithms in Spark ML which used to use SchemaRDD now use DataFrame. +* In Spark 1.2, we used implicit conversions from `RDD`s of `LabeledPoint` into `SchemaRDD`s by calling `import sqlContext._` where `sqlContext` was an instance of `SQLContext`. These implicits have been moved, so we now call `import sqlContext.implicits._`. +* Java APIs for SQL have also changed accordingly. Please see the examples above and the [Spark SQL Programming Guide](sql-programming-guide.html) for details. + +Other changes were in `LogisticRegression`: + +* The `scoreCol` output column (with default value "score") was renamed to be `probabilityCol` (with default value "probability"). The type was originally `Double` (for the probability of class 1.0), but it is now `Vector` (for the probability of each class, to support multiclass classification in the future). +* In Spark 1.2, `LogisticRegressionModel` did not include an intercept. In Spark 1.3, it includes an intercept; however, it will always be 0.0 since it uses the default settings for [spark.mllib.LogisticRegressionWithLBFGS](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS). The option to use an intercept will be added in the future. + ## From 1.1 to 1.2 The only API changes in MLlib v1.2 are in From bb7f35239385ec74b5ee69631b5480fbcee253e4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 28 Aug 2015 14:38:20 -0700 Subject: [PATCH 143/232] [SPARK-10323] [SQL] fix nullability of In/InSet/ArrayContain After this PR, In/InSet/ArrayContain will return null if value is null, instead of false. They also will return null even if there is a null in the set/array. Author: Davies Liu Closes #8492 from davies/fix_in. --- .../expressions/collectionOperations.scala | 62 ++++++++-------- .../sql/catalyst/expressions/predicates.scala | 71 +++++++++++++++---- .../sql/catalyst/optimizer/Optimizer.scala | 6 -- .../CollectionFunctionsSuite.scala | 12 +++- .../catalyst/expressions/PredicateSuite.scala | 49 +++++++++---- .../optimizer/ConstantFoldingSuite.scala | 21 +----- .../spark/sql/DataFrameFunctionsSuite.scala | 14 ++-- 7 files changed, 138 insertions(+), 97 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 646afa4047d84..7b8c5b723ded4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -19,9 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Comparator import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.codegen.{ - CodegenFallback, CodeGenContext, GeneratedExpressionCode} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, CodegenFallback, GeneratedExpressionCode} import org.apache.spark.sql.types._ /** @@ -145,46 +143,42 @@ case class ArrayContains(left: Expression, right: Expression) } } - override def nullable: Boolean = false + override def nullable: Boolean = { + left.nullable || right.nullable || left.dataType.asInstanceOf[ArrayType].containsNull + } - override def eval(input: InternalRow): Boolean = { - val arr = left.eval(input) - if (arr == null) { - false - } else { - val value = right.eval(input) - if (value == null) { - false - } else { - arr.asInstanceOf[ArrayData].foreach(right.dataType, (i, v) => - if (v == value) return true - ) - false + override def nullSafeEval(arr: Any, value: Any): Any = { + var hasNull = false + arr.asInstanceOf[ArrayData].foreach(right.dataType, (i, v) => + if (v == null) { + hasNull = true + } else if (v == value) { + return true } + ) + if (hasNull) { + null + } else { + false } } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val arrGen = left.gen(ctx) - val elementGen = right.gen(ctx) - val i = ctx.freshName("i") - val getValue = ctx.getValue(arrGen.primitive, right.dataType, i) - s""" - ${arrGen.code} - boolean ${ev.isNull} = false; - boolean ${ev.primitive} = false; - if (!${arrGen.isNull}) { - ${elementGen.code} - if (!${elementGen.isNull}) { - for (int $i = 0; $i < ${arrGen.primitive}.numElements(); $i ++) { - if (${ctx.genEqual(right.dataType, elementGen.primitive, getValue)}) { - ${ev.primitive} = true; - break; - } - } + nullSafeCodeGen(ctx, ev, (arr, value) => { + val i = ctx.freshName("i") + val getValue = ctx.getValue(arr, right.dataType, i) + s""" + for (int $i = 0; $i < $arr.numElements(); $i ++) { + if ($arr.isNullAt($i)) { + ${ev.isNull} = true; + } else if (${ctx.genEqual(right.dataType, value, getValue)}) { + ${ev.isNull} = false; + ${ev.primitive} = true; + break; } } """ + }) } override def prettyName: String = "array_contains" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index fe7dffb815987..65706dba7d975 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.mutable - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenFallback, GeneratedExpressionCode, CodeGenContext} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -103,6 +101,8 @@ case class Not(child: Expression) case class In(value: Expression, list: Seq[Expression]) extends Predicate with ImplicitCastInputTypes { + require(list != null, "list should not be null") + override def inputTypes: Seq[AbstractDataType] = value.dataType +: list.map(_.dataType) override def checkInputDataTypes(): TypeCheckResult = { @@ -116,12 +116,31 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate override def children: Seq[Expression] = value +: list - override def nullable: Boolean = false // TODO: Figure out correct nullability semantics of IN. + override def nullable: Boolean = children.exists(_.nullable) + override def foldable: Boolean = children.forall(_.foldable) + override def toString: String = s"$value IN ${list.mkString("(", ",", ")")}" override def eval(input: InternalRow): Any = { val evaluatedValue = value.eval(input) - list.exists(e => e.eval(input) == evaluatedValue) + if (evaluatedValue == null) { + null + } else { + var hasNull = false + list.foreach { e => + val v = e.eval(input) + if (v == evaluatedValue) { + return true + } else if (v == null) { + hasNull = true + } + } + if (hasNull) { + null + } else { + false + } + } } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -131,7 +150,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate s""" if (!${ev.primitive}) { ${x.code} - if (${ctx.genEqual(value.dataType, valueGen.primitive, x.primitive)}) { + if (${x.isNull}) { + ${ev.isNull} = true; + } else if (${ctx.genEqual(value.dataType, valueGen.primitive, x.primitive)}) { + ${ev.isNull} = false; ${ev.primitive} = true; } } @@ -139,8 +161,10 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate s""" ${valueGen.code} boolean ${ev.primitive} = false; - boolean ${ev.isNull} = false; - $listCode + boolean ${ev.isNull} = ${valueGen.isNull}; + if (!${ev.isNull}) { + $listCode + } """ } } @@ -151,11 +175,22 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate */ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with Predicate { - override def nullable: Boolean = false // TODO: Figure out correct nullability semantics of IN. + require(hset != null, "hset could not be null") + override def toString: String = s"$child INSET ${hset.mkString("(", ",", ")")}" - override def eval(input: InternalRow): Any = { - hset.contains(child.eval(input)) + @transient private[this] lazy val hasNull: Boolean = hset.contains(null) + + override def nullable: Boolean = child.nullable || hasNull + + protected override def nullSafeEval(value: Any): Any = { + if (hset.contains(value)) { + true + } else if (hasNull) { + null + } else { + false + } } def getHSet(): Set[Any] = hset @@ -166,12 +201,20 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with val childGen = child.gen(ctx) ctx.references += this val hsetTerm = ctx.freshName("hset") + val hasNullTerm = ctx.freshName("hasNull") ctx.addMutableState(setName, hsetTerm, s"$hsetTerm = (($InSetName)expressions[${ctx.references.size - 1}]).getHSet();") + ctx.addMutableState("boolean", hasNullTerm, s"$hasNullTerm = $hsetTerm.contains(null);") s""" ${childGen.code} - boolean ${ev.isNull} = false; - boolean ${ev.primitive} = $hsetTerm.contains(${childGen.primitive}); + boolean ${ev.isNull} = ${childGen.isNull}; + boolean ${ev.primitive} = false; + if (!${ev.isNull}) { + ${ev.primitive} = $hsetTerm.contains(${childGen.primitive}); + if (!${ev.primitive} && $hasNullTerm) { + ${ev.isNull} = true; + } + } """ } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 854463dd11c74..a430000bef653 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -395,12 +395,6 @@ object ConstantFolding extends Rule[LogicalPlan] { // Fold expressions that are foldable. case e if e.foldable => Literal.create(e.eval(EmptyRow), e.dataType) - - // Fold "literal in (item1, item2, ..., literal, ...)" into true directly. - case In(Literal(v, _), list) if list.exists { - case Literal(candidate, _) if candidate == v => true - case _ => false - } => Literal.create(true, BooleanType) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala index 95f0e38212a1a..a3e81888dfd0d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionFunctionsSuite.scala @@ -70,14 +70,20 @@ class CollectionFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val a0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType)) val a1 = Literal.create(Seq[String](null, ""), ArrayType(StringType)) val a2 = Literal.create(Seq(null), ArrayType(LongType)) + val a3 = Literal.create(null, ArrayType(StringType)) checkEvaluation(ArrayContains(a0, Literal(1)), true) checkEvaluation(ArrayContains(a0, Literal(0)), false) - checkEvaluation(ArrayContains(a0, Literal(null)), false) + checkEvaluation(ArrayContains(a0, Literal.create(null, IntegerType)), null) checkEvaluation(ArrayContains(a1, Literal("")), true) - checkEvaluation(ArrayContains(a1, Literal(null)), false) + checkEvaluation(ArrayContains(a1, Literal("a")), null) + checkEvaluation(ArrayContains(a1, Literal.create(null, StringType)), null) - checkEvaluation(ArrayContains(a2, Literal(null)), false) + checkEvaluation(ArrayContains(a2, Literal(1L)), null) + checkEvaluation(ArrayContains(a2, Literal.create(null, LongType)), null) + + checkEvaluation(ArrayContains(a3, Literal("")), null) + checkEvaluation(ArrayContains(a3, Literal.create(null, StringType)), null) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index 54c04faddb477..03e7611fce8ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.immutable.HashSet import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.RandomDataGenerator import org.apache.spark.sql.types._ @@ -119,6 +118,12 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { (null, null, null) :: Nil) test("IN") { + checkEvaluation(In(Literal.create(null, IntegerType), Seq(Literal(1), Literal(2))), null) + checkEvaluation(In(Literal.create(null, IntegerType), Seq(Literal.create(null, IntegerType))), + null) + checkEvaluation(In(Literal(1), Seq(Literal.create(null, IntegerType))), null) + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal.create(null, IntegerType))), true) + checkEvaluation(In(Literal(2), Seq(Literal(1), Literal.create(null, IntegerType))), null) checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) @@ -126,14 +131,18 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))), true) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"))), true) + val ns = Literal.create(null, StringType) + checkEvaluation(In(ns, Seq(Literal("1"), Literal("2"))), null) + checkEvaluation(In(ns, Seq(ns)), null) + checkEvaluation(In(Literal("a"), Seq(ns)), null) + checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"), ns)), true) checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^Ba*n"))), true) checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^n"))), false) val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, LongType, BinaryType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) primitiveTypes.map { t => - val dataGen = RandomDataGenerator.forType(t, nullable = false).get + val dataGen = RandomDataGenerator.forType(t, nullable = true).get val inputData = Seq.fill(10) { val value = dataGen.apply() value match { @@ -142,9 +151,17 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { case _ => value } } - val input = inputData.map(Literal(_)) - checkEvaluation(In(input(0), input.slice(1, 10)), - inputData.slice(1, 10).contains(inputData(0))) + val input = inputData.map(Literal.create(_, t)) + val expected = if (inputData(0) == null) { + null + } else if (inputData.slice(1, 10).contains(inputData(0))) { + true + } else if (inputData.slice(1, 10).contains(null)) { + null + } else { + false + } + checkEvaluation(In(input(0), input.slice(1, 10)), expected) } } @@ -158,15 +175,15 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(InSet(one, hS), true) checkEvaluation(InSet(two, hS), true) checkEvaluation(InSet(two, nS), true) - checkEvaluation(InSet(nl, nS), true) checkEvaluation(InSet(three, hS), false) - checkEvaluation(InSet(three, nS), false) - checkEvaluation(And(InSet(one, hS), InSet(two, hS)), true) + checkEvaluation(InSet(three, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, LongType, BinaryType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) primitiveTypes.map { t => - val dataGen = RandomDataGenerator.forType(t, nullable = false).get + val dataGen = RandomDataGenerator.forType(t, nullable = true).get val inputData = Seq.fill(10) { val value = dataGen.apply() value match { @@ -176,8 +193,16 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } } val input = inputData.map(Literal(_)) - checkEvaluation(InSet(input(0), inputData.slice(1, 10).toSet), - inputData.slice(1, 10).contains(inputData(0))) + val expected = if (inputData(0) == null) { + null + } else if (inputData.slice(1, 10).contains(inputData(0))) { + true + } else if (inputData.slice(1, 10).contains(null)) { + null + } else { + false + } + checkEvaluation(InSet(input(0), inputData.slice(1, 10).toSet), expected) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index ec3b2f1edfa05..e67606288f514 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -250,29 +250,14 @@ class ConstantFoldingSuite extends PlanTest { } test("Constant folding test: Fold In(v, list) into true or false") { - var originalQuery = + val originalQuery = testRelation .select('a) .where(In(Literal(1), Seq(Literal(1), Literal(2)))) - var optimized = Optimize.execute(originalQuery.analyze) - - var correctAnswer = - testRelation - .select('a) - .where(Literal(true)) - .analyze - - comparePlans(optimized, correctAnswer) - - originalQuery = - testRelation - .select('a) - .where(In(Literal(1), Seq(Literal(1), 'a.attr))) - - optimized = Optimize.execute(originalQuery.analyze) + val optimized = Optimize.execute(originalQuery.analyze) - correctAnswer = + val correctAnswer = testRelation .select('a) .where(Literal(true)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 9d965258e389d..3a3f19af1473b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -366,10 +366,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("array_contains(a, 1)"), Seq(Row(true), Row(false)) ) - checkAnswer( - df.select(array_contains(array(lit(2), lit(null)), 1)), - Seq(Row(false), Row(false)) - ) // In hive, this errors because null has no type information intercept[AnalysisException] { @@ -382,15 +378,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("array_contains(null, 1)") } - // In hive, if either argument has a matching type has a null value, return false, even if - // the first argument array contains a null and the second argument is null checkAnswer( - df.selectExpr("array_contains(array(array(1), null)[1], 1)"), - Seq(Row(false), Row(false)) + df.selectExpr("array_contains(array(array(1), null)[0], 1)"), + Seq(Row(true), Row(true)) ) checkAnswer( - df.selectExpr("array_contains(array(0, null), array(1, null)[1])"), - Seq(Row(false), Row(false)) + df.selectExpr("array_contains(array(1, null), array(1, null)[0])"), + Seq(Row(true), Row(true)) ) } } From 2a4e00ca4d4e7a148b4ff8ce0ad1c6d517cee55f Mon Sep 17 00:00:00 2001 From: felixcheung Date: Fri, 28 Aug 2015 18:35:01 -0700 Subject: [PATCH 144/232] [SPARK-9803] [SPARKR] Add subset and transform + tests Add subset and transform Also reorganize `[` & `[[` to subset instead of select Note: for transform, transform is very similar to mutate. Spark doesn't seem to replace existing column with the name in mutate (ie. `mutate(df, age = df$age + 2)` - returned DataFrame has 2 columns with the same name 'age'), so therefore not doing that for now in transform. Though it is clearly stated it should replace column with matching name (should I open a JIRA for mutate/transform?) Author: felixcheung Closes #8503 from felixcheung/rsubset_transform. --- R/pkg/NAMESPACE | 2 + R/pkg/R/DataFrame.R | 70 +++++++++++++++++++++++++------- R/pkg/R/generics.R | 10 ++++- R/pkg/inst/tests/test_sparkSQL.R | 20 ++++++++- 4 files changed, 85 insertions(+), 17 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 5286c01986204..9d39630706436 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -69,9 +69,11 @@ exportMethods("arrange", "selectExpr", "show", "showDF", + "subset", "summarize", "summary", "take", + "transform", "unionAll", "unique", "unpersist", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 74de7c81e35a6..8a00238b41d60 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -987,7 +987,7 @@ setMethod("$<-", signature(x = "DataFrame"), setClassUnion("numericOrcharacter", c("numeric", "character")) -#' @rdname select +#' @rdname subset #' @name [[ setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"), function(x, i) { @@ -998,7 +998,7 @@ setMethod("[[", signature(x = "DataFrame", i = "numericOrcharacter"), getColumn(x, i) }) -#' @rdname select +#' @rdname subset #' @name [ setMethod("[", signature(x = "DataFrame", i = "missing"), function(x, i, j, ...) { @@ -1012,7 +1012,7 @@ setMethod("[", signature(x = "DataFrame", i = "missing"), select(x, j) }) -#' @rdname select +#' @rdname subset #' @name [ setMethod("[", signature(x = "DataFrame", i = "Column"), function(x, i, j, ...) { @@ -1020,12 +1020,43 @@ setMethod("[", signature(x = "DataFrame", i = "Column"), # https://stat.ethz.ch/R-manual/R-devel/library/base/html/Extract.data.frame.html filtered <- filter(x, i) if (!missing(j)) { - filtered[, j] + filtered[, j, ...] } else { filtered } }) +#' Subset +#' +#' Return subsets of DataFrame according to given conditions +#' @param x A DataFrame +#' @param subset A logical expression to filter on rows +#' @param select expression for the single Column or a list of columns to select from the DataFrame +#' @return A new DataFrame containing only the rows that meet the condition with selected columns +#' @export +#' @rdname subset +#' @name subset +#' @aliases [ +#' @family subsetting functions +#' @examples +#' \dontrun{ +#' # Columns can be selected using `[[` and `[` +#' df[[2]] == df[["age"]] +#' df[,2] == df[,"age"] +#' df[,c("name", "age")] +#' # Or to filter rows +#' df[df$age > 20,] +#' # DataFrame can be subset on both rows and Columns +#' df[df$name == "Smith", c(1,2)] +#' df[df$age %in% c(19, 30), 1:2] +#' subset(df, df$age %in% c(19, 30), 1:2) +#' subset(df, df$age %in% c(19), select = c(1,2)) +#' } +setMethod("subset", signature(x = "DataFrame"), + function(x, subset, select, ...) { + x[subset, select, ...] + }) + #' Select #' #' Selects a set of columns with names or Column expressions. @@ -1034,6 +1065,8 @@ setMethod("[", signature(x = "DataFrame", i = "Column"), #' @return A new DataFrame with selected columns #' @export #' @rdname select +#' @name select +#' @family subsetting functions #' @examples #' \dontrun{ #' select(df, "*") @@ -1041,15 +1074,8 @@ setMethod("[", signature(x = "DataFrame", i = "Column"), #' select(df, df$name, df$age + 1) #' select(df, c("col1", "col2")) #' select(df, list(df$name, df$age + 1)) -#' # Columns can also be selected using `[[` and `[` -#' df[[2]] == df[["age"]] -#' df[,2] == df[,"age"] -#' df[,c("name", "age")] #' # Similar to R data frames columns can also be selected using `$` #' df$age -#' # It can also be subset on rows and Columns -#' df[df$name == "Smith", c(1,2)] -#' df[df$age %in% c(19, 30), 1:2] #' } setMethod("select", signature(x = "DataFrame", col = "character"), function(x, col, ...) { @@ -1121,7 +1147,7 @@ setMethod("selectExpr", #' @return A DataFrame with the new column added. #' @rdname withColumn #' @name withColumn -#' @aliases mutate +#' @aliases mutate transform #' @export #' @examples #'\dontrun{ @@ -1141,11 +1167,12 @@ setMethod("withColumn", #' #' Return a new DataFrame with the specified columns added. #' -#' @param x A DataFrame +#' @param .data A DataFrame #' @param col a named argument of the form name = col #' @return A new DataFrame with the new columns added. #' @rdname withColumn #' @name mutate +#' @aliases withColumn transform #' @export #' @examples #'\dontrun{ @@ -1155,10 +1182,12 @@ setMethod("withColumn", #' df <- jsonFile(sqlContext, path) #' newDF <- mutate(df, newCol = df$col1 * 5, newCol2 = df$col1 * 2) #' names(newDF) # Will contain newCol, newCol2 +#' newDF2 <- transform(df, newCol = df$col1 / 5, newCol2 = df$col1 * 2) #' } setMethod("mutate", - signature(x = "DataFrame"), - function(x, ...) { + signature(.data = "DataFrame"), + function(.data, ...) { + x <- .data cols <- list(...) stopifnot(length(cols) > 0) stopifnot(class(cols[[1]]) == "Column") @@ -1173,6 +1202,16 @@ setMethod("mutate", do.call(select, c(x, x$"*", cols)) }) +#' @export +#' @rdname withColumn +#' @name transform +#' @aliases withColumn mutate +setMethod("transform", + signature(`_data` = "DataFrame"), + function(`_data`, ...) { + mutate(`_data`, ...) + }) + #' WithColumnRenamed #' #' Rename an existing column in a DataFrame. @@ -1300,6 +1339,7 @@ setMethod("orderBy", #' @return A DataFrame containing only the rows that meet the condition. #' @rdname filter #' @name filter +#' @family subsetting functions #' @export #' @examples #'\dontrun{ diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index b578b8789d2c5..43dd8d283ab6b 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -467,7 +467,7 @@ setGeneric("merge") #' @rdname withColumn #' @export -setGeneric("mutate", function(x, ...) {standardGeneric("mutate") }) +setGeneric("mutate", function(.data, ...) {standardGeneric("mutate") }) #' @rdname arrange #' @export @@ -507,6 +507,10 @@ setGeneric("saveAsTable", function(df, tableName, source, mode, ...) { standardGeneric("saveAsTable") }) +#' @rdname withColumn +#' @export +setGeneric("transform", function(`_data`, ...) {standardGeneric("transform") }) + #' @rdname write.df #' @export setGeneric("write.df", function(df, path, ...) { standardGeneric("write.df") }) @@ -531,6 +535,10 @@ setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") #' @export setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) +# @rdname subset +# @export +setGeneric("subset", function(x, subset, select, ...) { standardGeneric("subset") }) + #' @rdname agg #' @export setGeneric("summarize", function(x,...) { standardGeneric("summarize") }) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 933b11c8ee7e2..0da5e38654732 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -612,6 +612,10 @@ test_that("subsetting", { df5 <- df[df$age %in% c(19), c(1,2)] expect_equal(count(df5), 1) expect_equal(columns(df5), c("name", "age")) + + df6 <- subset(df, df$age %in% c(30), c(1,2)) + expect_equal(count(df6), 1) + expect_equal(columns(df6), c("name", "age")) }) test_that("selectExpr() on a DataFrame", { @@ -1028,7 +1032,7 @@ test_that("withColumn() and withColumnRenamed()", { expect_equal(columns(newDF2)[1], "newerAge") }) -test_that("mutate(), rename() and names()", { +test_that("mutate(), transform(), rename() and names()", { df <- jsonFile(sqlContext, jsonPath) newDF <- mutate(df, newAge = df$age + 2) expect_equal(length(columns(newDF)), 3) @@ -1042,6 +1046,20 @@ test_that("mutate(), rename() and names()", { names(newDF2) <- c("newerName", "evenNewerAge") expect_equal(length(names(newDF2)), 2) expect_equal(names(newDF2)[1], "newerName") + + transformedDF <- transform(df, newAge = -df$age, newAge2 = df$age / 2) + expect_equal(length(columns(transformedDF)), 4) + expect_equal(columns(transformedDF)[3], "newAge") + expect_equal(columns(transformedDF)[4], "newAge2") + expect_equal(first(filter(transformedDF, transformedDF$name == "Andy"))$newAge, -30) + + # test if transform on local data frames works + # ensure the proper signature is used - otherwise this will fail to run + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + expect_equal(nrow(result), 153) + expect_equal(ncol(result), 2) + detach(airquality) }) test_that("write.df() on DataFrame and works with parquetFile", { From e8ea5bafee9ca734edf62021145d0c2d5491cba8 Mon Sep 17 00:00:00 2001 From: martinzapletal Date: Fri, 28 Aug 2015 21:03:48 -0700 Subject: [PATCH 145/232] [SPARK-9910] [ML] User guide for train validation split Author: martinzapletal Closes #8377 from zapletal-martin/SPARK-9910. --- docs/ml-guide.md | 117 ++++++++++++++++++ .../ml/JavaTrainValidationSplitExample.java | 90 ++++++++++++++ .../ml/TrainValidationSplitExample.scala | 80 ++++++++++++ 3 files changed, 287 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala diff --git a/docs/ml-guide.md b/docs/ml-guide.md index ce53400b6ee56..a92a285f3af85 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -872,3 +872,120 @@ jsc.stop(); + +## Example: Model Selection via Train Validation Split +In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. +`TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in + case of `CrossValidator`. It is therefore less expensive, + but will not produce as reliable results when the training dataset is not sufficiently large.. + +`TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, +and an `Evaluator`. +It begins by splitting the dataset into two parts using `trainRatio` parameter +which are used as separate training and test datasets. For example with `$trainRatio=0.75$` (default), +`TrainValidationSplit` will generate a training and test dataset pair where 75% of the data is used for training and 25% for validation. +Similar to `CrossValidator`, `TrainValidationSplit` also iterates through the set of `ParamMap`s. +For each combination of parameters, it trains the given `Estimator` and evaluates it using the given `Evaluator`. +The `ParamMap` which produces the best evaluation metric is selected as the best option. +`TrainValidationSplit` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. + +
+ +
+{% highlight scala %} +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} +import org.apache.spark.mllib.util.MLUtils + +// Prepare training and test data. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() +val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) + +val lr = new LinearRegression() + +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// TrainValidationSplit will try all combinations of values and determine best model using +// the evaluator. +val paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.01)) + .addGrid(lr.fitIntercept, Array(true, false)) + .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) + .build() + +// In this case the estimator is simply the linear regression. +// A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +val trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator) + .setEstimatorParamMaps(paramGrid) + +// 80% of the data will be used for training and the remaining 20% for validation. +trainValidationSplit.setTrainRatio(0.8) + +// Run train validation split, and choose the best set of parameters. +val model = trainValidationSplit.fit(training) + +// Make predictions on test data. model is the model with combination of parameters +// that performed best. +model.transform(test) + .select("features", "label", "prediction") + .show() + +{% endhighlight %} +
+ +
+{% highlight java %} +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.tuning.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.DataFrame; + +DataFrame data = jsql.createDataFrame( + MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt"), + LabeledPoint.class); + +// Prepare training and test data. +DataFrame[] splits = data.randomSplit(new double [] {0.9, 0.1}, 12345); +DataFrame training = splits[0]; +DataFrame test = splits[1]; + +LinearRegression lr = new LinearRegression(); + +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// TrainValidationSplit will try all combinations of values and determine best model using +// the evaluator. +ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam(), new double[] {0.1, 0.01}) + .addGrid(lr.fitIntercept()) + .addGrid(lr.elasticNetParam(), new double[] {0.0, 0.5, 1.0}) + .build(); + +// In this case the estimator is simply the linear regression. +// A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +TrainValidationSplit trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator()) + .setEstimatorParamMaps(paramGrid); + +// 80% of the data will be used for training and the remaining 20% for validation. +trainValidationSplit.setTrainRatio(0.8); + +// Run train validation split, and choose the best set of parameters. +TrainValidationSplitModel model = trainValidationSplit.fit(training); + +// Make predictions on test data. model is the model with combination of parameters +// that performed best. +model.transform(test) + .select("features", "label", "prediction") + .show(); + +{% endhighlight %} +
+ +
diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java new file mode 100644 index 0000000000000..23f834ab4332b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTrainValidationSplitExample.java @@ -0,0 +1,90 @@ +/* + * 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.ml; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.evaluation.RegressionEvaluator; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.regression.LinearRegression; +import org.apache.spark.ml.tuning.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; + +/** + * A simple example demonstrating model selection using TrainValidationSplit. + * + * The example is based on {@link org.apache.spark.examples.ml.JavaSimpleParamsExample} + * using linear regression. + * + * Run with + * {{{ + * bin/run-example ml.JavaTrainValidationSplitExample + * }}} + */ +public class JavaTrainValidationSplitExample { + + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaTrainValidationSplitExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + SQLContext jsql = new SQLContext(jsc); + + DataFrame data = jsql.createDataFrame( + MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt"), + LabeledPoint.class); + + // Prepare training and test data. + DataFrame[] splits = data.randomSplit(new double [] {0.9, 0.1}, 12345); + DataFrame training = splits[0]; + DataFrame test = splits[1]; + + LinearRegression lr = new LinearRegression(); + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // TrainValidationSplit will try all combinations of values and determine best model using + // the evaluator. + ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam(), new double[] {0.1, 0.01}) + .addGrid(lr.fitIntercept()) + .addGrid(lr.elasticNetParam(), new double[] {0.0, 0.5, 1.0}) + .build(); + + // In this case the estimator is simply the linear regression. + // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + TrainValidationSplit trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator()) + .setEstimatorParamMaps(paramGrid); + + // 80% of the data will be used for training and the remaining 20% for validation. + trainValidationSplit.setTrainRatio(0.8); + + // Run train validation split, and choose the best set of parameters. + TrainValidationSplitModel model = trainValidationSplit.fit(training); + + // Make predictions on test data. model is the model with combination of parameters + // that performed best. + model.transform(test) + .select("features", "label", "prediction") + .show(); + + jsc.stop(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala new file mode 100644 index 0000000000000..1abdf219b1c00 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/TrainValidationSplitExample.scala @@ -0,0 +1,80 @@ +/* + * 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.ml + +import org.apache.spark.ml.evaluation.RegressionEvaluator +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.ml.tuning.{ParamGridBuilder, TrainValidationSplit} +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.sql.SQLContext +import org.apache.spark.{SparkConf, SparkContext} + +/** + * A simple example demonstrating model selection using TrainValidationSplit. + * + * The example is based on [[SimpleParamsExample]] using linear regression. + * Run with + * {{{ + * bin/run-example ml.TrainValidationSplitExample + * }}} + */ +object TrainValidationSplitExample { + + def main(args: Array[String]): Unit = { + val conf = new SparkConf().setAppName("TrainValidationSplitExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Prepare training and test data. + val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").toDF() + val Array(training, test) = data.randomSplit(Array(0.9, 0.1), seed = 12345) + + val lr = new LinearRegression() + + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // TrainValidationSplit will try all combinations of values and determine best model using + // the evaluator. + val paramGrid = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.1, 0.01)) + .addGrid(lr.fitIntercept, Array(true, false)) + .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) + .build() + + // In this case the estimator is simply the linear regression. + // A TrainValidationSplit requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + val trainValidationSplit = new TrainValidationSplit() + .setEstimator(lr) + .setEvaluator(new RegressionEvaluator) + .setEstimatorParamMaps(paramGrid) + + // 80% of the data will be used for training and the remaining 20% for validation. + trainValidationSplit.setTrainRatio(0.8) + + // Run train validation split, and choose the best set of parameters. + val model = trainValidationSplit.fit(training) + + // Make predictions on test data. model is the model with combination of parameters + // that performed best. + model.transform(test) + .select("features", "label", "prediction") + .show() + + sc.stop() + } +} From 5369be806848f43cb87c76504258c4e7de930c90 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 29 Aug 2015 13:20:22 -0700 Subject: [PATCH 146/232] [SPARK-10350] [DOC] [SQL] Removed duplicated option description from SQL guide Author: GuoQiang Li Closes #8520 from witgo/SPARK-10350. --- docs/sql-programming-guide.md | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e8eb88488ee24..6a1b0fbfa1eb3 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1405,16 +1405,6 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`

- - - - -
Property NameDefaultMeaning
spark.akka.failure-detector.threshold300.0 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be - enabled again, if you plan to use this feature (Not recommended). This maps to akka's - `akka.remote.transport-failure-detector.threshold`. Tune this in combination of - `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to. -
spark.akka.frameSize 128
spark.sql.parquet.mergeSchemafalse -

- When true, the Parquet data source merges schemas collected from all data files, otherwise the - schema is picked from the summary file or a random data file if no summary file is available. -

-
## JSON Datasets From 24ffa85c002a095ffb270175ec838995d3ed5469 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 29 Aug 2015 13:24:32 -0700 Subject: [PATCH 147/232] [SPARK-10289] [SQL] A direct write API for testing Parquet This PR introduces a direct write API for testing Parquet. It's a DSL flavored version of the [`writeDirect` method] [1] comes with parquet-avro testing code. With this API, it's much easier to construct arbitrary Parquet structures. It's especially useful when adding regression tests for various compatibility corner cases. Sample usage of this API can be found in the new test case added in `ParquetThriftCompatibilitySuite`. [1]: https://github.com/apache/parquet-mr/blob/apache-parquet-1.8.1/parquet-avro/src/test/java/org/apache/parquet/avro/TestArrayCompatibility.java#L945-L972 Author: Cheng Lian Closes #8454 from liancheng/spark-10289/parquet-testing-direct-write-api. --- .../parquet/ParquetCompatibilityTest.scala | 84 +++++++++++++-- .../ParquetThriftCompatibilitySuite.scala | 100 +++++++++++++++--- 2 files changed, 160 insertions(+), 24 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala index df68432faeeb3..91f3ce4d34c8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.execution.datasources.parquet -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, mapAsJavaMapConverter, seqAsJavaListConverter} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} -import org.apache.parquet.hadoop.ParquetFileReader -import org.apache.parquet.schema.MessageType +import org.apache.parquet.hadoop.api.WriteSupport +import org.apache.parquet.hadoop.api.WriteSupport.WriteContext +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetWriter} +import org.apache.parquet.io.api.RecordConsumer +import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.sql.QueryTest @@ -38,11 +42,10 @@ private[sql] abstract class ParquetCompatibilityTest extends QueryTest with Parq val fs = fsPath.getFileSystem(configuration) val parquetFiles = fs.listStatus(fsPath, new PathFilter { override def accept(path: Path): Boolean = pathFilter(path) - }).toSeq + }).toSeq.asJava - val footers = - ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles.asJava, true) - footers.iterator().next().getParquetMetadata.getFileMetaData.getSchema + val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true) + footers.asScala.head.getParquetMetadata.getFileMetaData.getSchema } protected def logParquetSchema(path: String): Unit = { @@ -53,8 +56,69 @@ private[sql] abstract class ParquetCompatibilityTest extends QueryTest with Parq } } -object ParquetCompatibilityTest { - def makeNullable[T <: AnyRef](i: Int)(f: => T): T = { - if (i % 3 == 0) null.asInstanceOf[T] else f +private[sql] object ParquetCompatibilityTest { + implicit class RecordConsumerDSL(consumer: RecordConsumer) { + def message(f: => Unit): Unit = { + consumer.startMessage() + f + consumer.endMessage() + } + + def group(f: => Unit): Unit = { + consumer.startGroup() + f + consumer.endGroup() + } + + def field(name: String, index: Int)(f: => Unit): Unit = { + consumer.startField(name, index) + f + consumer.endField(name, index) + } + } + + /** + * A testing Parquet [[WriteSupport]] implementation used to write manually constructed Parquet + * records with arbitrary structures. + */ + private class DirectWriteSupport(schema: MessageType, metadata: Map[String, String]) + extends WriteSupport[RecordConsumer => Unit] { + + private var recordConsumer: RecordConsumer = _ + + override def init(configuration: Configuration): WriteContext = { + new WriteContext(schema, metadata.asJava) + } + + override def write(recordWriter: RecordConsumer => Unit): Unit = { + recordWriter.apply(recordConsumer) + } + + override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { + this.recordConsumer = recordConsumer + } + } + + /** + * Writes arbitrary messages conforming to a given `schema` to a Parquet file located by `path`. + * Records are produced by `recordWriters`. + */ + def writeDirect(path: String, schema: String, recordWriters: (RecordConsumer => Unit)*): Unit = { + writeDirect(path, schema, Map.empty[String, String], recordWriters: _*) + } + + /** + * Writes arbitrary messages conforming to a given `schema` to a Parquet file located by `path` + * with given user-defined key-value `metadata`. Records are produced by `recordWriters`. + */ + def writeDirect( + path: String, + schema: String, + metadata: Map[String, String], + recordWriters: (RecordConsumer => Unit)*): Unit = { + val messageType = MessageTypeParser.parseMessageType(schema) + val writeSupport = new DirectWriteSupport(messageType, metadata) + val parquetWriter = new ParquetWriter[RecordConsumer => Unit](new Path(path), writeSupport) + try recordWriters.foreach(parquetWriter.write) finally parquetWriter.close() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala index b789c5a106e56..88a3d878f97fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala @@ -33,11 +33,9 @@ class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with Shar """.stripMargin) checkAnswer(sqlContext.read.parquet(parquetFilePath.toString), (0 until 10).map { i => - def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) - val suits = Array("SPADES", "HEARTS", "DIAMONDS", "CLUBS") - Row( + val nonNullablePrimitiveValues = Seq( i % 2 == 0, i.toByte, (i + 1).toShort, @@ -50,18 +48,15 @@ class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with Shar s"val_$i", s"val_$i", // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings - suits(i % 4), - - nullable(i % 2 == 0: java.lang.Boolean), - nullable(i.toByte: java.lang.Byte), - nullable((i + 1).toShort: java.lang.Short), - nullable(i + 2: Integer), - nullable((i * 10).toLong: java.lang.Long), - nullable(i.toDouble + 0.2d: java.lang.Double), - nullable(s"val_$i"), - nullable(s"val_$i"), - nullable(suits(i % 4)), + suits(i % 4)) + + val nullablePrimitiveValues = if (i % 3 == 0) { + Seq.fill(nonNullablePrimitiveValues.length)(null) + } else { + nonNullablePrimitiveValues + } + val complexValues = Seq( Seq.tabulate(3)(n => s"arr_${i + n}"), // Thrift `SET`s are converted to Parquet `LIST`s Seq(i), @@ -71,6 +66,83 @@ class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest with Shar Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") } }.toMap) + + Row(nonNullablePrimitiveValues ++ nullablePrimitiveValues ++ complexValues: _*) }) } + + test("SPARK-10136 list of primitive list") { + withTempPath { dir => + val path = dir.getCanonicalPath + + // This Parquet schema is translated from the following Thrift schema: + // + // struct ListOfPrimitiveList { + // 1: list> f; + // } + val schema = + s"""message ListOfPrimitiveList { + | required group f (LIST) { + | repeated group f_tuple (LIST) { + | repeated int32 f_tuple_tuple; + | } + | } + |} + """.stripMargin + + writeDirect(path, schema, { rc => + rc.message { + rc.field("f", 0) { + rc.group { + rc.field("f_tuple", 0) { + rc.group { + rc.field("f_tuple_tuple", 0) { + rc.addInteger(0) + rc.addInteger(1) + } + } + + rc.group { + rc.field("f_tuple_tuple", 0) { + rc.addInteger(2) + rc.addInteger(3) + } + } + } + } + } + } + }, { rc => + rc.message { + rc.field("f", 0) { + rc.group { + rc.field("f_tuple", 0) { + rc.group { + rc.field("f_tuple_tuple", 0) { + rc.addInteger(4) + rc.addInteger(5) + } + } + + rc.group { + rc.field("f_tuple_tuple", 0) { + rc.addInteger(6) + rc.addInteger(7) + } + } + } + } + } + } + }) + + logParquetSchema(path) + + checkAnswer( + sqlContext.read.parquet(path), + Seq( + Row(Seq(Seq(0, 1), Seq(2, 3))), + Row(Seq(Seq(4, 5), Seq(6, 7))))) + } + } } From 5c3d16a9b91bb9a458d3ba141f7bef525cf3d285 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 29 Aug 2015 13:26:01 -0700 Subject: [PATCH 148/232] [SPARK-10344] [SQL] Add tests for extraStrategies Actually using this API requires access to a lot of classes that we might make private by accident. I've added some tests to prevent this. Author: Michael Armbrust Closes #8516 from marmbrus/extraStrategiesTests. --- .../spark/sql/ExtraStrategiesSuite.scala | 67 +++++++++++++++++++ .../spark/sql/test/SharedSQLContext.scala | 2 +- 2 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala new file mode 100644 index 0000000000000..8d2f45d70308b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExtraStrategiesSuite.scala @@ -0,0 +1,67 @@ +/* + * 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 test.org.apache.spark.sql + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Literal, GenericInternalRow, Attribute} +import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.{Row, Strategy, QueryTest} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.unsafe.types.UTF8String + +case class FastOperator(output: Seq[Attribute]) extends SparkPlan { + + override protected def doExecute(): RDD[InternalRow] = { + val str = Literal("so fast").value + val row = new GenericInternalRow(Array[Any](str)) + sparkContext.parallelize(Seq(row)) + } + + override def children: Seq[SparkPlan] = Nil +} + +object TestStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case Project(Seq(attr), _) if attr.name == "a" => + FastOperator(attr.toAttribute :: Nil) :: Nil + case _ => Nil + } +} + +class ExtraStrategiesSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("insert an extraStrategy") { + try { + sqlContext.experimental.extraStrategies = TestStrategy :: Nil + + val df = sqlContext.sparkContext.parallelize(Seq(("so slow", 1))).toDF("a", "b") + checkAnswer( + df.select("a"), + Row("so fast")) + + checkAnswer( + df.select("a", "b"), + Row("so slow", 1)) + } finally { + sqlContext.experimental.extraStrategies = Nil + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index 8a061b6bc690d..d23c6a0732669 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.{ColumnName, SQLContext} /** * Helper trait for SQL test suites where all tests share a single [[TestSQLContext]]. */ -private[sql] trait SharedSQLContext extends SQLTestUtils { +trait SharedSQLContext extends SQLTestUtils { /** * The [[TestSQLContext]] to use for all tests in this suite. From 277148b285748e863f2b9fdf6cf12963977f91ca Mon Sep 17 00:00:00 2001 From: wangwei Date: Sat, 29 Aug 2015 13:29:50 -0700 Subject: [PATCH 149/232] [SPARK-10226] [SQL] Fix exclamation mark issue in SparkSQL When I tested the latest version of spark with exclamation mark, I got some errors. Then I reseted the spark version and found that commit id "a2409d1c8e8ddec04b529ac6f6a12b5993f0eeda" brought the bug. With jline version changing from 0.9.94 to 2.12 after this commit, exclamation mark would be treated as a special character in ConsoleReader. Author: wangwei Closes #8420 from small-wang/jline-SPARK-10226. --- .../apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index a29df567983b1..b5073961a1c84 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -171,6 +171,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { val reader = new ConsoleReader() reader.setBellEnabled(false) + reader.setExpandEvents(false) // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) CliDriver.getCommandCompleter.foreach((e) => reader.addCompleter(e)) From 6a6f3c91ee1f63dd464eb03d156d02c1a5887d88 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Aug 2015 13:36:25 -0700 Subject: [PATCH 150/232] [SPARK-10330] Use SparkHadoopUtil TaskAttemptContext reflection methods in more places SparkHadoopUtil contains methods that use reflection to work around TaskAttemptContext binary incompatibilities between Hadoop 1.x and 2.x. We should use these methods in more places. Author: Josh Rosen Closes #8499 from JoshRosen/use-hadoop-reflection-in-more-places. --- .../sql/execution/datasources/WriterContainer.scala | 10 +++++++--- .../sql/execution/datasources/json/JSONRelation.scala | 7 +++++-- .../datasources/parquet/ParquetRelation.scala | 7 +++++-- .../org/apache/spark/sql/hive/orc/OrcRelation.scala | 9 ++++++--- .../apache/spark/sql/sources/SimpleTextRelation.scala | 7 +++++-- 5 files changed, 28 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 78f48a5cd72c7..879fd69863211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter} import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.sql._ @@ -145,7 +146,8 @@ private[sql] abstract class BaseWriterContainer( "because spark.speculation is configured to be true.") defaultOutputCommitter } else { - val committerClass = context.getConfiguration.getClass( + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + val committerClass = configuration.getClass( SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) Option(committerClass).map { clazz => @@ -227,7 +229,8 @@ private[sql] class DefaultWriterContainer( def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { executorSideSetup(taskContext) - taskAttemptContext.getConfiguration.set("spark.sql.sources.output.path", outputPath) + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(taskAttemptContext) + configuration.set("spark.sql.sources.output.path", outputPath) val writer = outputWriterFactory.newInstance(getWorkPath, dataSchema, taskAttemptContext) writer.initConverter(dataSchema) @@ -395,7 +398,8 @@ private[sql] class DynamicPartitionWriterContainer( def newOutputWriter(key: InternalRow): OutputWriter = { val partitionPath = getPartitionString(key).getString(0) val path = new Path(getWorkPath, partitionPath) - taskAttemptContext.getConfiguration.set( + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(taskAttemptContext) + configuration.set( "spark.sql.sources.output.path", new Path(outputPath, partitionPath).toString) val newWriter = outputWriterFactory.newInstance(path.toString, dataSchema, taskAttemptContext) newWriter.initConverter(dataSchema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala index ab8ca5f748f24..7a49157d9e72c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JSONRelation.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.spark.Logging import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -169,8 +170,10 @@ private[json] class JsonOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val uniqueWriteJobId = context.getConfiguration.get("spark.sql.sources.writeJobUUID") - val split = context.getTaskAttemptID.getTaskID.getId + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) + val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") } }.getRecordWriter(context) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 64982f37cf872..c6bbc392cad4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -40,6 +40,7 @@ import org.apache.parquet.{Log => ApacheParquetLog} import org.slf4j.bridge.SLF4JBridgeHandler import org.apache.spark.broadcast.Broadcast +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -81,8 +82,10 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all // partitions in the case of dynamic partitioning. override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val uniqueWriteJobId = context.getConfiguration.get("spark.sql.sources.writeJobUUID") - val split = context.getTaskAttemptID.getTaskID.getId + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) + val split = taskAttemptId.getTaskID.getId new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 1cff5cf9c3543..4eeca9aec12bd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.Logging +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.{HadoopRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow @@ -77,7 +78,8 @@ private[orc] class OrcOutputWriter( }.mkString(":")) val serde = new OrcSerde - serde.initialize(context.getConfiguration, table) + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + serde.initialize(configuration, table) serde } @@ -109,9 +111,10 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - val conf = context.getConfiguration + val conf = SparkHadoopUtil.get.getConfigurationFromJobContext(context) val uniqueWriteJobId = conf.get("spark.sql.sources.writeJobUUID") - val partition = context.getTaskAttemptID.getTaskID.getId + val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) + val partition = taskAttemptId.getTaskID.getId val filename = f"part-r-$partition%05d-$uniqueWriteJobId.orc" new OrcOutputFormat().getRecordWriter( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index e8141923a9b5c..527ca7a81cad8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputForma import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} import org.apache.spark.rdd.RDD +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types.{DataType, StructType} @@ -53,8 +54,10 @@ class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullW numberFormat.setGroupingUsed(false) override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val uniqueWriteJobId = context.getConfiguration.get("spark.sql.sources.writeJobUUID") - val split = context.getTaskAttemptID.getTaskID.getId + val configuration = SparkHadoopUtil.get.getConfigurationFromJobContext(context) + val uniqueWriteJobId = configuration.get("spark.sql.sources.writeJobUUID") + val taskAttemptId = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(context) + val split = taskAttemptId.getTaskID.getId val name = FileOutputFormat.getOutputName(context) new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId") } From 097a7e36e0bf7290b1879331375bacc905583bd3 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 29 Aug 2015 16:39:40 -0700 Subject: [PATCH 151/232] [SPARK-10339] [SPARK-10334] [SPARK-10301] [SQL] Partitioned table scan can OOM driver and throw a better error message when users need to enable parquet schema merging This fixes the problem that scanning partitioned table causes driver have a high memory pressure and takes down the cluster. Also, with this fix, we will be able to correctly show the query plan of a query consuming partitioned tables. https://issues.apache.org/jira/browse/SPARK-10339 https://issues.apache.org/jira/browse/SPARK-10334 Finally, this PR squeeze in a "quick fix" for SPARK-10301. It is not a real fix, but it just throw a better error message to let user know what to do. Author: Yin Huai Closes #8515 from yhuai/partitionedTableScan. --- .../datasources/DataSourceStrategy.scala | 85 ++++++++++--------- .../parquet/CatalystRowConverter.scala | 7 ++ .../ParquetHadoopFsRelationSuite.scala | 15 +++- 3 files changed, 65 insertions(+), 42 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 6c1ef6a6df887..c58213155daa8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.sql.{SaveMode, Strategy, execution, sources, _} @@ -121,7 +122,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { projections: Seq[NamedExpression], filters: Seq[Expression], partitionColumns: StructType, - partitions: Array[Partition]) = { + partitions: Array[Partition]): SparkPlan = { val relation = logicalRelation.relation.asInstanceOf[HadoopFsRelation] // Because we are creating one RDD per partition, we need to have a shared HadoopConf. @@ -130,49 +131,51 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { val confBroadcast = relation.sqlContext.sparkContext.broadcast(new SerializableConfiguration(sharedHadoopConf)) - // Builds RDD[Row]s for each selected partition. - val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => - // The table scan operator (PhysicalRDD) which retrieves required columns from data files. - // Notice that the schema of data files, represented by `relation.dataSchema`, may contain - // some partition column(s). - val scan = - pruneFilterProject( - logicalRelation, - projections, - filters, - (columns: Seq[Attribute], filters) => { - val partitionColNames = partitionColumns.fieldNames - - // Don't scan any partition columns to save I/O. Here we are being optimistic and - // assuming partition columns data stored in data files are always consistent with those - // partition values encoded in partition directory paths. - val needed = columns.filterNot(a => partitionColNames.contains(a.name)) - val dataRows = - relation.buildScan(needed.map(_.name).toArray, filters, Array(dir), confBroadcast) - - // Merges data values with partition values. - mergeWithPartitionValues( - relation.schema, - columns.map(_.name).toArray, - partitionColNames, - partitionValues, - toCatalystRDD(logicalRelation, needed, dataRows)) - }) - - scan.execute() - } + // Now, we create a scan builder, which will be used by pruneFilterProject. This scan builder + // will union all partitions and attach partition values if needed. + val scanBuilder = { + (columns: Seq[Attribute], filters: Array[Filter]) => { + // Builds RDD[Row]s for each selected partition. + val perPartitionRows = partitions.map { case Partition(partitionValues, dir) => + val partitionColNames = partitionColumns.fieldNames + + // Don't scan any partition columns to save I/O. Here we are being optimistic and + // assuming partition columns data stored in data files are always consistent with those + // partition values encoded in partition directory paths. + val needed = columns.filterNot(a => partitionColNames.contains(a.name)) + val dataRows = + relation.buildScan(needed.map(_.name).toArray, filters, Array(dir), confBroadcast) + + // Merges data values with partition values. + mergeWithPartitionValues( + relation.schema, + columns.map(_.name).toArray, + partitionColNames, + partitionValues, + toCatalystRDD(logicalRelation, needed, dataRows)) + } + + val unionedRows = + if (perPartitionRows.length == 0) { + relation.sqlContext.emptyResult + } else { + new UnionRDD(relation.sqlContext.sparkContext, perPartitionRows) + } - val unionedRows = - if (perPartitionRows.length == 0) { - relation.sqlContext.emptyResult - } else { - new UnionRDD(relation.sqlContext.sparkContext, perPartitionRows) + unionedRows } + } + + // Create the scan operator. If needed, add Filter and/or Project on top of the scan. + // The added Filter/Project is on top of the unioned RDD. We do not want to create + // one Filter/Project for every partition. + val sparkPlan = pruneFilterProject( + logicalRelation, + projections, + filters, + scanBuilder) - execution.PhysicalRDD.createFromDataSource( - projections.map(_.toAttribute), - unionedRows, - logicalRelation.relation) + sparkPlan } // TODO: refactor this thing. It is very complicated because it does projection internally. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index f682ca0d8ff4f..fe13dfbbed385 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -196,6 +196,13 @@ private[parquet] class CatalystRowConverter( } } + if (paddedParquetFields.length != catalystType.length) { + throw new UnsupportedOperationException( + "A Parquet file's schema has different number of fields with the table schema. " + + "Please enable schema merging by setting \"mergeSchema\" to true when load " + + "a Parquet dataset or set spark.sql.parquet.mergeSchema to true in SQLConf.") + } + paddedParquetFields.zip(catalystType).zipWithIndex.map { case ((parquetFieldType, catalystField), ordinal) => // Converted field value should be set to the `ordinal`-th cell of `currentRow` diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala index cb4cedddbfddd..06dadbb5feab0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/ParquetHadoopFsRelationSuite.scala @@ -23,7 +23,7 @@ import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.{execution, AnalysisException, SaveMode} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} @@ -136,4 +136,17 @@ class ParquetHadoopFsRelationSuite extends HadoopFsRelationTest { assert(fs.exists(commonSummaryPath)) } } + + test("SPARK-10334 Projections and filters should be kept in physical plan") { + withTempPath { dir => + val path = dir.getCanonicalPath + + sqlContext.range(2).select('id as 'a, 'id as 'b).write.partitionBy("b").parquet(path) + val df = sqlContext.read.parquet(path).filter('a === 0).select('b) + val physicalPlan = df.queryExecution.executedPlan + + assert(physicalPlan.collect { case p: execution.Project => p }.length === 1) + assert(physicalPlan.collect { case p: execution.Filter => p }.length === 1) + } + } } From 13f5f8ec97c6886346641b73bd99004e0d70836c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sat, 29 Aug 2015 18:10:44 -0700 Subject: [PATCH 152/232] [SPARK-9986] [SPARK-9991] [SPARK-9993] [SQL] Create a simple test framework for local operators This PR includes the following changes: - Add `LocalNodeTest` for local operator tests and add unit tests for FilterNode and ProjectNode. - Add `LimitNode` and `UnionNode` and their unit tests to show how to use `LocalNodeTest`. (SPARK-9991, SPARK-9993) Author: zsxwing Closes #8464 from zsxwing/local-execution. --- .../sql/execution/local/FilterNode.scala | 6 +- .../spark/sql/execution/local/LimitNode.scala | 45 ++++++ .../spark/sql/execution/local/LocalNode.scala | 13 +- .../sql/execution/local/ProjectNode.scala | 4 +- .../sql/execution/local/SeqScanNode.scala | 2 +- .../spark/sql/execution/local/UnionNode.scala | 72 +++++++++ .../spark/sql/execution/SparkPlanTest.scala | 46 +----- .../sql/execution/local/FilterNodeSuite.scala | 41 +++++ .../sql/execution/local/LimitNodeSuite.scala | 39 +++++ .../sql/execution/local/LocalNodeTest.scala | 146 ++++++++++++++++++ .../execution/local/ProjectNodeSuite.scala | 44 ++++++ .../sql/execution/local/UnionNodeSuite.scala | 52 +++++++ .../apache/spark/sql/test/SQLTestData.scala | 8 + .../apache/spark/sql/test/SQLTestUtils.scala | 46 +++++- 14 files changed, 509 insertions(+), 55 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/local/LimitNode.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/local/UnionNode.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/FilterNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/FilterNode.scala index a485a1a1d7ae4..81dd37c7da733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/FilterNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/FilterNode.scala @@ -35,13 +35,13 @@ case class FilterNode(condition: Expression, child: LocalNode) extends UnaryLoca override def next(): Boolean = { var found = false - while (child.next() && !found) { - found = predicate.apply(child.get()) + while (!found && child.next()) { + found = predicate.apply(child.fetch()) } found } - override def get(): InternalRow = child.get() + override def fetch(): InternalRow = child.fetch() override def close(): Unit = child.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LimitNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LimitNode.scala new file mode 100644 index 0000000000000..fffc52abf6dd5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LimitNode.scala @@ -0,0 +1,45 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute + + +case class LimitNode(limit: Int, child: LocalNode) extends UnaryLocalNode { + + private[this] var count = 0 + + override def output: Seq[Attribute] = child.output + + override def open(): Unit = child.open() + + override def close(): Unit = child.close() + + override def fetch(): InternalRow = child.fetch() + + override def next(): Boolean = { + if (count < limit) { + count += 1 + child.next() + } else { + false + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala index 341c81438e6d6..1c4469acbf264 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/LocalNode.scala @@ -48,10 +48,10 @@ abstract class LocalNode extends TreeNode[LocalNode] { /** * Returns the current tuple. */ - def get(): InternalRow + def fetch(): InternalRow /** - * Closes the iterator and releases all resources. + * Closes the iterator and releases all resources. It should be idempotent. * * Implementations of this must also call the `close()` function of its children. */ @@ -64,10 +64,13 @@ abstract class LocalNode extends TreeNode[LocalNode] { val converter = CatalystTypeConverters.createToScalaConverter(StructType.fromAttributes(output)) val result = new scala.collection.mutable.ArrayBuffer[Row] open() - while (next()) { - result += converter.apply(get()).asInstanceOf[Row] + try { + while (next()) { + result += converter.apply(fetch()).asInstanceOf[Row] + } + } finally { + close() } - close() result } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ProjectNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ProjectNode.scala index e574d1473cdcb..9b8a4fe493026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ProjectNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/ProjectNode.scala @@ -34,8 +34,8 @@ case class ProjectNode(projectList: Seq[NamedExpression], child: LocalNode) exte override def next(): Boolean = child.next() - override def get(): InternalRow = { - project.apply(child.get()) + override def fetch(): InternalRow = { + project.apply(child.fetch()) } override def close(): Unit = child.close() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SeqScanNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SeqScanNode.scala index 994de8afa9a02..242cb66e07b7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SeqScanNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/SeqScanNode.scala @@ -41,7 +41,7 @@ case class SeqScanNode(output: Seq[Attribute], data: Seq[InternalRow]) extends L } } - override def get(): InternalRow = currentRow + override def fetch(): InternalRow = currentRow override def close(): Unit = { // Do nothing diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/local/UnionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/UnionNode.scala new file mode 100644 index 0000000000000..ba4aa7671aebd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/local/UnionNode.scala @@ -0,0 +1,72 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class UnionNode(children: Seq[LocalNode]) extends LocalNode { + + override def output: Seq[Attribute] = children.head.output + + private[this] var currentChild: LocalNode = _ + + private[this] var nextChildIndex: Int = _ + + override def open(): Unit = { + currentChild = children.head + currentChild.open() + nextChildIndex = 1 + } + + private def advanceToNextChild(): Boolean = { + var found = false + var exit = false + while (!exit && !found) { + if (currentChild != null) { + currentChild.close() + } + if (nextChildIndex >= children.size) { + found = false + exit = true + } else { + currentChild = children(nextChildIndex) + nextChildIndex += 1 + currentChild.open() + found = currentChild.next() + } + } + found + } + + override def close(): Unit = { + if (currentChild != null) { + currentChild.close() + } + } + + override def fetch(): InternalRow = currentChild.fetch() + + override def next(): Boolean = { + if (currentChild.next()) { + true + } else { + advanceToNextChild() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index 3a87f374d94b0..5ab8f44faebf6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -24,7 +24,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{DataFrame, DataFrameHolder, Row, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.test.SQLTestUtils /** * Base class for writing tests for individual physical operators. For an example of how this @@ -184,7 +184,7 @@ object SparkPlanTest { return Some(errorMessage) } - compareAnswers(actualAnswer, expectedAnswer, sortAnswers).map { errorMessage => + SQLTestUtils.compareAnswers(actualAnswer, expectedAnswer, sortAnswers).map { errorMessage => s""" | Results do not match. | Actual result Spark plan: @@ -229,7 +229,7 @@ object SparkPlanTest { return Some(errorMessage) } - compareAnswers(sparkAnswer, expectedAnswer, sortAnswers).map { errorMessage => + SQLTestUtils.compareAnswers(sparkAnswer, expectedAnswer, sortAnswers).map { errorMessage => s""" | Results do not match for Spark plan: | $outputPlan @@ -238,46 +238,6 @@ object SparkPlanTest { } } - private def compareAnswers( - sparkAnswer: Seq[Row], - expectedAnswer: Seq[Row], - sort: Boolean): Option[String] = { - def prepareAnswer(answer: Seq[Row]): Seq[Row] = { - // Converts data to types that we can do equality comparison using Scala collections. - // For BigDecimal type, the Scala type has a better definition of equality test (similar to - // Java's java.math.BigDecimal.compareTo). - // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for - // equality test. - // This function is copied from Catalyst's QueryTest - val converted: Seq[Row] = answer.map { s => - Row.fromSeq(s.toSeq.map { - case d: java.math.BigDecimal => BigDecimal(d) - case b: Array[Byte] => b.toSeq - case o => o - }) - } - if (sort) { - converted.sortBy(_.toString()) - } else { - converted - } - } - if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { - val errorMessage = - s""" - | == Results == - | ${sideBySide( - s"== Expected Answer - ${expectedAnswer.size} ==" +: - prepareAnswer(expectedAnswer).map(_.toString()), - s"== Actual Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")} - """.stripMargin - Some(errorMessage) - } else { - None - } - } - private def executePlan(outputPlan: SparkPlan, _sqlContext: SQLContext): Seq[Row] = { // A very simple resolver to make writing tests easier. In contrast to the real resolver // this is always case sensitive and does not try to handle scoping or complex type resolution. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala new file mode 100644 index 0000000000000..07209f3779248 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/FilterNodeSuite.scala @@ -0,0 +1,41 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.test.SharedSQLContext + +class FilterNodeSuite extends LocalNodeTest with SharedSQLContext { + + test("basic") { + val condition = (testData.col("key") % 2) === 0 + checkAnswer( + testData, + node => FilterNode(condition.expr, node), + testData.filter(condition).collect() + ) + } + + test("empty") { + val condition = (emptyTestData.col("key") % 2) === 0 + checkAnswer( + emptyTestData, + node => FilterNode(condition.expr, node), + emptyTestData.filter(condition).collect() + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala new file mode 100644 index 0000000000000..523c02f4a6014 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LimitNodeSuite.scala @@ -0,0 +1,39 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.test.SharedSQLContext + +class LimitNodeSuite extends LocalNodeTest with SharedSQLContext { + + test("basic") { + checkAnswer( + testData, + node => LimitNode(10, node), + testData.limit(10).collect() + ) + } + + test("empty") { + checkAnswer( + emptyTestData, + node => LimitNode(10, node), + emptyTestData.limit(10).collect() + ) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala new file mode 100644 index 0000000000000..95f06081bd0a8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/LocalNodeTest.scala @@ -0,0 +1,146 @@ +/* +* 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.execution.local + +import scala.util.control.NonFatal + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.test.SQLTestUtils + +class LocalNodeTest extends SparkFunSuite { + + /** + * Runs the LocalNode and makes sure the answer matches the expected result. + * @param input the input data to be used. + * @param nodeFunction a function which accepts the input LocalNode and uses it to instantiate + * the local physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param sortAnswers if true, the answers will be sorted by their toString representations prior + * to being compared. + */ + protected def checkAnswer( + input: DataFrame, + nodeFunction: LocalNode => LocalNode, + expectedAnswer: Seq[Row], + sortAnswers: Boolean = true): Unit = { + doCheckAnswer( + input :: Nil, + nodes => nodeFunction(nodes.head), + expectedAnswer, + sortAnswers) + } + + /** + * Runs the LocalNode and makes sure the answer matches the expected result. + * @param left the left input data to be used. + * @param right the right input data to be used. + * @param nodeFunction a function which accepts the input LocalNode and uses it to instantiate + * the local physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param sortAnswers if true, the answers will be sorted by their toString representations prior + * to being compared. + */ + protected def checkAnswer2( + left: DataFrame, + right: DataFrame, + nodeFunction: (LocalNode, LocalNode) => LocalNode, + expectedAnswer: Seq[Row], + sortAnswers: Boolean = true): Unit = { + doCheckAnswer( + left :: right :: Nil, + nodes => nodeFunction(nodes(0), nodes(1)), + expectedAnswer, + sortAnswers) + } + + /** + * Runs the `LocalNode`s and makes sure the answer matches the expected result. + * @param input the input data to be used. + * @param nodeFunction a function which accepts a sequence of input `LocalNode`s and uses them to + * instantiate the local physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param sortAnswers if true, the answers will be sorted by their toString representations prior + * to being compared. + */ + protected def doCheckAnswer( + input: Seq[DataFrame], + nodeFunction: Seq[LocalNode] => LocalNode, + expectedAnswer: Seq[Row], + sortAnswers: Boolean = true): Unit = { + LocalNodeTest.checkAnswer( + input.map(dataFrameToSeqScanNode), nodeFunction, expectedAnswer, sortAnswers) match { + case Some(errorMessage) => fail(errorMessage) + case None => + } + } + + protected def dataFrameToSeqScanNode(df: DataFrame): SeqScanNode = { + new SeqScanNode( + df.queryExecution.sparkPlan.output, + df.queryExecution.toRdd.map(_.copy()).collect()) + } + +} + +/** + * Helper methods for writing tests of individual local physical operators. + */ +object LocalNodeTest { + + /** + * Runs the `LocalNode`s and makes sure the answer matches the expected result. + * @param input the input data to be used. + * @param nodeFunction a function which accepts the input `LocalNode`s and uses them to + * instantiate the local physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + * @param sortAnswers if true, the answers will be sorted by their toString representations prior + * to being compared. + */ + def checkAnswer( + input: Seq[SeqScanNode], + nodeFunction: Seq[LocalNode] => LocalNode, + expectedAnswer: Seq[Row], + sortAnswers: Boolean): Option[String] = { + + val outputNode = nodeFunction(input) + + val outputResult: Seq[Row] = try { + outputNode.collect() + } catch { + case NonFatal(e) => + val errorMessage = + s""" + | Exception thrown while executing local plan: + | $outputNode + | == Exception == + | $e + | ${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} + """.stripMargin + return Some(errorMessage) + } + + SQLTestUtils.compareAnswers(outputResult, expectedAnswer, sortAnswers).map { errorMessage => + s""" + | Results do not match for local plan: + | $outputNode + | $errorMessage + """.stripMargin + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala new file mode 100644 index 0000000000000..ffcf092e2c66a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/ProjectNodeSuite.scala @@ -0,0 +1,44 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.test.SharedSQLContext + +class ProjectNodeSuite extends LocalNodeTest with SharedSQLContext { + + test("basic") { + val output = testData.queryExecution.sparkPlan.output + val columns = Seq(output(1), output(0)) + checkAnswer( + testData, + node => ProjectNode(columns, node), + testData.select("value", "key").collect() + ) + } + + test("empty") { + val output = emptyTestData.queryExecution.sparkPlan.output + val columns = Seq(output(1), output(0)) + checkAnswer( + emptyTestData, + node => ProjectNode(columns, node), + emptyTestData.select("value", "key").collect() + ) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala new file mode 100644 index 0000000000000..34670287c3e1d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/local/UnionNodeSuite.scala @@ -0,0 +1,52 @@ +/* +* 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.execution.local + +import org.apache.spark.sql.test.SharedSQLContext + +class UnionNodeSuite extends LocalNodeTest with SharedSQLContext { + + test("basic") { + checkAnswer2( + testData, + testData, + (node1, node2) => UnionNode(Seq(node1, node2)), + testData.unionAll(testData).collect() + ) + } + + test("empty") { + checkAnswer2( + emptyTestData, + emptyTestData, + (node1, node2) => UnionNode(Seq(node1, node2)), + emptyTestData.unionAll(emptyTestData).collect() + ) + } + + test("complicated union") { + val dfs = Seq(testData, emptyTestData, emptyTestData, testData, testData, emptyTestData, + emptyTestData, emptyTestData, testData, emptyTestData) + doCheckAnswer( + dfs, + nodes => UnionNode(nodes), + dfs.reduce(_.unionAll(_)).collect() + ) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 1374a97476ca1..3fc02df954e23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -36,6 +36,13 @@ private[sql] trait SQLTestData { self => // Note: all test data should be lazy because the SQLContext is not set up yet. + protected lazy val emptyTestData: DataFrame = { + val df = _sqlContext.sparkContext.parallelize( + Seq.empty[Int].map(i => TestData(i, i.toString))).toDF() + df.registerTempTable("emptyTestData") + df + } + protected lazy val testData: DataFrame = { val df = _sqlContext.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))).toDF() @@ -240,6 +247,7 @@ private[sql] trait SQLTestData { self => */ def loadTestData(): Unit = { assert(_sqlContext != null, "attempted to initialize test data before SQLContext.") + emptyTestData testData testData2 testData3 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index cdd691e035897..dc08306ad9cb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,8 +27,9 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{DataFrame, SQLContext, SQLImplicits} +import org.apache.spark.sql.{DataFrame, Row, SQLContext, SQLImplicits} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.util.Utils /** @@ -179,3 +180,46 @@ private[sql] trait SQLTestUtils DataFrame(_sqlContext, plan) } } + +private[sql] object SQLTestUtils { + + def compareAnswers( + sparkAnswer: Seq[Row], + expectedAnswer: Seq[Row], + sort: Boolean): Option[String] = { + def prepareAnswer(answer: Seq[Row]): Seq[Row] = { + // Converts data to types that we can do equality comparison using Scala collections. + // For BigDecimal type, the Scala type has a better definition of equality test (similar to + // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. + // This function is copied from Catalyst's QueryTest + val converted: Seq[Row] = answer.map { s => + Row.fromSeq(s.toSeq.map { + case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq + case o => o + }) + } + if (sort) { + converted.sortBy(_.toString()) + } else { + converted + } + } + if (prepareAnswer(expectedAnswer) != prepareAnswer(sparkAnswer)) { + val errorMessage = + s""" + | == Results == + | ${sideBySide( + s"== Expected Answer - ${expectedAnswer.size} ==" +: + prepareAnswer(expectedAnswer).map(_.toString()), + s"== Actual Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer).map(_.toString())).mkString("\n")} + """.stripMargin + Some(errorMessage) + } else { + None + } + } +} From 905fbe498bdd29116468628e6a2a553c1fd57165 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 29 Aug 2015 23:26:23 -0700 Subject: [PATCH 153/232] [SPARK-10348] [MLLIB] updates ml-guide * replace `ML Dataset` by `DataFrame` to unify the abstraction * ML algorithms -> pipeline components to describe the main concept * remove Scala API doc links from the main guide * `Section Title` -> `Section tile` to be consistent with other section titles in MLlib guide * modified lines break at 100 chars or periods jkbradley feynmanliang Author: Xiangrui Meng Closes #8517 from mengxr/SPARK-10348. --- docs/ml-guide.md | 118 +++++++++++++++++++++++++++----------------- docs/mllib-guide.md | 12 ++--- 2 files changed, 78 insertions(+), 52 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index a92a285f3af85..4ba07542bfb40 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -24,61 +24,74 @@ title: Spark ML Programming Guide The `spark.ml` package aims to provide a uniform set of high-level APIs built on top of [DataFrames](sql-programming-guide.html#dataframes) that help users create and tune practical machine learning pipelines. -See the [Algorithm Guides section](#algorithm-guides) below for guides on sub-packages of +See the [algorithm guides](#algorithm-guides) section below for guides on sub-packages of `spark.ml`, including feature transformers unique to the Pipelines API, ensembles, and more. -**Table of Contents** +**Table of contents** * This will become a table of contents (this text will be scraped). {:toc} -# Main Concepts +# Main concepts -Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple algorithms into a single pipeline, or workflow. This section covers the key concepts introduced by the Spark ML API. +Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple +algorithms into a single pipeline, or workflow. +This section covers the key concepts introduced by the Spark ML API, where the pipeline concept is +mostly inspired by the [scikit-learn](http://scikit-learn.org/) project. -* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL as a dataset which can hold a variety of data types. -E.g., a dataset could have different columns storing text, feature vectors, true labels, and predictions. +* **[`DataFrame`](ml-guide.html#dataframe)**: Spark ML uses `DataFrame` from Spark SQL as an ML + dataset, which can hold a variety of data types. + E.g., a `DataFrame` could have different columns storing text, feature vectors, true labels, and predictions. * **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `DataFrame` into another `DataFrame`. -E.g., an ML model is a `Transformer` which transforms an RDD with features into an RDD with predictions. +E.g., an ML model is a `Transformer` which transforms `DataFrame` with features into a `DataFrame` with predictions. * **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `DataFrame` to produce a `Transformer`. -E.g., a learning algorithm is an `Estimator` which trains on a dataset and produces a model. +E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and produces a model. * **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. -* **[`Param`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. +* **[`Parameter`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. -## ML Dataset +## DataFrame Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. -Spark ML adopts the [`DataFrame`](api/scala/index.html#org.apache.spark.sql.DataFrame) from Spark SQL in order to support a variety of data types under a unified Dataset concept. +Spark ML adopts the `DataFrame` from Spark SQL in order to support a variety of data types. `DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. -In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types. +In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." -## ML Algorithms +## Pipeline components ### Transformers -A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `DataFrame` into another, generally by appending one or more columns. +A `Transformer` is an abstraction that includes feature transformers and learned models. +Technically, a `Transformer` implements a method `transform()`, which converts one `DataFrame` into +another, generally by appending one or more columns. For example: -* A feature transformer might take a dataset, read a column (e.g., text), convert it into a new column (e.g., feature vectors), append the new column to the dataset, and output the updated dataset. -* A learning model might take a dataset, read the column containing feature vectors, predict the label for each feature vector, append the labels as a new column, and output the updated dataset. +* A feature transformer might take a `DataFrame`, read a column (e.g., text), map it into a new + column (e.g., feature vectors), and output a new `DataFrame` with the mapped column appended. +* A learning model might take a `DataFrame`, read the column containing feature vectors, predict the + label for each feature vector, and output a new `DataFrame` with predicted labels appended as a + column. ### Estimators -An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `DataFrame` and produces a `Transformer`. -For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling `fit()` trains a `LogisticRegressionModel`, which is a `Transformer`. +An `Estimator` abstracts the concept of a learning algorithm or any algorithm that fits or trains on +data. +Technically, an `Estimator` implements a method `fit()`, which accepts a `DataFrame` and produces a +`Model`, which is a `Transformer`. +For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling +`fit()` trains a `LogisticRegressionModel`, which is a `Model` and hence a `Transformer`. -### Properties of ML Algorithms +### Properties of pipeline components -`Transformer`s and `Estimator`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. +`Transformer.transform()`s and `Estimator.fit()`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). @@ -91,15 +104,16 @@ E.g., a simple text document processing workflow might include several stages: * Convert each document's words into a numerical feature vector. * Learn a prediction model using the feature vectors and labels. -Spark ML represents such a workflow as a [`Pipeline`](api/scala/index.html#org.apache.spark.ml.Pipeline), -which consists of a sequence of [`PipelineStage`s](api/scala/index.html#org.apache.spark.ml.PipelineStage) (`Transformer`s and `Estimator`s) to be run in a specific order. We will use this simple workflow as a running example in this section. +Spark ML represents such a workflow as a `Pipeline`, which consists of a sequence of +`PipelineStage`s (`Transformer`s and `Estimator`s) to be run in a specific order. +We will use this simple workflow as a running example in this section. -### How It Works +### How it works A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. -These stages are run in order, and the input dataset is modified as it passes through each stage. -For `Transformer` stages, the `transform()` method is called on the dataset. -For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the dataset. +These stages are run in order, and the input `DataFrame` is transformed as it passes through each stage. +For `Transformer` stages, the `transform()` method is called on the `DataFrame`. +For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the `DataFrame`. We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. @@ -115,14 +129,17 @@ We illustrate this for the simple text document workflow. The figure below is f Above, the top row represents a `Pipeline` with three stages. The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). The bottom row represents data flowing through the pipeline, where cylinders indicate `DataFrame`s. -The `Pipeline.fit()` method is called on the original dataset which has raw text documents and labels. -The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words into the dataset. -The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the dataset. +The `Pipeline.fit()` method is called on the original `DataFrame`, which has raw text documents and labels. +The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words to the `DataFrame`. +The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the `DataFrame`. Now, since `LogisticRegression` is an `Estimator`, the `Pipeline` first calls `LogisticRegression.fit()` to produce a `LogisticRegressionModel`. -If the `Pipeline` had more stages, it would call the `LogisticRegressionModel`'s `transform()` method on the dataset before passing the dataset to the next stage. +If the `Pipeline` had more stages, it would call the `LogisticRegressionModel`'s `transform()` +method on the `DataFrame` before passing the `DataFrame` to the next stage. A `Pipeline` is an `Estimator`. -Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel` which is a `Transformer`. This `PipelineModel` is used at *test time*; the figure below illustrates this usage. +Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel`, which is a +`Transformer`. +This `PipelineModel` is used at *test time*; the figure below illustrates this usage.

In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. -When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed through the `Pipeline` in order. +When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed +through the fitted pipeline in order. Each stage's `transform()` method updates the dataset and passes it to the next stage. `Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. @@ -143,40 +161,48 @@ Each stage's `transform()` method updates the dataset and passes it to the next *DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. -*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `DataFrame`. +*Runtime checking*: Since `Pipeline`s can operate on `DataFrame`s with varied types, they cannot use +compile-time type checking. +`Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. +This type checking is done using the `DataFrame` *schema*, a description of the data types of columns in the `DataFrame`. ## Parameters Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. -A [`Param`](api/scala/index.html#org.apache.spark.ml.param.Param) is a named parameter with self-contained documentation. -A [`ParamMap`](api/scala/index.html#org.apache.spark.ml.param.ParamMap) is a set of (parameter, value) pairs. +A `Param` is a named parameter with self-contained documentation. +A `ParamMap` is a set of (parameter, value) pairs. There are two main ways to pass parameters to an algorithm: -1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. This API resembles the API used in MLlib. +1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could + call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. + This API resembles the API used in `spark.mllib` package. 2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. Parameters belong to specific instances of `Estimator`s and `Transformer`s. For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. -# Algorithm Guides +# Algorithm guides There are now several algorithms in the Pipelines API which are not in the `spark.mllib` API, so we link to documentation for them here. These algorithms are mostly feature transformers, which fit naturally into the `Transformer` abstraction in Pipelines, and ensembles, which fit naturally into the `Estimator` abstraction in the Pipelines. -**Pipelines API Algorithm Guides** - -* [Feature Extraction, Transformation, and Selection](ml-features.html) -* [Decision Trees for Classification and Regression](ml-decision-tree.html) +* [Feature extraction, transformation, and selection](ml-features.html) +* [Decision Trees for classification and regression](ml-decision-tree.html) * [Ensembles](ml-ensembles.html) * [Linear methods with elastic net regularization](ml-linear-methods.html) * [Multilayer perceptron classifier](ml-ann.html) -# Code Examples +# Code examples This section gives code examples illustrating the functionality discussed above. -There is not yet documentation for specific algorithms in Spark ML. For more info, please refer to the [API Documentation](api/scala/index.html#org.apache.spark.ml.package). Spark ML algorithms are currently wrappers for MLlib algorithms, and the [MLlib programming guide](mllib-guide.html) has details on specific algorithms. +For more info, please refer to the API documentation +([Scala](api/scala/index.html#org.apache.spark.ml.package), +[Java](api/java/org/apache/spark/ml/package-summary.html), +and [Python](api/python/pyspark.ml.html)). +Some Spark ML algorithms are wrappers for `spark.mllib` algorithms, and the +[MLlib programming guide](mllib-guide.html) has details on specific algorithms. ## Example: Estimator, Transformer, and Param @@ -627,7 +653,7 @@ sc.stop() -## Example: Model Selection via Cross-Validation +## Example: model selection via cross-validation An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. `Pipeline`s facilitate model selection by making it easy to tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. @@ -873,11 +899,11 @@ jsc.stop(); -## Example: Model Selection via Train Validation Split +## Example: model selection via train validation split In addition to `CrossValidator` Spark also offers `TrainValidationSplit` for hyper-parameter tuning. `TrainValidationSplit` only evaluates each combination of parameters once as opposed to k times in case of `CrossValidator`. It is therefore less expensive, - but will not produce as reliable results when the training dataset is not sufficiently large.. + but will not produce as reliable results when the training dataset is not sufficiently large. `TrainValidationSplit` takes an `Estimator`, a set of `ParamMap`s provided in the `estimatorParamMaps` parameter, and an `Evaluator`. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 876dcfd40ed7b..257f7cc7603fa 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -14,9 +14,9 @@ primitives and higher-level pipeline APIs. It divides into two packages: * [`spark.mllib`](mllib-guide.html#mllib-types-algorithms-and-utilities) contains the original API - built on top of RDDs. + built on top of [RDDs](programming-guide.html#resilient-distributed-datasets-rdds). * [`spark.ml`](mllib-guide.html#sparkml-high-level-apis-for-ml-pipelines) provides higher-level API - built on top of DataFrames for constructing ML pipelines. + built on top of [DataFrames](sql-programming-guide.html#dataframes) for constructing ML pipelines. Using `spark.ml` is recommended because with DataFrames the API is more versatile and flexible. But we will keep supporting `spark.mllib` along with the development of `spark.ml`. @@ -57,19 +57,19 @@ We list major functionality from both below, with links to detailed guides. * [FP-growth](mllib-frequent-pattern-mining.html#fp-growth) * [association rules](mllib-frequent-pattern-mining.html#association-rules) * [PrefixSpan](mllib-frequent-pattern-mining.html#prefix-span) -* [Evaluation Metrics](mllib-evaluation-metrics.html) +* [Evaluation metrics](mllib-evaluation-metrics.html) +* [PMML model export](mllib-pmml-model-export.html) * [Optimization (developer)](mllib-optimization.html) * [stochastic gradient descent](mllib-optimization.html#stochastic-gradient-descent-sgd) * [limited-memory BFGS (L-BFGS)](mllib-optimization.html#limited-memory-bfgs-l-bfgs) -* [PMML model export](mllib-pmml-model-export.html) # spark.ml: high-level APIs for ML pipelines **[spark.ml programming guide](ml-guide.html)** provides an overview of the Pipelines API and major concepts. It also contains sections on using algorithms within the Pipelines API, for example: -* [Feature Extraction, Transformation, and Selection](ml-features.html) -* [Decision Trees for Classification and Regression](ml-decision-tree.html) +* [Feature extraction, transformation, and selection](ml-features.html) +* [Decision trees for classification and regression](ml-decision-tree.html) * [Ensembles](ml-ensembles.html) * [Linear methods with elastic net regularization](ml-linear-methods.html) * [Multilayer perceptron classifier](ml-ann.html) From ca69fc8efda8a3e5442ffa16692a2b1eb86b7673 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 29 Aug 2015 23:57:09 -0700 Subject: [PATCH 154/232] [SPARK-10331] [MLLIB] Update example code in ml-guide * The example code was added in 1.2, before `createDataFrame`. This PR switches to `createDataFrame`. Java code still uses JavaBean. * assume `sqlContext` is available * fix some minor issues from previous code review jkbradley srowen feynmanliang Author: Xiangrui Meng Closes #8518 from mengxr/SPARK-10331. --- docs/ml-guide.md | 362 +++++++++++++++++++---------------------------- 1 file changed, 147 insertions(+), 215 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 4ba07542bfb40..78c93a95c7807 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -212,26 +212,18 @@ This example covers the concepts of `Estimator`, `Transformer`, and `Param`.

{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.param.ParamMap import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.Row -val conf = new SparkConf().setAppName("SimpleParamsExample") -val sc = new SparkContext(conf) -val sqlContext = new SQLContext(sc) -import sqlContext.implicits._ - -// Prepare training data. -// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes -// into DataFrames, where it uses the case class metadata to infer the schema. -val training = sc.parallelize(Seq( - LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), - LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), - LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) +// Prepare training data from a list of (label, features) tuples. +val training = sqlContext.createDataFrame(Seq( + (1.0, Vectors.dense(0.0, 1.1, 0.1)), + (0.0, Vectors.dense(2.0, 1.0, -1.0)), + (0.0, Vectors.dense(2.0, 1.3, 1.0)), + (1.0, Vectors.dense(0.0, 1.2, -0.5)) +)).toDF("label", "features") // Create a LogisticRegression instance. This instance is an Estimator. val lr = new LogisticRegression() @@ -243,7 +235,7 @@ lr.setMaxIter(10) .setRegParam(0.01) // Learn a LogisticRegression model. This uses the parameters stored in lr. -val model1 = lr.fit(training.toDF) +val model1 = lr.fit(training) // Since model1 is a Model (i.e., a Transformer produced by an Estimator), // we can view the parameters it used during fit(). // This prints the parameter (name: value) pairs, where names are unique IDs for this @@ -253,8 +245,8 @@ println("Model 1 was fit using parameters: " + model1.parent.extractParamMap) // We may alternatively specify parameters using a ParamMap, // which supports several methods for specifying parameters. val paramMap = ParamMap(lr.maxIter -> 20) -paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. -paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. + .put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + .put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. // One can also combine ParamMaps. val paramMap2 = ParamMap(lr.probabilityCol -> "myProbability") // Change output column name @@ -262,27 +254,27 @@ val paramMapCombined = paramMap ++ paramMap2 // Now learn a new model using the paramMapCombined parameters. // paramMapCombined overrides all parameters set earlier via lr.set* methods. -val model2 = lr.fit(training.toDF, paramMapCombined) +val model2 = lr.fit(training, paramMapCombined) println("Model 2 was fit using parameters: " + model2.parent.extractParamMap) // Prepare test data. -val test = sc.parallelize(Seq( - LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) +val test = sqlContext.createDataFrame(Seq( + (1.0, Vectors.dense(-1.0, 1.5, 1.3)), + (0.0, Vectors.dense(3.0, 2.0, -0.1)), + (1.0, Vectors.dense(0.0, 2.2, -1.5)) +)).toDF("label", "features") // Make predictions on test data using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. -model2.transform(test.toDF) +model2.transform(test) .select("features", "label", "myProbability", "prediction") .collect() .foreach { case Row(features: Vector, label: Double, prob: Vector, prediction: Double) => println(s"($features, $label) -> prob=$prob, prediction=$prediction") } -sc.stop() {% endhighlight %}
@@ -291,30 +283,23 @@ sc.stop() import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.classification.LogisticRegressionModel; import org.apache.spark.ml.param.ParamMap; import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.DataFrame; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.Row; -SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); -JavaSparkContext jsc = new JavaSparkContext(conf); -SQLContext jsql = new SQLContext(jsc); - // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans // into DataFrames, where it uses the bean metadata to infer the schema. -List localTraining = Arrays.asList( +DataFrame training = sqlContext.createDataFrame(Arrays.asList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), - new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); -DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledPoint.class); + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)) +), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -334,14 +319,14 @@ LogisticRegressionModel model1 = lr.fit(training); System.out.println("Model 1 was fit using parameters: " + model1.parent().extractParamMap()); // We may alternatively specify parameters using a ParamMap. -ParamMap paramMap = new ParamMap(); -paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. -paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. -paramMap.put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. +ParamMap paramMap = new ParamMap() + .put(lr.maxIter().w(20)) // Specify 1 Param. + .put(lr.maxIter(), 30) // This overwrites the original maxIter. + .put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. // One can also combine ParamMaps. -ParamMap paramMap2 = new ParamMap(); -paramMap2.put(lr.probabilityCol().w("myProbability")); // Change output column name +ParamMap paramMap2 = new ParamMap() + .put(lr.probabilityCol().w("myProbability")); // Change output column name ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); // Now learn a new model using the paramMapCombined parameters. @@ -350,11 +335,11 @@ LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); System.out.println("Model 2 was fit using parameters: " + model2.parent().extractParamMap()); // Prepare test documents. -List localTest = Arrays.asList( - new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), - new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), - new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); -DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), LabeledPoint.class); +DataFrame test = sqlContext.createDataFrame(Arrays.asList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)) +), LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. @@ -366,28 +351,21 @@ for (Row r: results.select("features", "label", "myProbability", "prediction").c + ", prediction=" + r.get(3)); } -jsc.stop(); {% endhighlight %}
{% highlight python %} -from pyspark import SparkContext -from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.linalg import Vectors from pyspark.ml.classification import LogisticRegression from pyspark.ml.param import Param, Params -from pyspark.sql import Row, SQLContext -sc = SparkContext(appName="SimpleParamsExample") -sqlContext = SQLContext(sc) - -# Prepare training data. -# We use LabeledPoint. -# Spark SQL can convert RDDs of LabeledPoints into DataFrames. -training = sc.parallelize([LabeledPoint(1.0, [0.0, 1.1, 0.1]), - LabeledPoint(0.0, [2.0, 1.0, -1.0]), - LabeledPoint(0.0, [2.0, 1.3, 1.0]), - LabeledPoint(1.0, [0.0, 1.2, -0.5])]) +# Prepare training data from a list of (label, features) tuples. +training = sqlContext.createDataFrame([ + (1.0, Vectors.dense([0.0, 1.1, 0.1])), + (0.0, Vectors.dense([2.0, 1.0, -1.0])), + (0.0, Vectors.dense([2.0, 1.3, 1.0])), + (1.0, Vectors.dense([0.0, 1.2, -0.5]))], ["label", "features"]) # Create a LogisticRegression instance. This instance is an Estimator. lr = LogisticRegression(maxIter=10, regParam=0.01) @@ -395,7 +373,7 @@ lr = LogisticRegression(maxIter=10, regParam=0.01) print "LogisticRegression parameters:\n" + lr.explainParams() + "\n" # Learn a LogisticRegression model. This uses the parameters stored in lr. -model1 = lr.fit(training.toDF()) +model1 = lr.fit(training) # Since model1 is a Model (i.e., a transformer produced by an Estimator), # we can view the parameters it used during fit(). @@ -416,25 +394,25 @@ paramMapCombined.update(paramMap2) # Now learn a new model using the paramMapCombined parameters. # paramMapCombined overrides all parameters set earlier via lr.set* methods. -model2 = lr.fit(training.toDF(), paramMapCombined) +model2 = lr.fit(training, paramMapCombined) print "Model 2 was fit using parameters: " print model2.extractParamMap() # Prepare test data -test = sc.parallelize([LabeledPoint(1.0, [-1.0, 1.5, 1.3]), - LabeledPoint(0.0, [ 3.0, 2.0, -0.1]), - LabeledPoint(1.0, [ 0.0, 2.2, -1.5])]) +test = sqlContext.createDataFrame([ + (1.0, Vectors.dense([-1.0, 1.5, 1.3])), + (0.0, Vectors.dense([3.0, 2.0, -0.1])), + (1.0, Vectors.dense([0.0, 2.2, -1.5]))], ["label", "features"]) # Make predictions on test data using the Transformer.transform() method. # LogisticRegression.transform will only use the 'features' column. # Note that model2.transform() outputs a "myProbability" column instead of the usual # 'probability' column since we renamed the lr.probabilityCol parameter previously. -prediction = model2.transform(test.toDF()) +prediction = model2.transform(test) selected = prediction.select("features", "label", "myProbability", "prediction") for row in selected.collect(): print row -sc.stop() {% endhighlight %}
@@ -448,30 +426,19 @@ This example follows the simple text document `Pipeline` illustrated in the figu
{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} - -// Labeled and unlabeled instance types. -// Spark SQL can infer schema from case classes. -case class LabeledDocument(id: Long, text: String, label: Double) -case class Document(id: Long, text: String) +import org.apache.spark.sql.Row -// Set up contexts. Import implicit conversions to DataFrame from sqlContext. -val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") -val sc = new SparkContext(conf) -val sqlContext = new SQLContext(sc) -import sqlContext.implicits._ - -// Prepare training documents, which are labeled. -val training = sc.parallelize(Seq( - LabeledDocument(0L, "a b c d e spark", 1.0), - LabeledDocument(1L, "b d", 0.0), - LabeledDocument(2L, "spark f g h", 1.0), - LabeledDocument(3L, "hadoop mapreduce", 0.0))) +// Prepare training documents from a list of (id, text, label) tuples. +val training = sqlContext.createDataFrame(Seq( + (0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0) +)).toDF("id", "text", "label") // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. val tokenizer = new Tokenizer() @@ -488,14 +455,15 @@ val pipeline = new Pipeline() .setStages(Array(tokenizer, hashingTF, lr)) // Fit the pipeline to training documents. -val model = pipeline.fit(training.toDF) +val model = pipeline.fit(training) -// Prepare test documents, which are unlabeled. -val test = sc.parallelize(Seq( - Document(4L, "spark i j k"), - Document(5L, "l m n"), - Document(6L, "mapreduce spark"), - Document(7L, "apache hadoop"))) +// Prepare test documents, which are unlabeled (id, text) tuples. +val test = sqlContext.createDataFrame(Seq( + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") +)).toDF("id", "text") // Make predictions on test documents. model.transform(test.toDF) @@ -505,7 +473,6 @@ model.transform(test.toDF) println(s"($id, $text) --> prob=$prob, prediction=$prediction") } -sc.stop() {% endhighlight %}
@@ -514,8 +481,6 @@ sc.stop() import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; import org.apache.spark.ml.PipelineStage; @@ -524,7 +489,6 @@ import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.Tokenizer; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; // Labeled and unlabeled instance types. // Spark SQL can infer schema from Java Beans. @@ -556,18 +520,13 @@ public class LabeledDocument extends Document implements Serializable { public void setLabel(double label) { this.label = label; } } -// Set up contexts. -SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); -JavaSparkContext jsc = new JavaSparkContext(conf); -SQLContext jsql = new SQLContext(jsc); - // Prepare training documents, which are labeled. -List localTraining = Arrays.asList( +DataFrame training = sqlContext.createDataFrame(Arrays.asList( new LabeledDocument(0L, "a b c d e spark", 1.0), new LabeledDocument(1L, "b d", 0.0), new LabeledDocument(2L, "spark f g h", 1.0), - new LabeledDocument(3L, "hadoop mapreduce", 0.0)); -DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); + new LabeledDocument(3L, "hadoop mapreduce", 0.0) +), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -587,12 +546,12 @@ Pipeline pipeline = new Pipeline() PipelineModel model = pipeline.fit(training); // Prepare test documents, which are unlabeled. -List localTest = Arrays.asList( +DataFrame test = sqlContext.createDataFrame(Arrays.asList( new Document(4L, "spark i j k"), new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), - new Document(7L, "apache hadoop")); -DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); + new Document(7L, "apache hadoop") +), Document.class); // Make predictions on test documents. DataFrame predictions = model.transform(test); @@ -601,28 +560,23 @@ for (Row r: predictions.select("id", "text", "probability", "prediction").collec + ", prediction=" + r.get(3)); } -jsc.stop(); {% endhighlight %}
{% highlight python %} -from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression from pyspark.ml.feature import HashingTF, Tokenizer -from pyspark.sql import Row, SQLContext - -sc = SparkContext(appName="SimpleTextClassificationPipeline") -sqlContext = SQLContext(sc) +from pyspark.sql import Row -# Prepare training documents, which are labeled. +# Prepare training documents from a list of (id, text, label) tuples. LabeledDocument = Row("id", "text", "label") -training = sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ - .map(lambda x: LabeledDocument(*x)).toDF() +training = sqlContext.createDataFrame([ + (0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0)], ["id", "text", "label"]) # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. tokenizer = Tokenizer(inputCol="text", outputCol="words") @@ -633,13 +587,12 @@ pipeline = Pipeline(stages=[tokenizer, hashingTF, lr]) # Fit the pipeline to training documents. model = pipeline.fit(training) -# Prepare test documents, which are unlabeled. -Document = Row("id", "text") -test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ - .map(lambda x: Document(*x)).toDF() +# Prepare test documents, which are unlabeled (id, text) tuples. +test = sqlContext.createDataFrame([ + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop")], ["id", "text"]) # Make predictions on test documents and print columns of interest. prediction = model.transform(test) @@ -647,7 +600,6 @@ selected = prediction.select("id", "text", "prediction") for row in selected.collect(): print(row) -sc.stop() {% endhighlight %}
@@ -664,8 +616,8 @@ Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/ The `Evaluator` can be a [`RegressionEvaluator`](api/scala/index.html#org.apache.spark.ml.RegressionEvaluator) for regression problems, a [`BinaryClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.BinaryClassificationEvaluator) -for binary data or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.MultiClassClassificationEvaluator) -for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the setMetric +for binary data, or a [`MultiClassClassificationEvaluator`](api/scala/index.html#org.apache.spark.ml.MultiClassClassificationEvaluator) +for multiclass problems. The default metric used to choose the best `ParamMap` can be overriden by the `setMetric` method in each of these evaluators. The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. @@ -684,39 +636,29 @@ However, it is also a well-established method for choosing parameters which is m
{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator import org.apache.spark.ml.feature.{HashingTF, Tokenizer} import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.sql.{Row, SQLContext} - -// Labeled and unlabeled instance types. -// Spark SQL can infer schema from case classes. -case class LabeledDocument(id: Long, text: String, label: Double) -case class Document(id: Long, text: String) - -val conf = new SparkConf().setAppName("CrossValidatorExample") -val sc = new SparkContext(conf) -val sqlContext = new SQLContext(sc) -import sqlContext.implicits._ - -// Prepare training documents, which are labeled. -val training = sc.parallelize(Seq( - LabeledDocument(0L, "a b c d e spark", 1.0), - LabeledDocument(1L, "b d", 0.0), - LabeledDocument(2L, "spark f g h", 1.0), - LabeledDocument(3L, "hadoop mapreduce", 0.0), - LabeledDocument(4L, "b spark who", 1.0), - LabeledDocument(5L, "g d a y", 0.0), - LabeledDocument(6L, "spark fly", 1.0), - LabeledDocument(7L, "was mapreduce", 0.0), - LabeledDocument(8L, "e spark program", 1.0), - LabeledDocument(9L, "a e c l", 0.0), - LabeledDocument(10L, "spark compile", 1.0), - LabeledDocument(11L, "hadoop software", 0.0))) +import org.apache.spark.sql.Row + +// Prepare training data from a list of (id, text, label) tuples. +val training = sqlContext.createDataFrame(Seq( + (0L, "a b c d e spark", 1.0), + (1L, "b d", 0.0), + (2L, "spark f g h", 1.0), + (3L, "hadoop mapreduce", 0.0), + (4L, "b spark who", 1.0), + (5L, "g d a y", 0.0), + (6L, "spark fly", 1.0), + (7L, "was mapreduce", 0.0), + (8L, "e spark program", 1.0), + (9L, "a e c l", 0.0), + (10L, "spark compile", 1.0), + (11L, "hadoop software", 0.0) +)).toDF("id", "text", "label") // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. val tokenizer = new Tokenizer() @@ -730,15 +672,6 @@ val lr = new LogisticRegression() val pipeline = new Pipeline() .setStages(Array(tokenizer, hashingTF, lr)) -// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. -// This will allow us to jointly choose parameters for all Pipeline stages. -// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -// Note that the evaluator here is a BinaryClassificationEvaluator and the default metric -// used is areaUnderROC. -val crossval = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator) - // We use a ParamGridBuilder to construct a grid of parameters to search over. // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. @@ -746,28 +679,37 @@ val paramGrid = new ParamGridBuilder() .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) .addGrid(lr.regParam, Array(0.1, 0.01)) .build() -crossval.setEstimatorParamMaps(paramGrid) -crossval.setNumFolds(2) // Use 3+ in practice + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +// Note that the evaluator here is a BinaryClassificationEvaluator and its default metric +// is areaUnderROC. +val cv = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) + .setEstimatorParamMaps(paramGrid) + .setNumFolds(2) // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. -val cvModel = crossval.fit(training.toDF) +val cvModel = cv.fit(training) -// Prepare test documents, which are unlabeled. -val test = sc.parallelize(Seq( - Document(4L, "spark i j k"), - Document(5L, "l m n"), - Document(6L, "mapreduce spark"), - Document(7L, "apache hadoop"))) +// Prepare test documents, which are unlabeled (id, text) tuples. +val test = sqlContext.createDataFrame(Seq( + (4L, "spark i j k"), + (5L, "l m n"), + (6L, "mapreduce spark"), + (7L, "apache hadoop") +)).toDF("id", "text") // Make predictions on test documents. cvModel uses the best model found (lrModel). -cvModel.transform(test.toDF) +cvModel.transform(test) .select("id", "text", "probability", "prediction") .collect() .foreach { case Row(id: Long, text: String, prob: Vector, prediction: Double) => - println(s"($id, $text) --> prob=$prob, prediction=$prediction") -} + println(s"($id, $text) --> prob=$prob, prediction=$prediction") + } -sc.stop() {% endhighlight %}
@@ -776,8 +718,6 @@ sc.stop() import java.util.Arrays; import java.util.List; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineStage; import org.apache.spark.ml.classification.LogisticRegression; @@ -790,7 +730,6 @@ import org.apache.spark.ml.tuning.CrossValidatorModel; import org.apache.spark.ml.tuning.ParamGridBuilder; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; // Labeled and unlabeled instance types. // Spark SQL can infer schema from Java Beans. @@ -822,12 +761,9 @@ public class LabeledDocument extends Document implements Serializable { public void setLabel(double label) { this.label = label; } } -SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); -JavaSparkContext jsc = new JavaSparkContext(conf); -SQLContext jsql = new SQLContext(jsc); // Prepare training documents, which are labeled. -List localTraining = Arrays.asList( +DataFrame training = sqlContext.createDataFrame(Arrays.asList( new LabeledDocument(0L, "a b c d e spark", 1.0), new LabeledDocument(1L, "b d", 0.0), new LabeledDocument(2L, "spark f g h", 1.0), @@ -839,8 +775,8 @@ List localTraining = Arrays.asList( new LabeledDocument(8L, "e spark program", 1.0), new LabeledDocument(9L, "a e c l", 0.0), new LabeledDocument(10L, "spark compile", 1.0), - new LabeledDocument(11L, "hadoop software", 0.0)); -DataFrame training = jsql.createDataFrame(jsc.parallelize(localTraining), LabeledDocument.class); + new LabeledDocument(11L, "hadoop software", 0.0) +), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -856,15 +792,6 @@ LogisticRegression lr = new LogisticRegression() Pipeline pipeline = new Pipeline() .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); -// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. -// This will allow us to jointly choose parameters for all Pipeline stages. -// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. -// Note that the evaluator here is a BinaryClassificationEvaluator and the default metric -// used is areaUnderROC. -CrossValidator crossval = new CrossValidator() - .setEstimator(pipeline) - .setEvaluator(new BinaryClassificationEvaluator()); - // We use a ParamGridBuilder to construct a grid of parameters to search over. // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. @@ -872,19 +799,28 @@ ParamMap[] paramGrid = new ParamGridBuilder() .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) .addGrid(lr.regParam(), new double[]{0.1, 0.01}) .build(); -crossval.setEstimatorParamMaps(paramGrid); -crossval.setNumFolds(2); // Use 3+ in practice + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +// Note that the evaluator here is a BinaryClassificationEvaluator and its default metric +// is areaUnderROC. +CrossValidator cv = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()) + .setEstimatorParamMaps(paramGrid) + .setNumFolds(2); // Use 3+ in practice // Run cross-validation, and choose the best set of parameters. -CrossValidatorModel cvModel = crossval.fit(training); +CrossValidatorModel cvModel = cv.fit(training); // Prepare test documents, which are unlabeled. -List localTest = Arrays.asList( +DataFrame test = sqlContext.createDataFrame(Arrays.asList( new Document(4L, "spark i j k"), new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), - new Document(7L, "apache hadoop")); -DataFrame test = jsql.createDataFrame(jsc.parallelize(localTest), Document.class); + new Document(7L, "apache hadoop") +), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). DataFrame predictions = cvModel.transform(test); @@ -893,7 +829,6 @@ for (Row r: predictions.select("id", "text", "probability", "prediction").collec + ", prediction=" + r.get(3)); } -jsc.stop(); {% endhighlight %} @@ -935,7 +870,7 @@ val lr = new LinearRegression() // the evaluator. val paramGrid = new ParamGridBuilder() .addGrid(lr.regParam, Array(0.1, 0.01)) - .addGrid(lr.fitIntercept, Array(true, false)) + .addGrid(lr.fitIntercept) .addGrid(lr.elasticNetParam, Array(0.0, 0.5, 1.0)) .build() @@ -945,9 +880,8 @@ val trainValidationSplit = new TrainValidationSplit() .setEstimator(lr) .setEvaluator(new RegressionEvaluator) .setEstimatorParamMaps(paramGrid) - -// 80% of the data will be used for training and the remaining 20% for validation. -trainValidationSplit.setTrainRatio(0.8) + // 80% of the data will be used for training and the remaining 20% for validation. + .setTrainRatio(0.8) // Run train validation split, and choose the best set of parameters. val model = trainValidationSplit.fit(training) @@ -972,12 +906,12 @@ import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.rdd.RDD; import org.apache.spark.sql.DataFrame; -DataFrame data = jsql.createDataFrame( +DataFrame data = sqlContext.createDataFrame( MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt"), LabeledPoint.class); // Prepare training and test data. -DataFrame[] splits = data.randomSplit(new double [] {0.9, 0.1}, 12345); +DataFrame[] splits = data.randomSplit(new double[] {0.9, 0.1}, 12345); DataFrame training = splits[0]; DataFrame test = splits[1]; @@ -997,10 +931,8 @@ ParamMap[] paramGrid = new ParamGridBuilder() TrainValidationSplit trainValidationSplit = new TrainValidationSplit() .setEstimator(lr) .setEvaluator(new RegressionEvaluator()) - .setEstimatorParamMaps(paramGrid); - -// 80% of the data will be used for training and the remaining 20% for validation. -trainValidationSplit.setTrainRatio(0.8); + .setEstimatorParamMaps(paramGrid) + .setTrainRatio(0.8); // 80% for training and the remaining 20% for validation // Run train validation split, and choose the best set of parameters. TrainValidationSplitModel model = trainValidationSplit.fit(training); From 1bfd9347822df65e76201c4c471a26488d722319 Mon Sep 17 00:00:00 2001 From: ihainan Date: Sun, 30 Aug 2015 08:26:14 +0100 Subject: [PATCH 155/232] [SPARK-10184] [CORE] Optimization for bounds determination in RangePartitioner JIRA Issue: https://issues.apache.org/jira/browse/SPARK-10184 Change `cumWeight > target` to `cumWeight >= target` in `RangePartitioner.determineBounds` method to make the output partitions more balanced. Author: ihainan Closes #8397 from ihainan/opt_for_rangepartitioner. --- core/src/main/scala/org/apache/spark/Partitioner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 4b9d59975bdc2..29e581bb57cbc 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -291,7 +291,7 @@ private[spark] object RangePartitioner { while ((i < numCandidates) && (j < partitions - 1)) { val (key, weight) = ordered(i) cumWeight += weight - if (cumWeight > target) { + if (cumWeight >= target) { // Skip duplicate values. if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) { bounds += key From 8d2ab75d3b71b632f2394f2453af32f417cb45e5 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Sun, 30 Aug 2015 12:21:15 -0700 Subject: [PATCH 156/232] [SPARK-10353] [MLLIB] BLAS gemm not scaling when beta = 0.0 for some subset of matrix multiplications mengxr jkbradley rxin It would be great if this fix made it into RC3! Author: Burak Yavuz Closes #8525 from brkyvz/blas-scaling. --- .../org/apache/spark/mllib/linalg/BLAS.scala | 26 +++++++------------ .../apache/spark/mllib/linalg/BLASSuite.scala | 5 ++++ 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index bbbcc8436b7c2..ab475af264dd3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -305,6 +305,8 @@ private[spark] object BLAS extends Serializable with Logging { "The matrix C cannot be the product of a transpose() call. C.isTransposed must be false.") if (alpha == 0.0 && beta == 1.0) { logDebug("gemm: alpha is equal to 0 and beta is equal to 1. Returning C.") + } else if (alpha == 0.0) { + f2jBLAS.dscal(C.values.length, beta, C.values, 1) } else { A match { case sparse: SparseMatrix => gemm(alpha, sparse, B, beta, C) @@ -408,8 +410,8 @@ private[spark] object BLAS extends Serializable with Logging { } } } else { - // Scale matrix first if `beta` is not equal to 0.0 - if (beta != 0.0) { + // Scale matrix first if `beta` is not equal to 1.0 + if (beta != 1.0) { f2jBLAS.dscal(C.values.length, beta, C.values, 1) } // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of @@ -470,8 +472,10 @@ private[spark] object BLAS extends Serializable with Logging { s"The columns of A don't match the number of elements of x. A: ${A.numCols}, x: ${x.size}") require(A.numRows == y.size, s"The rows of A don't match the number of elements of y. A: ${A.numRows}, y:${y.size}") - if (alpha == 0.0) { - logDebug("gemv: alpha is equal to 0. Returning y.") + if (alpha == 0.0 && beta == 1.0) { + logDebug("gemv: alpha is equal to 0 and beta is equal to 1. Returning y.") + } else if (alpha == 0.0) { + scal(beta, y) } else { (A, x) match { case (smA: SparseMatrix, dvx: DenseVector) => @@ -526,11 +530,6 @@ private[spark] object BLAS extends Serializable with Logging { val xValues = x.values val yValues = y.values - if (alpha == 0.0) { - scal(beta, y) - return - } - if (A.isTransposed) { var rowCounterForA = 0 while (rowCounterForA < mA) { @@ -581,11 +580,6 @@ private[spark] object BLAS extends Serializable with Logging { val Arows = if (!A.isTransposed) A.rowIndices else A.colPtrs val Acols = if (!A.isTransposed) A.colPtrs else A.rowIndices - if (alpha == 0.0) { - scal(beta, y) - return - } - if (A.isTransposed) { var rowCounter = 0 while (rowCounter < mA) { @@ -604,7 +598,7 @@ private[spark] object BLAS extends Serializable with Logging { rowCounter += 1 } } else { - scal(beta, y) + if (beta != 1.0) scal(beta, y) var colCounterForA = 0 var k = 0 @@ -659,7 +653,7 @@ private[spark] object BLAS extends Serializable with Logging { rowCounter += 1 } } else { - scal(beta, y) + if (beta != 1.0) scal(beta, y) // Perform matrix-vector multiplication and add to y var colCounterForA = 0 while (colCounterForA < nA) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index d119e0b50a393..8db5c8424abe9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -204,6 +204,7 @@ class BLASSuite extends SparkFunSuite { val C14 = C1.copy val C15 = C1.copy val C16 = C1.copy + val C17 = C1.copy val expected2 = new DenseMatrix(4, 2, Array(2.0, 1.0, 4.0, 2.0, 4.0, 0.0, 4.0, 3.0)) val expected3 = new DenseMatrix(4, 2, Array(2.0, 2.0, 4.0, 2.0, 8.0, 0.0, 6.0, 6.0)) val expected4 = new DenseMatrix(4, 2, Array(5.0, 0.0, 10.0, 5.0, 0.0, 0.0, 5.0, 0.0)) @@ -217,6 +218,10 @@ class BLASSuite extends SparkFunSuite { assert(C2 ~== expected2 absTol 1e-15) assert(C3 ~== expected3 absTol 1e-15) assert(C4 ~== expected3 absTol 1e-15) + gemm(1.0, dA, B, 0.0, C17) + assert(C17 ~== expected absTol 1e-15) + gemm(1.0, sA, B, 0.0, C17) + assert(C17 ~== expected absTol 1e-15) withClue("columns of A don't match the rows of B") { intercept[Exception] { From 35e896a79bb5e72d63b82b047f46f4f6fa2e1970 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Aug 2015 21:39:16 -0700 Subject: [PATCH 157/232] SPARK-9545, SPARK-9547: Use Maven in PRB if title contains "[test-maven]" This is just some small glue code to actually make use of the AMPLAB_JENKINS_BUILD_TOOL switch. As far as I can tell, we actually don't currently use the Maven support in the tool even though it exists. This patch switches to Maven when the PR title contains "test-maven". There are a few small other pieces of cleanup in the patch as well. Author: Patrick Wendell Closes #7878 from pwendell/maven-tests. --- dev/run-tests-jenkins | 18 ++++++++++++++++-- dev/run-tests.py | 28 ++++++++++++++++++++++++++-- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 39cf54f78104c..3be78575e70f1 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -164,8 +164,9 @@ pr_message="" current_pr_head="`git rev-parse HEAD`" echo "HEAD: `git rev-parse HEAD`" -echo "GHPRB: $ghprbActualCommit" -echo "SHA1: $sha1" +echo "\$ghprbActualCommit: $ghprbActualCommit" +echo "\$sha1: $sha1" +echo "\$ghprbPullTitle: $ghprbPullTitle" # Run pull request tests for t in "${PR_TESTS[@]}"; do @@ -189,6 +190,19 @@ done { # Marks this build is a pull request build. export AMP_JENKINS_PRB=true + if [[ $ghprbPullTitle == *"test-maven"* ]]; then + export AMPLAB_JENKINS_BUILD_TOOL="maven" + fi + if [[ $ghprbPullTitle == *"test-hadoop1.0"* ]]; then + export AMPLAB_JENKINS_BUILD_PROFILE="hadoop1.0" + elif [[ $ghprbPullTitle == *"test-hadoop2.0"* ]]; then + export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.0" + elif [[ $ghprbPullTitle == *"test-hadoop2.2"* ]]; then + export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.2" + elif [[ $ghprbPullTitle == *"test-hadoop2.3"* ]]; then + export AMPLAB_JENKINS_BUILD_PROFILE="hadoop2.3" + fi + timeout "${TESTS_TIMEOUT}" ./dev/run-tests test_result="$?" diff --git a/dev/run-tests.py b/dev/run-tests.py index 4fd703a7c219f..d8b22e1665e7b 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -21,6 +21,7 @@ import itertools from optparse import OptionParser import os +import random import re import sys import subprocess @@ -239,11 +240,32 @@ def build_spark_documentation(): os.chdir(SPARK_HOME) +def get_zinc_port(): + """ + Get a randomized port on which to start Zinc + """ + return random.randrange(3030, 4030) + + +def kill_zinc_on_port(zinc_port): + """ + Kill the Zinc process running on the given port, if one exists. + """ + cmd = ("/usr/sbin/lsof -P |grep %s | grep LISTEN " + "| awk '{ print $2; }' | xargs kill") % zinc_port + subprocess.check_call(cmd, shell=True) + + def exec_maven(mvn_args=()): """Will call Maven in the current directory with the list of mvn_args passed in and returns the subprocess for any further processing""" - run_cmd([os.path.join(SPARK_HOME, "build", "mvn")] + mvn_args) + zinc_port = get_zinc_port() + os.environ["ZINC_PORT"] = "%s" % zinc_port + zinc_flag = "-DzincPort=%s" % zinc_port + flags = [os.path.join(SPARK_HOME, "build", "mvn"), "--force", zinc_flag] + run_cmd(flags + mvn_args) + kill_zinc_on_port(zinc_port) def exec_sbt(sbt_args=()): @@ -514,7 +536,9 @@ def main(): build_apache_spark(build_tool, hadoop_version) # backwards compatibility checks - detect_binary_inop_with_mima() + if build_tool == "sbt": + # Note: compatiblity tests only supported in sbt for now + detect_binary_inop_with_mima() # run the test suites run_scala_tests(build_tool, hadoop_version, test_modules) From 8694c3ad7dcafca9563649e93b7a08076748d6f2 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Sun, 30 Aug 2015 23:12:56 -0700 Subject: [PATCH 158/232] [SPARK-10351] [SQL] Fixes UTF8String.fromAddress to handle off-heap memory CC rxin marmbrus Author: Feynman Liang Closes #8523 from feynmanliang/SPARK-10351. --- .../test/scala/org/apache/spark/sql/UnsafeRowSuite.scala | 9 +++++---- .../java/org/apache/spark/unsafe/types/UTF8String.java | 6 +----- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala index 219435dff5bc8..2476b10e3cf9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UnsafeRowSuite.scala @@ -43,12 +43,12 @@ class UnsafeRowSuite extends SparkFunSuite { val arrayBackedUnsafeRow: UnsafeRow = UnsafeProjection.create(Array[DataType](StringType, StringType, IntegerType)).apply(row) assert(arrayBackedUnsafeRow.getBaseObject.isInstanceOf[Array[Byte]]) - val bytesFromArrayBackedRow: Array[Byte] = { + val (bytesFromArrayBackedRow, field0StringFromArrayBackedRow): (Array[Byte], String) = { val baos = new ByteArrayOutputStream() arrayBackedUnsafeRow.writeToStream(baos, null) - baos.toByteArray + (baos.toByteArray, arrayBackedUnsafeRow.getString(0)) } - val bytesFromOffheapRow: Array[Byte] = { + val (bytesFromOffheapRow, field0StringFromOffheapRow): (Array[Byte], String) = { val offheapRowPage = MemoryAllocator.UNSAFE.allocate(arrayBackedUnsafeRow.getSizeInBytes) try { Platform.copyMemory( @@ -69,13 +69,14 @@ class UnsafeRowSuite extends SparkFunSuite { val baos = new ByteArrayOutputStream() val writeBuffer = new Array[Byte](1024) offheapUnsafeRow.writeToStream(baos, writeBuffer) - baos.toByteArray + (baos.toByteArray, offheapUnsafeRow.getString(0)) } finally { MemoryAllocator.UNSAFE.free(offheapRowPage) } } assert(bytesFromArrayBackedRow === bytesFromOffheapRow) + assert(field0StringFromArrayBackedRow === field0StringFromOffheapRow) } test("calling getDouble() and getFloat() on null columns") { diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index cbcab958c05a9..216aeea60d1c8 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -90,11 +90,7 @@ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { * Creates an UTF8String from given address (base and offset) and length. */ public static UTF8String fromAddress(Object base, long offset, int numBytes) { - if (base != null) { - return new UTF8String(base, offset, numBytes); - } else { - return null; - } + return new UTF8String(base, offset, numBytes); } /** From f0f563a3c43fc9683e6920890cce44611c0c5f4b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Aug 2015 23:20:03 -0700 Subject: [PATCH 159/232] [SPARK-100354] [MLLIB] fix some apparent memory issues in k-means|| initializaiton * do not cache first cost RDD * change following cost RDD cache level to MEMORY_AND_DISK * remove Vector wrapper to save a object per instance Further improvements will be addressed in SPARK-10329 cc: yu-iskw HuJiayin Author: Xiangrui Meng Closes #8526 from mengxr/SPARK-10354. --- .../spark/mllib/clustering/KMeans.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) 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 46920fffe6e1a..7168aac32c997 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 @@ -369,7 +369,7 @@ class KMeans private ( : Array[Array[VectorWithNorm]] = { // Initialize empty centers and point costs. val centers = Array.tabulate(runs)(r => ArrayBuffer.empty[VectorWithNorm]) - var costs = data.map(_ => Vectors.dense(Array.fill(runs)(Double.PositiveInfinity))).cache() + var costs = data.map(_ => Array.fill(runs)(Double.PositiveInfinity)) // Initialize each run's first center to a random point. val seed = new XORShiftRandom(this.seed).nextInt() @@ -394,21 +394,28 @@ class KMeans private ( val bcNewCenters = data.context.broadcast(newCenters) val preCosts = costs costs = data.zip(preCosts).map { case (point, cost) => - Vectors.dense( Array.tabulate(runs) { r => math.min(KMeans.pointCost(bcNewCenters.value(r), point), cost(r)) - }) - }.cache() + } + }.persist(StorageLevel.MEMORY_AND_DISK) val sumCosts = costs - .aggregate(Vectors.zeros(runs))( + .aggregate(new Array[Double](runs))( seqOp = (s, v) => { // s += v - axpy(1.0, v, s) + var r = 0 + while (r < runs) { + s(r) += v(r) + r += 1 + } s }, combOp = (s0, s1) => { // s0 += s1 - axpy(1.0, s1, s0) + var r = 0 + while (r < runs) { + s0(r) += s1(r) + r += 1 + } s0 } ) From 72f6dbf7b0c8b271f5f9c762374422c69c8ab43d Mon Sep 17 00:00:00 2001 From: EugenCepoi Date: Mon, 31 Aug 2015 13:24:35 -0500 Subject: [PATCH 160/232] [SPARK-8730] Fixes - Deser objects containing a primitive class attribute Author: EugenCepoi Closes #7122 from EugenCepoi/master. --- .../spark/serializer/JavaSerializer.scala | 27 +++++++++++++++---- .../serializer/JavaSerializerSuite.scala | 18 +++++++++++++ 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 4a5274b46b7a0..b463a71d5bd7d 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -62,17 +62,34 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa extends DeserializationStream { private val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass): Class[_] = { - // scalastyle:off classforname - Class.forName(desc.getName, false, loader) - // scalastyle:on classforname - } + override def resolveClass(desc: ObjectStreamClass): Class[_] = + try { + // scalastyle:off classforname + Class.forName(desc.getName, false, loader) + // scalastyle:on classforname + } catch { + case e: ClassNotFoundException => + JavaDeserializationStream.primitiveMappings.get(desc.getName).getOrElse(throw e) + } } def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] def close() { objIn.close() } } +private object JavaDeserializationStream { + val primitiveMappings = Map[String, Class[_]]( + "boolean" -> classOf[Boolean], + "byte" -> classOf[Byte], + "char" -> classOf[Char], + "short" -> classOf[Short], + "int" -> classOf[Int], + "long" -> classOf[Long], + "float" -> classOf[Float], + "double" -> classOf[Double], + "void" -> classOf[Void] + ) +} private[spark] class JavaSerializerInstance( counterReset: Int, extraDebugInfo: Boolean, defaultClassLoader: ClassLoader) diff --git a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala index 329a2b6dad831..20f45670bc2ba 100644 --- a/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/JavaSerializerSuite.scala @@ -25,4 +25,22 @@ class JavaSerializerSuite extends SparkFunSuite { val instance = serializer.newInstance() instance.deserialize[JavaSerializer](instance.serialize(serializer)) } + + test("Deserialize object containing a primitive Class as attribute") { + val serializer = new JavaSerializer(new SparkConf()) + val instance = serializer.newInstance() + instance.deserialize[JavaSerializer](instance.serialize(new ContainsPrimitiveClass())) + } +} + +private class ContainsPrimitiveClass extends Serializable { + val intClass = classOf[Int] + val longClass = classOf[Long] + val shortClass = classOf[Short] + val charClass = classOf[Char] + val doubleClass = classOf[Double] + val floatClass = classOf[Float] + val booleanClass = classOf[Boolean] + val byteClass = classOf[Byte] + val voidClass = classOf[Void] } From 4a5fe091658b1d06f427e404a11a84fc84f953c5 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 31 Aug 2015 12:19:11 -0700 Subject: [PATCH 161/232] [SPARK-10369] [STREAMING] Don't remove ReceiverTrackingInfo when deregisterReceivering since we may reuse it later `deregisterReceiver` should not remove `ReceiverTrackingInfo`. Otherwise, it will throw `java.util.NoSuchElementException: key not found` when restarting it. Author: zsxwing Closes #8538 from zsxwing/SPARK-10369. --- .../streaming/scheduler/ReceiverTracker.scala | 4 +- .../scheduler/ReceiverTrackerSuite.scala | 51 +++++++++++++++++++ 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 3d532a675db02..f86fd44b48719 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -291,7 +291,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false ReceiverTrackingInfo( streamId, ReceiverState.INACTIVE, None, None, None, None, Some(errorInfo)) } - receiverTrackingInfos -= streamId + receiverTrackingInfos(streamId) = newReceiverTrackingInfo listenerBus.post(StreamingListenerReceiverStopped(newReceiverTrackingInfo.toReceiverInfo)) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" @@ -483,7 +483,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(true) // Local messages case AllReceiverIds => - context.reply(receiverTrackingInfos.keys.toSeq) + context.reply(receiverTrackingInfos.filter(_._2.state != ReceiverState.INACTIVE).keys.toSeq) case StopAllReceivers => assert(isTrackerStopping || isTrackerStopped) stopReceivers() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index dd292ba4dd949..45138b748ecab 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -60,6 +60,26 @@ class ReceiverTrackerSuite extends TestSuiteBase { } } } + + test("should restart receiver after stopping it") { + withStreamingContext(new StreamingContext(conf, Milliseconds(100))) { ssc => + @volatile var startTimes = 0 + ssc.addStreamingListener(new StreamingListener { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted): Unit = { + startTimes += 1 + } + }) + val input = ssc.receiverStream(new StoppableReceiver) + val output = new TestOutputStream(input) + output.register() + ssc.start() + StoppableReceiver.shouldStop = true + eventually(timeout(10 seconds), interval(10 millis)) { + // The receiver is stopped once, so if it's restarted, it should be started twice. + assert(startTimes === 2) + } + } + } } /** An input DStream with for testing rate controlling */ @@ -132,3 +152,34 @@ private[streaming] object RateTestReceiver { def getActive(): Option[RateTestReceiver] = Option(activeReceiver) } + +/** + * A custom receiver that could be stopped via StoppableReceiver.shouldStop + */ +class StoppableReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + + var receivingThreadOption: Option[Thread] = None + + def onStart() { + val thread = new Thread() { + override def run() { + while (!StoppableReceiver.shouldStop) { + Thread.sleep(10) + } + StoppableReceiver.this.stop("stop") + } + } + thread.start() + } + + def onStop() { + StoppableReceiver.shouldStop = true + receivingThreadOption.foreach(_.join()) + // Reset it so as to restart it + StoppableReceiver.shouldStop = false + } +} + +object StoppableReceiver { + @volatile var shouldStop = false +} From a2d5c72091b1c602694dbca823a7b26f86b02864 Mon Sep 17 00:00:00 2001 From: sureshthalamati Date: Mon, 31 Aug 2015 12:39:58 -0700 Subject: [PATCH 162/232] [SPARK-10170] [SQL] Add DB2 JDBC dialect support. Data frame write to DB2 database is failing because by default JDBC data source implementation is generating a table schema with DB2 unsupported data types TEXT for String, and BIT1(1) for Boolean. This patch registers DB2 JDBC Dialect that maps String, Boolean to valid DB2 data types. Author: sureshthalamati Closes #8393 from sureshthalamati/db2_dialect_spark-10170. --- .../apache/spark/sql/jdbc/JdbcDialects.scala | 18 ++++++++++++++++++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 7 +++++++ 2 files changed, 25 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index 8849fc2f1f0ef..c6d05c9b83b98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -125,6 +125,7 @@ object JdbcDialects { registerDialect(MySQLDialect) registerDialect(PostgresDialect) + registerDialect(DB2Dialect) /** * Fetch the JdbcDialect class corresponding to a given database url. @@ -222,3 +223,20 @@ case object MySQLDialect extends JdbcDialect { s"`$colName`" } } + +/** + * :: DeveloperApi :: + * Default DB2 dialect, mapping string/boolean on write to valid DB2 types. + * By default string, and boolean gets mapped to db2 invalid types TEXT, and BIT(1). + */ +@DeveloperApi +case object DB2Dialect extends JdbcDialect { + + override def canHandle(url: String): Boolean = url.startsWith("jdbc:db2") + + override def getJDBCType(dt: DataType): Option[JdbcType] = dt match { + case StringType => Some(JdbcType("CLOB", java.sql.Types.CLOB)) + case BooleanType => Some(JdbcType("CHAR(1)", java.sql.Types.CHAR)) + case _ => None + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 0edac0848c3bb..d8c9a08d84c61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -407,6 +407,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext test("Default jdbc dialect registration") { assert(JdbcDialects.get("jdbc:mysql://127.0.0.1/db") == MySQLDialect) assert(JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") == PostgresDialect) + assert(JdbcDialects.get("jdbc:db2://127.0.0.1/db") == DB2Dialect) assert(JdbcDialects.get("test.invalid") == NoopDialect) } @@ -443,4 +444,10 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext assert(agg.getCatalystType(0, "", 1, null) === Some(LongType)) assert(agg.getCatalystType(1, "", 1, null) === Some(StringType)) } + + test("DB2Dialect type mapping") { + val db2Dialect = JdbcDialects.get("jdbc:db2://127.0.0.1/db") + assert(db2Dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "CLOB") + assert(db2Dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)") + } } From 23e39cc7b1bb7f1087c4706234c9b5165a571357 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Aug 2015 15:49:25 -0700 Subject: [PATCH 163/232] [SPARK-9954] [MLLIB] use first 128 nonzeros to compute Vector.hashCode This could help reduce hash collisions, e.g., in `RDD[Vector].repartition`. jkbradley Author: Xiangrui Meng Closes #8182 from mengxr/SPARK-9954. --- .../apache/spark/mllib/linalg/Vectors.scala | 38 ++++++++++--------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 06ebb15869909..3642e9286504f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -71,20 +71,22 @@ sealed trait Vector extends Serializable { } /** - * Returns a hash code value for the vector. The hash code is based on its size and its nonzeros - * in the first 16 entries, using a hash algorithm similar to [[java.util.Arrays.hashCode]]. + * Returns a hash code value for the vector. The hash code is based on its size and its first 128 + * nonzero entries, using a hash algorithm similar to [[java.util.Arrays.hashCode]]. */ override def hashCode(): Int = { // This is a reference implementation. It calls return in foreachActive, which is slow. // Subclasses should override it with optimized implementation. var result: Int = 31 + size + var nnz = 0 this.foreachActive { (index, value) => - if (index < 16) { + if (nnz < Vectors.MAX_HASH_NNZ) { // ignore explicit 0 for comparison between sparse and dense if (value != 0) { result = 31 * result + index val bits = java.lang.Double.doubleToLongBits(value) result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 } } else { return result @@ -536,6 +538,9 @@ object Vectors { } allEqual } + + /** Max number of nonzero entries used in computing hash code. */ + private[linalg] val MAX_HASH_NNZ = 128 } /** @@ -578,13 +583,15 @@ class DenseVector @Since("1.0.0") ( override def hashCode(): Int = { var result: Int = 31 + size var i = 0 - val end = math.min(values.length, 16) - while (i < end) { + val end = values.length + var nnz = 0 + while (i < end && nnz < Vectors.MAX_HASH_NNZ) { val v = values(i) if (v != 0.0) { result = 31 * result + i val bits = java.lang.Double.doubleToLongBits(values(i)) result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 } i += 1 } @@ -707,19 +714,16 @@ class SparseVector @Since("1.0.0") ( override def hashCode(): Int = { var result: Int = 31 + size val end = values.length - var continue = true var k = 0 - while ((k < end) & continue) { - val i = indices(k) - if (i < 16) { - val v = values(k) - if (v != 0.0) { - result = 31 * result + i - val bits = java.lang.Double.doubleToLongBits(v) - result = 31 * result + (bits ^ (bits >>> 32)).toInt - } - } else { - continue = false + var nnz = 0 + while (k < end && nnz < Vectors.MAX_HASH_NNZ) { + val v = values(k) + if (v != 0.0) { + val i = indices(k) + result = 31 * result + i + val bits = java.lang.Double.doubleToLongBits(v) + result = 31 * result + (bits ^ (bits >>> 32)).toInt + nnz += 1 } k += 1 } From 5b3245d6dff65972fc39c73f90d5cbdf84d19129 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 31 Aug 2015 15:50:41 -0700 Subject: [PATCH 164/232] [SPARK-8472] [ML] [PySpark] Python API for DCT Add Python API for ml.feature.DCT. Author: Yanbo Liang Closes #8485 from yanboliang/spark-8472. --- python/pyspark/ml/feature.py | 65 +++++++++++++++++++++++++++++++++++- 1 file changed, 64 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 04b2b2ccc9e55..59300a607815b 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -26,7 +26,7 @@ from pyspark.mllib.common import inherit_doc from pyspark.mllib.linalg import _convert_to_vector -__all__ = ['Binarizer', 'Bucketizer', 'ElementwiseProduct', 'HashingTF', 'IDF', 'IDFModel', +__all__ = ['Binarizer', 'Bucketizer', 'DCT', 'ElementwiseProduct', 'HashingTF', 'IDF', 'IDFModel', 'NGram', 'Normalizer', 'OneHotEncoder', 'PolynomialExpansion', 'RegexTokenizer', 'StandardScaler', 'StandardScalerModel', 'StringIndexer', 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', 'Word2Vec', 'Word2VecModel', @@ -166,6 +166,69 @@ def getSplits(self): return self.getOrDefault(self.splits) +@inherit_doc +class DCT(JavaTransformer, HasInputCol, HasOutputCol): + """ + A feature transformer that takes the 1D discrete cosine transform + of a real vector. No zero padding is performed on the input vector. + It returns a real vector of the same length representing the DCT. + The return vector is scaled such that the transform matrix is + unitary (aka scaled DCT-II). + + More information on + `https://en.wikipedia.org/wiki/Discrete_cosine_transform#DCT-II Wikipedia`. + + >>> from pyspark.mllib.linalg import Vectors + >>> df1 = sqlContext.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) + >>> dct = DCT(inverse=False, inputCol="vec", outputCol="resultVec") + >>> df2 = dct.transform(df1) + >>> df2.head().resultVec + DenseVector([10.969..., -0.707..., -2.041...]) + >>> df3 = DCT(inverse=True, inputCol="resultVec", outputCol="origVec").transform(df2) + >>> df3.head().origVec + DenseVector([5.0, 8.0, 6.0]) + """ + + # a placeholder to make it appear in the generated doc + inverse = Param(Params._dummy(), "inverse", "Set transformer to perform inverse DCT, " + + "default False.") + + @keyword_only + def __init__(self, inverse=False, inputCol=None, outputCol=None): + """ + __init__(self, inverse=False, inputCol=None, outputCol=None) + """ + super(DCT, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.DCT", self.uid) + self.inverse = Param(self, "inverse", "Set transformer to perform inverse DCT, " + + "default False.") + self._setDefault(inverse=False) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inverse=False, inputCol=None, outputCol=None): + """ + setParams(self, inverse=False, inputCol=None, outputCol=None) + Sets params for this DCT. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setInverse(self, value): + """ + Sets the value of :py:attr:`inverse`. + """ + self._paramMap[self.inverse] = value + return self + + def getInverse(self): + """ + Gets the value of inverse or its default value. + """ + return self.getOrDefault(self.inverse) + + @inherit_doc class ElementwiseProduct(JavaTransformer, HasInputCol, HasOutputCol): """ From 540bdee93103a73736d282b95db6a8cda8f6a2b1 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 31 Aug 2015 15:55:22 -0700 Subject: [PATCH 165/232] [SPARK-10341] [SQL] fix memory starving in unsafe SMJ In SMJ, the first ExternalSorter could consume all the memory before spilling, then the second can not even acquire the first page. Before we have a better memory allocator, SMJ should call prepare() before call any compute() of it's children. cc rxin JoshRosen Author: Davies Liu Closes #8511 from davies/smj_memory. --- .../rdd/MapPartitionsWithPreparationRDD.scala | 21 +++++++++++++++++-- .../spark/rdd/ZippedPartitionsRDD.scala | 13 ++++++++++++ ...MapPartitionsWithPreparationRDDSuite.scala | 14 +++++++++---- 3 files changed, 42 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala index b475bd8d79f85..1f2213d0c4346 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Partition, Partitioner, TaskContext} @@ -38,12 +39,28 @@ private[spark] class MapPartitionsWithPreparationRDD[U: ClassTag, T: ClassTag, M override def getPartitions: Array[Partition] = firstParent[T].partitions + // In certain join operations, prepare can be called on the same partition multiple times. + // In this case, we need to ensure that each call to compute gets a separate prepare argument. + private[this] var preparedArguments: ArrayBuffer[M] = new ArrayBuffer[M] + + /** + * Prepare a partition for a single call to compute. + */ + def prepare(): Unit = { + preparedArguments += preparePartition() + } + /** * Prepare a partition before computing it from its parent. */ override def compute(partition: Partition, context: TaskContext): Iterator[U] = { - val preparedArgument = preparePartition() + val prepared = + if (preparedArguments.isEmpty) { + preparePartition() + } else { + preparedArguments.remove(0) + } val parentIterator = firstParent[T].iterator(partition, context) - executePartition(context, partition.index, preparedArgument, parentIterator) + executePartition(context, partition.index, prepared, parentIterator) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 81f40ad33aa5d..b3c64394abc76 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -73,6 +73,16 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( super.clearDependencies() rdds = null } + + /** + * Call the prepare method of every parent that has one. + * This is needed for reserving execution memory in advance. + */ + protected def tryPrepareParents(): Unit = { + rdds.collect { + case rdd: MapPartitionsWithPreparationRDD[_, _, _] => rdd.prepare() + } + } } private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( @@ -84,6 +94,7 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { + tryPrepareParents() val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context)) } @@ -107,6 +118,7 @@ private[spark] class ZippedPartitionsRDD3 extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { + tryPrepareParents() val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), @@ -134,6 +146,7 @@ private[spark] class ZippedPartitionsRDD4 extends ZippedPartitionsBaseRDD[V](sc, List(rdd1, rdd2, rdd3, rdd4), preservesPartitioning) { override def compute(s: Partition, context: TaskContext): Iterator[V] = { + tryPrepareParents() val partitions = s.asInstanceOf[ZippedPartitionsPartition].partitions f(rdd1.iterator(partitions(0), context), rdd2.iterator(partitions(1), context), diff --git a/core/src/test/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDDSuite.scala index c16930e7d6491..e281e817e493d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/MapPartitionsWithPreparationRDDSuite.scala @@ -46,11 +46,17 @@ class MapPartitionsWithPreparationRDDSuite extends SparkFunSuite with LocalSpark } // Verify that the numbers are pushed in the order expected - val result = { - new MapPartitionsWithPreparationRDD[Int, Int, Unit]( - parent, preparePartition, executePartition).collect() - } + val rdd = new MapPartitionsWithPreparationRDD[Int, Int, Unit]( + parent, preparePartition, executePartition) + val result = rdd.collect() assert(result === Array(10, 20, 30)) + + TestObject.things.clear() + // Zip two of these RDDs, both should be prepared before the parent is executed + val rdd2 = new MapPartitionsWithPreparationRDD[Int, Int, Unit]( + parent, preparePartition, executePartition) + val result2 = rdd.zipPartitions(rdd2)((a, b) => a).collect() + assert(result2 === Array(10, 10, 20, 30, 20, 30)) } } From fe16fd0b8b717f01151bc659ec3299dab091c97a Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 31 Aug 2015 16:06:38 -0700 Subject: [PATCH 166/232] [SPARK-10349] [ML] OneVsRest use 'when ... otherwise' not UDF to generate new label at binary reduction Currently OneVsRest use UDF to generate new binary label during training. Considering that [SPARK-7321](https://issues.apache.org/jira/browse/SPARK-7321) has been merged, we can use ```when ... otherwise``` which will be more efficiency. Author: Yanbo Liang Closes #8519 from yanboliang/spark-10349. --- .../org/apache/spark/ml/classification/OneVsRest.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index c62e132f5d533..debc164bf2432 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -91,7 +91,6 @@ final class OneVsRestModel private[ml] ( // add an accumulator column to store predictions of all the models val accColName = "mbc$acc" + UUID.randomUUID().toString val initUDF = udf { () => Map[Int, Double]() } - val mapType = MapType(IntegerType, DoubleType, valueContainsNull = false) val newDataset = dataset.withColumn(accColName, initUDF()) // persist if underlying dataset is not persistent. @@ -195,16 +194,11 @@ final class OneVsRest(override val uid: String) // create k columns, one for each binary classifier. val models = Range(0, numClasses).par.map { index => - val labelUDF = udf { (label: Double) => - if (label.toInt == index) 1.0 else 0.0 - } - // generate new label metadata for the binary problem. - // TODO: use when ... otherwise after SPARK-7321 is merged val newLabelMeta = BinaryAttribute.defaultAttr.withName("label").toMetadata() val labelColName = "mc2b$" + index - val trainingDataset = - multiclassLabeled.withColumn(labelColName, labelUDF(col($(labelCol))), newLabelMeta) + val trainingDataset = multiclassLabeled.withColumn( + labelColName, when(col($(labelCol)) === index.toDouble, 1.0).otherwise(0.0), newLabelMeta) val classifier = getClassifier val paramMap = new ParamMap() paramMap.put(classifier.labelCol -> labelColName) From 52ea399e6ee37b7c44aae7709863e006fca88906 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 31 Aug 2015 16:11:27 -0700 Subject: [PATCH 167/232] [SPARK-10355] [ML] [PySpark] Add Python API for SQLTransformer Add Python API for SQLTransformer Author: Yanbo Liang Closes #8527 from yanboliang/spark-10355. --- python/pyspark/ml/feature.py | 57 ++++++++++++++++++++++++++++++++++-- 1 file changed, 54 insertions(+), 3 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 59300a607815b..0626281e200a1 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -28,9 +28,9 @@ __all__ = ['Binarizer', 'Bucketizer', 'DCT', 'ElementwiseProduct', 'HashingTF', 'IDF', 'IDFModel', 'NGram', 'Normalizer', 'OneHotEncoder', 'PolynomialExpansion', 'RegexTokenizer', - 'StandardScaler', 'StandardScalerModel', 'StringIndexer', 'StringIndexerModel', - 'Tokenizer', 'VectorAssembler', 'VectorIndexer', 'Word2Vec', 'Word2VecModel', - 'PCA', 'PCAModel', 'RFormula', 'RFormulaModel'] + 'SQLTransformer', 'StandardScaler', 'StandardScalerModel', 'StringIndexer', + 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', 'Word2Vec', + 'Word2VecModel', 'PCA', 'PCAModel', 'RFormula', 'RFormulaModel'] @inherit_doc @@ -743,6 +743,57 @@ def getPattern(self): return self.getOrDefault(self.pattern) +@inherit_doc +class SQLTransformer(JavaTransformer): + """ + Implements the transforms which are defined by SQL statement. + Currently we only support SQL syntax like 'SELECT ... FROM __THIS__' + where '__THIS__' represents the underlying table of the input dataset. + + >>> df = sqlContext.createDataFrame([(0, 1.0, 3.0), (2, 2.0, 5.0)], ["id", "v1", "v2"]) + >>> sqlTrans = SQLTransformer( + ... statement="SELECT *, (v1 + v2) AS v3, (v1 * v2) AS v4 FROM __THIS__") + >>> sqlTrans.transform(df).head() + Row(id=0, v1=1.0, v2=3.0, v3=4.0, v4=3.0) + """ + + # a placeholder to make it appear in the generated doc + statement = Param(Params._dummy(), "statement", "SQL statement") + + @keyword_only + def __init__(self, statement=None): + """ + __init__(self, statement=None) + """ + super(SQLTransformer, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.SQLTransformer", self.uid) + self.statement = Param(self, "statement", "SQL statement") + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, statement=None): + """ + setParams(self, statement=None) + Sets params for this SQLTransformer. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setStatement(self, value): + """ + Sets the value of :py:attr:`statement`. + """ + self._paramMap[self.statement] = value + return self + + def getStatement(self): + """ + Gets the value of statement or its default value. + """ + return self.getOrDefault(self.statement) + + @inherit_doc class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol): """ From d65656c455d19b83c6412571873586b458aa355e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 31 Aug 2015 18:09:24 -0700 Subject: [PATCH 168/232] [SPARK-10378][SQL][Test] Remove HashJoinCompatibilitySuite. They don't bring much value since we now have better unit test coverage for hash joins. This will also help reduce the test time. Author: Reynold Xin Closes #8542 from rxin/SPARK-10378. --- .../HashJoinCompatibilitySuite.scala | 169 ------------------ 1 file changed, 169 deletions(-) delete mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HashJoinCompatibilitySuite.scala diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HashJoinCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HashJoinCompatibilitySuite.scala deleted file mode 100644 index 1a5ba20404c4e..0000000000000 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HashJoinCompatibilitySuite.scala +++ /dev/null @@ -1,169 +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.execution - -import java.io.File - -import org.apache.spark.sql.SQLConf -import org.apache.spark.sql.hive.test.TestHive - -/** - * Runs the test cases that are included in the hive distribution with hash joins. - */ -class HashJoinCompatibilitySuite extends HiveCompatibilitySuite { - override def beforeAll() { - super.beforeAll() - TestHive.setConf(SQLConf.SORTMERGE_JOIN, false) - } - - override def afterAll() { - TestHive.setConf(SQLConf.SORTMERGE_JOIN, true) - super.afterAll() - } - - override def whiteList = Seq( - "auto_join0", - "auto_join1", - "auto_join10", - "auto_join11", - "auto_join12", - "auto_join13", - "auto_join14", - "auto_join14_hadoop20", - "auto_join15", - "auto_join17", - "auto_join18", - "auto_join19", - "auto_join2", - "auto_join20", - "auto_join21", - "auto_join22", - "auto_join23", - "auto_join24", - "auto_join25", - "auto_join26", - "auto_join27", - "auto_join28", - "auto_join3", - "auto_join30", - "auto_join31", - "auto_join32", - "auto_join4", - "auto_join5", - "auto_join6", - "auto_join7", - "auto_join8", - "auto_join9", - "auto_join_filters", - "auto_join_nulls", - "auto_join_reordering_values", - "auto_smb_mapjoin_14", - "auto_sortmerge_join_1", - "auto_sortmerge_join_10", - "auto_sortmerge_join_11", - "auto_sortmerge_join_12", - "auto_sortmerge_join_13", - "auto_sortmerge_join_14", - "auto_sortmerge_join_15", - "auto_sortmerge_join_16", - "auto_sortmerge_join_2", - "auto_sortmerge_join_3", - "auto_sortmerge_join_4", - "auto_sortmerge_join_5", - "auto_sortmerge_join_6", - "auto_sortmerge_join_7", - "auto_sortmerge_join_8", - "auto_sortmerge_join_9", - "correlationoptimizer1", - "correlationoptimizer10", - "correlationoptimizer11", - "correlationoptimizer13", - "correlationoptimizer14", - "correlationoptimizer15", - "correlationoptimizer2", - "correlationoptimizer3", - "correlationoptimizer4", - "correlationoptimizer6", - "correlationoptimizer7", - "correlationoptimizer8", - "correlationoptimizer9", - "join0", - "join1", - "join10", - "join11", - "join12", - "join13", - "join14", - "join14_hadoop20", - "join15", - "join16", - "join17", - "join18", - "join19", - "join2", - "join20", - "join21", - "join22", - "join23", - "join24", - "join25", - "join26", - "join27", - "join28", - "join29", - "join3", - "join30", - "join31", - "join32", - "join32_lessSize", - "join33", - "join34", - "join35", - "join36", - "join37", - "join38", - "join39", - "join4", - "join40", - "join41", - "join5", - "join6", - "join7", - "join8", - "join9", - "join_1to1", - "join_array", - "join_casesensitive", - "join_empty", - "join_filters", - "join_hive_626", - "join_map_ppr", - "join_nulls", - "join_nullsafe", - "join_rc", - "join_reorder2", - "join_reorder3", - "join_reorder4", - "join_star" - ) - - // Only run those query tests in the realWhileList (do not try other ignored query files). - override def testCases: Seq[(String, File)] = super.testCases.filter { - case (name, _) => realWhiteList.contains(name) - } -} From 391e6be0ae883f3ea0fab79463eb8b618af79afb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 1 Sep 2015 16:52:59 +0800 Subject: [PATCH 169/232] [SPARK-10301] [SQL] Fixes schema merging for nested structs This PR can be quite challenging to review. I'm trying to give a detailed description of the problem as well as its solution here. When reading Parquet files, we need to specify a potentially nested Parquet schema (of type `MessageType`) as requested schema for column pruning. This Parquet schema is translated from a Catalyst schema (of type `StructType`), which is generated by the query planner and represents all requested columns. However, this translation can be fairly complicated because of several reasons: 1. Requested schema must conform to the real schema of the physical file to be read. This means we have to tailor the actual file schema of every individual physical Parquet file to be read according to the given Catalyst schema. Fortunately we are already doing this in Spark 1.5 by pushing request schema conversion to executor side in PR #7231. 1. Support for schema merging. A single Parquet dataset may consist of multiple physical Parquet files come with different but compatible schemas. This means we may request for a column path that doesn't exist in a physical Parquet file. All requested column paths can be nested. For example, for a Parquet file schema ``` message root { required group f0 { required group f00 { required int32 f000; required binary f001 (UTF8); } } } ``` we may request for column paths defined in the following schema: ``` message root { required group f0 { required group f00 { required binary f001 (UTF8); required float f002; } } optional double f1; } ``` Notice that we pruned column path `f0.f00.f000`, but added `f0.f00.f002` and `f1`. The good news is that Parquet handles non-existing column paths properly and always returns null for them. 1. The map from `StructType` to `MessageType` is a one-to-many map. This is the most unfortunate part. Due to historical reasons (dark histories!), schemas of Parquet files generated by different libraries have different "flavors". For example, to handle a schema with a single non-nullable column, whose type is an array of non-nullable integers, parquet-protobuf generates the following Parquet schema: ``` message m0 { repeated int32 f; } ``` while parquet-avro generates another version: ``` message m1 { required group f (LIST) { repeated int32 array; } } ``` and parquet-thrift spills this: ``` message m1 { required group f (LIST) { repeated int32 f_tuple; } } ``` All of them can be mapped to the following _unique_ Catalyst schema: ``` StructType( StructField( "f", ArrayType(IntegerType, containsNull = false), nullable = false)) ``` This greatly complicates Parquet requested schema construction, since the path of a given column varies in different cases. To read the array elements from files with the above schemas, we must use `f` for `m0`, `f.array` for `m1`, and `f.f_tuple` for `m2`. In earlier Spark versions, we didn't try to fix this issue properly. Spark 1.4 and prior versions simply translate the Catalyst schema in a way more or less compatible with parquet-hive and parquet-avro, but is broken in many other cases. Earlier revisions of Spark 1.5 only try to tailor the Parquet file schema at the first level, and ignore nested ones. This caused [SPARK-10301] [spark-10301] as well as [SPARK-10005] [spark-10005]. In PR #8228, I tried to avoid the hard part of the problem and made a minimum change in `CatalystRowConverter` to fix SPARK-10005. However, when taking SPARK-10301 into consideration, keeping hacking `CatalystRowConverter` doesn't seem to be a good idea. So this PR is an attempt to fix the problem in a proper way. For a given physical Parquet file with schema `ps` and a compatible Catalyst requested schema `cs`, we use the following algorithm to tailor `ps` to get the result Parquet requested schema `ps'`: For a leaf column path `c` in `cs`: - if `c` exists in `cs` and a corresponding Parquet column path `c'` can be found in `ps`, `c'` should be included in `ps'`; - otherwise, we convert `c` to a Parquet column path `c"` using `CatalystSchemaConverter`, and include `c"` in `ps'`; - no other column paths should exist in `ps'`. Then comes the most tedious part: > Given `cs`, `ps`, and `c`, how to locate `c'` in `ps`? Unfortunately, there's no quick answer, and we have to enumerate all possible structures defined in parquet-format spec. They are: 1. the standard structure of nested types, and 1. cases defined in all backwards-compatibility rules for `LIST` and `MAP`. The core part of this PR is `CatalystReadSupport.clipParquetType()`, which tailors a given Parquet file schema according to a requested schema in its Catalyst form. Backwards-compatibility rules of `LIST` and `MAP` are covered in `clipParquetListType()` and `clipParquetMapType()` respectively. The column path selection algorithm is implemented in `clipParquetGroupFields()`. With this PR, we no longer need to do schema tailoring in `CatalystReadSupport` and `CatalystRowConverter`. Another benefit is that, now we can also read Parquet datasets consist of files with different physical Parquet schema but share the same logical schema, for example, files generated by different Parquet libraries. This situation is illustrated by [this test case] [test-case]. [spark-10301]: https://issues.apache.org/jira/browse/SPARK-10301 [spark-10005]: https://issues.apache.org/jira/browse/SPARK-10005 [test-case]: https://github.com/liancheng/spark/commit/38644d8a45175cbdf20d2ace021c2c2544a50ab3#diff-a9b98e28ce3ae30641829dffd1173be2R26 Author: Cheng Lian Closes #8509 from liancheng/spark-10301/fix-parquet-requested-schema. --- .../parquet/CatalystReadSupport.scala | 235 +++++++++---- .../parquet/CatalystRowConverter.scala | 51 +-- .../parquet/CatalystSchemaConverter.scala | 14 +- .../ParquetAvroCompatibilitySuite.scala | 1 + .../ParquetInteroperabilitySuite.scala | 90 +++++ .../parquet/ParquetQuerySuite.scala | 77 +++++ .../parquet/ParquetSchemaSuite.scala | 310 ++++++++++++++++++ 7 files changed, 653 insertions(+), 125 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 0a6bb44445f6e..dc4ff06df6f22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -19,17 +19,18 @@ package org.apache.spark.sql.execution.datasources.parquet import java.util.{Map => JMap} -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters.{collectionAsScalaIterableConverter, mapAsJavaMapConverter, mapAsScalaMapConverter} import org.apache.hadoop.conf.Configuration import org.apache.parquet.hadoop.api.ReadSupport.ReadContext import org.apache.parquet.hadoop.api.{InitContext, ReadSupport} import org.apache.parquet.io.api.RecordMaterializer -import org.apache.parquet.schema.MessageType +import org.apache.parquet.schema.Type.Repetition +import org.apache.parquet.schema._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with Logging { // Called after `init()` when initializing Parquet record reader. @@ -81,70 +82,10 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with // `StructType` containing all requested columns. val maybeRequestedSchema = Option(conf.get(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) - // Below we construct a Parquet schema containing all requested columns. This schema tells - // Parquet which columns to read. - // - // If `maybeRequestedSchema` is defined, we assemble an equivalent Parquet schema. Otherwise, - // we have to fallback to the full file schema which contains all columns in the file. - // Obviously this may waste IO bandwidth since it may read more columns than requested. - // - // Two things to note: - // - // 1. It's possible that some requested columns don't exist in the target Parquet file. For - // example, in the case of schema merging, the globally merged schema may contain extra - // columns gathered from other Parquet files. These columns will be simply filled with nulls - // when actually reading the target Parquet file. - // - // 2. When `maybeRequestedSchema` is available, we can't simply convert the Catalyst schema to - // Parquet schema using `CatalystSchemaConverter`, because the mapping is not unique due to - // non-standard behaviors of some Parquet libraries/tools. For example, a Parquet file - // containing a single integer array field `f1` may have the following legacy 2-level - // structure: - // - // message root { - // optional group f1 (LIST) { - // required INT32 element; - // } - // } - // - // while `CatalystSchemaConverter` may generate a standard 3-level structure: - // - // message root { - // optional group f1 (LIST) { - // repeated group list { - // required INT32 element; - // } - // } - // } - // - // Apparently, we can't use the 2nd schema to read the target Parquet file as they have - // different physical structures. val parquetRequestedSchema = maybeRequestedSchema.fold(context.getFileSchema) { schemaString => - val toParquet = new CatalystSchemaConverter(conf) - val fileSchema = context.getFileSchema.asGroupType() - val fileFieldNames = fileSchema.getFields.asScala.map(_.getName).toSet - - StructType - // Deserializes the Catalyst schema of requested columns - .fromString(schemaString) - .map { field => - if (fileFieldNames.contains(field.name)) { - // If the field exists in the target Parquet file, extracts the field type from the - // full file schema and makes a single-field Parquet schema - new MessageType("root", fileSchema.getType(field.name)) - } else { - // Otherwise, just resorts to `CatalystSchemaConverter` - toParquet.convert(StructType(Array(field))) - } - } - // Merges all single-field Parquet schemas to form a complete schema for all requested - // columns. Note that it's possible that no columns are requested at all (e.g., count - // some partition column of a partitioned Parquet table). That's why `fold` is used here - // and always fallback to an empty Parquet schema. - .fold(new MessageType("root")) { - _ union _ - } + val catalystRequestedSchema = StructType.fromString(schemaString) + CatalystReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema) } val metadata = @@ -160,4 +101,168 @@ private[parquet] object CatalystReadSupport { val SPARK_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" val SPARK_METADATA_KEY = "org.apache.spark.sql.parquet.row.metadata" + + /** + * Tailors `parquetSchema` according to `catalystSchema` by removing column paths don't exist + * in `catalystSchema`, and adding those only exist in `catalystSchema`. + */ + def clipParquetSchema(parquetSchema: MessageType, catalystSchema: StructType): MessageType = { + val clippedParquetFields = clipParquetGroupFields(parquetSchema.asGroupType(), catalystSchema) + Types.buildMessage().addFields(clippedParquetFields: _*).named("root") + } + + private def clipParquetType(parquetType: Type, catalystType: DataType): Type = { + catalystType match { + case t: ArrayType if !isPrimitiveCatalystType(t.elementType) => + // Only clips array types with nested type as element type. + clipParquetListType(parquetType.asGroupType(), t.elementType) + + case t: MapType if !isPrimitiveCatalystType(t.valueType) => + // Only clips map types with nested type as value type. + clipParquetMapType(parquetType.asGroupType(), t.keyType, t.valueType) + + case t: StructType => + clipParquetGroup(parquetType.asGroupType(), t) + + case _ => + parquetType + } + } + + /** + * Whether a Catalyst [[DataType]] is primitive. Primitive [[DataType]] is not equivalent to + * [[AtomicType]]. For example, [[CalendarIntervalType]] is primitive, but it's not an + * [[AtomicType]]. + */ + private def isPrimitiveCatalystType(dataType: DataType): Boolean = { + dataType match { + case _: ArrayType | _: MapType | _: StructType => false + case _ => true + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[ArrayType]]. The element type + * of the [[ArrayType]] should also be a nested type, namely an [[ArrayType]], a [[MapType]], or a + * [[StructType]]. + */ + private def clipParquetListType(parquetList: GroupType, elementType: DataType): Type = { + // Precondition of this method, should only be called for lists with nested element types. + assert(!isPrimitiveCatalystType(elementType)) + + // Unannotated repeated group should be interpreted as required list of required element, so + // list element type is just the group itself. Clip it. + if (parquetList.getOriginalType == null && parquetList.isRepetition(Repetition.REPEATED)) { + clipParquetType(parquetList, elementType) + } else { + assert( + parquetList.getOriginalType == OriginalType.LIST, + "Invalid Parquet schema. " + + "Original type of annotated Parquet lists must be LIST: " + + parquetList.toString) + + assert( + parquetList.getFieldCount == 1 && parquetList.getType(0).isRepetition(Repetition.REPEATED), + "Invalid Parquet schema. " + + "LIST-annotated group should only have exactly one repeated field: " + + parquetList) + + // Precondition of this method, should only be called for lists with nested element types. + assert(!parquetList.getType(0).isPrimitive) + + val repeatedGroup = parquetList.getType(0).asGroupType() + + // If the repeated field is a group with multiple fields, or the repeated field is a group + // with one field and is named either "array" or uses the LIST-annotated group's name with + // "_tuple" appended then the repeated type is the element type and elements are required. + // Build a new LIST-annotated group with clipped `repeatedGroup` as element type and the + // only field. + if ( + repeatedGroup.getFieldCount > 1 || + repeatedGroup.getName == "array" || + repeatedGroup.getName == parquetList.getName + "_tuple" + ) { + Types + .buildGroup(parquetList.getRepetition) + .as(OriginalType.LIST) + .addField(clipParquetType(repeatedGroup, elementType)) + .named(parquetList.getName) + } else { + // Otherwise, the repeated field's type is the element type with the repeated field's + // repetition. + Types + .buildGroup(parquetList.getRepetition) + .as(OriginalType.LIST) + .addField( + Types + .repeatedGroup() + .addField(clipParquetType(repeatedGroup.getType(0), elementType)) + .named(repeatedGroup.getName)) + .named(parquetList.getName) + } + } + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[MapType]]. The value type + * of the [[MapType]] should also be a nested type, namely an [[ArrayType]], a [[MapType]], or a + * [[StructType]]. Note that key type of any [[MapType]] is always a primitive type. + */ + private def clipParquetMapType( + parquetMap: GroupType, keyType: DataType, valueType: DataType): GroupType = { + // Precondition of this method, should only be called for maps with nested value types. + assert(!isPrimitiveCatalystType(valueType)) + + val repeatedGroup = parquetMap.getType(0).asGroupType() + val parquetKeyType = repeatedGroup.getType(0) + val parquetValueType = repeatedGroup.getType(1) + + val clippedRepeatedGroup = + Types + .repeatedGroup() + .as(repeatedGroup.getOriginalType) + .addField(parquetKeyType) + .addField(clipParquetType(parquetValueType, valueType)) + .named(repeatedGroup.getName) + + Types + .buildGroup(parquetMap.getRepetition) + .as(parquetMap.getOriginalType) + .addField(clippedRepeatedGroup) + .named(parquetMap.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return A clipped [[GroupType]], which has at least one field. + * @note Parquet doesn't allow creating empty [[GroupType]] instances except for empty + * [[MessageType]]. Because it's legal to construct an empty requested schema for column + * pruning. + */ + private def clipParquetGroup(parquetRecord: GroupType, structType: StructType): GroupType = { + val clippedParquetFields = clipParquetGroupFields(parquetRecord, structType) + Types + .buildGroup(parquetRecord.getRepetition) + .as(parquetRecord.getOriginalType) + .addFields(clippedParquetFields: _*) + .named(parquetRecord.getName) + } + + /** + * Clips a Parquet [[GroupType]] which corresponds to a Catalyst [[StructType]]. + * + * @return A list of clipped [[GroupType]] fields, which can be empty. + */ + private def clipParquetGroupFields( + parquetRecord: GroupType, structType: StructType): Seq[Type] = { + val parquetFieldMap = parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap + val toParquet = new CatalystSchemaConverter(followParquetFormatSpec = true) + structType.map { f => + parquetFieldMap + .get(f.name) + .map(clipParquetType(_, f.dataType)) + .getOrElse(toParquet.convertField(f)) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala index fe13dfbbed385..f17e794b76650 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala @@ -113,31 +113,6 @@ private[parquet] class CatalystPrimitiveConverter(val updater: ParentContainerUp * When used as a root converter, [[NoopUpdater]] should be used since root converters don't have * any "parent" container. * - * @note Constructor argument [[parquetType]] refers to requested fields of the actual schema of the - * Parquet file being read, while constructor argument [[catalystType]] refers to requested - * fields of the global schema. The key difference is that, in case of schema merging, - * [[parquetType]] can be a subset of [[catalystType]]. For example, it's possible to have - * the following [[catalystType]]: - * {{{ - * new StructType() - * .add("f1", IntegerType, nullable = false) - * .add("f2", StringType, nullable = true) - * .add("f3", new StructType() - * .add("f31", DoubleType, nullable = false) - * .add("f32", IntegerType, nullable = true) - * .add("f33", StringType, nullable = true), nullable = false) - * }}} - * and the following [[parquetType]] (`f2` and `f32` are missing): - * {{{ - * message root { - * required int32 f1; - * required group f3 { - * required double f31; - * optional binary f33 (utf8); - * } - * } - * }}} - * * @param parquetType Parquet schema of Parquet records * @param catalystType Spark SQL schema that corresponds to the Parquet record type * @param updater An updater which propagates converted field values to the parent container @@ -179,31 +154,7 @@ private[parquet] class CatalystRowConverter( // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { - // In case of schema merging, `parquetType` can be a subset of `catalystType`. We need to pad - // those missing fields and create converters for them, although values of these fields are - // always null. - val paddedParquetFields = { - val parquetFields = parquetType.getFields.asScala - val parquetFieldNames = parquetFields.map(_.getName).toSet - val missingFields = catalystType.filterNot(f => parquetFieldNames.contains(f.name)) - - // We don't need to worry about feature flag arguments like `assumeBinaryIsString` when - // creating the schema converter here, since values of missing fields are always null. - val toParquet = new CatalystSchemaConverter() - - (parquetFields ++ missingFields.map(toParquet.convertField)).sortBy { f => - catalystType.indexWhere(_.name == f.getName) - } - } - - if (paddedParquetFields.length != catalystType.length) { - throw new UnsupportedOperationException( - "A Parquet file's schema has different number of fields with the table schema. " + - "Please enable schema merging by setting \"mergeSchema\" to true when load " + - "a Parquet dataset or set spark.sql.parquet.mergeSchema to true in SQLConf.") - } - - paddedParquetFields.zip(catalystType).zipWithIndex.map { + parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map { case ((parquetFieldType, catalystField), ordinal) => // Converted field value should be set to the `ordinal`-th cell of `currentRow` newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala index be6c0545f5a0a..a21ab1dbb25d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystSchemaConverter.scala @@ -55,16 +55,10 @@ import org.apache.spark.sql.{AnalysisException, SQLConf} * to old style non-standard behaviors. */ private[parquet] class CatalystSchemaConverter( - private val assumeBinaryIsString: Boolean, - private val assumeInt96IsTimestamp: Boolean, - private val followParquetFormatSpec: Boolean) { - - // Only used when constructing converter for converting Spark SQL schema to Parquet schema, in - // which case `assumeInt96IsTimestamp` and `assumeBinaryIsString` are irrelevant. - def this() = this( - assumeBinaryIsString = SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get, - assumeInt96IsTimestamp = SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get, - followParquetFormatSpec = SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.defaultValue.get) + assumeBinaryIsString: Boolean = SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get, + assumeInt96IsTimestamp: Boolean = SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get, + followParquetFormatSpec: Boolean = SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.defaultValue.get +) { def this(conf: SQLConf) = this( assumeBinaryIsString = conf.isParquetBinaryAsString, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala index bd7cf8c10abef..36b929ee1f409 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet +import java.io.File import java.nio.ByteBuffer import java.util.{List => JList, Map => JMap} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala new file mode 100644 index 0000000000000..83b65fb419ed3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -0,0 +1,90 @@ +/* + * 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.execution.datasources.parquet + +import java.io.File + +import org.apache.spark.sql.Row +import org.apache.spark.sql.test.SharedSQLContext + +class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSQLContext { + test("parquet files with different physical schemas but share the same logical schema") { + import ParquetCompatibilityTest._ + + // This test case writes two Parquet files, both representing the following Catalyst schema + // + // StructType( + // StructField( + // "f", + // ArrayType(IntegerType, containsNull = false), + // nullable = false)) + // + // The first Parquet file comes with parquet-avro style 2-level LIST-annotated group, while the + // other one comes with parquet-protobuf style 1-level unannotated primitive field. + withTempDir { dir => + val avroStylePath = new File(dir, "avro-style").getCanonicalPath + val protobufStylePath = new File(dir, "protobuf-style").getCanonicalPath + + val avroStyleSchema = + """message avro_style { + | required group f (LIST) { + | repeated int32 array; + | } + |} + """.stripMargin + + writeDirect(avroStylePath, avroStyleSchema, { rc => + rc.message { + rc.field("f", 0) { + rc.group { + rc.field("array", 0) { + rc.addInteger(0) + rc.addInteger(1) + } + } + } + } + }) + + logParquetSchema(avroStylePath) + + val protobufStyleSchema = + """message protobuf_style { + | repeated int32 f; + |} + """.stripMargin + + writeDirect(protobufStylePath, protobufStyleSchema, { rc => + rc.message { + rc.field("f", 0) { + rc.addInteger(2) + rc.addInteger(3) + } + } + }) + + logParquetSchema(protobufStylePath) + + checkAnswer( + sqlContext.read.parquet(dir.getCanonicalPath), + Seq( + Row(Seq(0, 1)), + Row(Seq(2, 3)))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index b7b70c2bbbd5c..a379523d67f80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -229,4 +229,81 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-10301 Clipping nested structs in requested schema") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = sqlContext + .range(1) + .selectExpr("NAMED_STRUCT('a', id, 'b', id) AS s") + .coalesce(1) + + df.write.mode("append").parquet(path) + + val userDefinedSchema = new StructType() + .add("s", new StructType().add("a", LongType, nullable = true), nullable = true) + + checkAnswer( + sqlContext.read.schema(userDefinedSchema).parquet(path), + Row(Row(0))) + } + + withTempPath { dir => + val path = dir.getCanonicalPath + + val df1 = sqlContext + .range(1) + .selectExpr("NAMED_STRUCT('a', id, 'b', id) AS s") + .coalesce(1) + + val df2 = sqlContext + .range(1, 2) + .selectExpr("NAMED_STRUCT('b', id, 'c', id) AS s") + .coalesce(1) + + df1.write.parquet(path) + df2.write.mode(SaveMode.Append).parquet(path) + + val userDefinedSchema = new StructType() + .add("s", + new StructType() + .add("a", LongType, nullable = true) + .add("c", LongType, nullable = true), + nullable = true) + + checkAnswer( + sqlContext.read.schema(userDefinedSchema).parquet(path), + Seq( + Row(Row(0, null)), + Row(Row(null, 1)))) + } + + withTempPath { dir => + val path = dir.getCanonicalPath + + val df = sqlContext + .range(1) + .selectExpr("NAMED_STRUCT('a', ARRAY(NAMED_STRUCT('b', id, 'c', id))) AS s") + .coalesce(1) + + df.write.parquet(path) + + val userDefinedSchema = new StructType() + .add("s", + new StructType() + .add( + "a", + ArrayType( + new StructType() + .add("b", LongType, nullable = true) + .add("d", StringType, nullable = true), + containsNull = true), + nullable = true), + nullable = true) + + checkAnswer( + sqlContext.read.schema(userDefinedSchema).parquet(path), + Row(Row(Seq(Row(0, null))))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 9dcbc1a047bea..28c59a4abdd76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -22,6 +22,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.parquet.schema.MessageTypeParser +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -941,4 +942,313 @@ class ParquetSchemaSuite extends ParquetSchemaTest { | optional fixed_len_byte_array(8) f1 (DECIMAL(18, 3)); |} """.stripMargin) + + private def testSchemaClipping( + testName: String, + parquetSchema: String, + catalystSchema: StructType, + expectedSchema: String): Unit = { + test(s"Clipping - $testName") { + val expected = MessageTypeParser.parseMessageType(expectedSchema) + val actual = CatalystReadSupport.clipParquetSchema( + MessageTypeParser.parseMessageType(parquetSchema), catalystSchema) + + try { + expected.checkContains(actual) + actual.checkContains(expected) + } catch { case cause: Throwable => + fail( + s"""Expected clipped schema: + |$expected + |Actual clipped schema: + |$actual + """.stripMargin, + cause) + } + } + } + + testSchemaClipping( + "simple nested struct", + + parquetSchema = + """message root { + | required group f0 { + | optional int32 f00; + | optional int32 f01; + | } + |} + """.stripMargin, + + catalystSchema = { + val f0Type = new StructType().add("f00", IntegerType, nullable = true) + new StructType() + .add("f0", f0Type, nullable = false) + .add("f1", IntegerType, nullable = true) + }, + + expectedSchema = + """message root { + | required group f0 { + | optional int32 f00; + | } + | optional int32 f1; + |} + """.stripMargin) + + testSchemaClipping( + "parquet-protobuf style array", + + parquetSchema = + """message root { + | required group f0 { + | repeated binary f00 (UTF8); + | repeated group f01 { + | optional int32 f010; + | optional double f011; + | } + | } + |} + """.stripMargin, + + catalystSchema = { + val f11Type = new StructType().add("f011", DoubleType, nullable = true) + val f01Type = ArrayType(StringType, containsNull = false) + val f0Type = new StructType() + .add("f00", f01Type, nullable = false) + .add("f01", f11Type, nullable = false) + val f1Type = ArrayType(IntegerType, containsNull = true) + new StructType() + .add("f0", f0Type, nullable = false) + .add("f1", f1Type, nullable = true) + }, + + expectedSchema = + """message root { + | required group f0 { + | repeated binary f00 (UTF8); + | repeated group f01 { + | optional double f011; + | } + | } + | + | optional group f1 (LIST) { + | repeated group list { + | optional int32 element; + | } + | } + |} + """.stripMargin) + + testSchemaClipping( + "parquet-thrift style array", + + parquetSchema = + """message root { + | required group f0 { + | optional group f00 { + | repeated binary f00_tuple (UTF8); + | } + | + | optional group f01 (LIST) { + | repeated group f01_tuple { + | optional int32 f010; + | optional double f011; + | } + | } + | } + |} + """.stripMargin, + + catalystSchema = { + val f11ElementType = new StructType() + .add("f011", DoubleType, nullable = true) + .add("f012", LongType, nullable = true) + + val f0Type = new StructType() + .add("f00", ArrayType(StringType, containsNull = false), nullable = false) + .add("f01", ArrayType(f11ElementType, containsNull = false), nullable = false) + + new StructType().add("f0", f0Type, nullable = false) + }, + + expectedSchema = + """message root { + | required group f0 { + | optional group f00 { + | repeated binary f00_tuple (UTF8); + | } + | + | optional group f01 (LIST) { + | repeated group f01_tuple { + | optional double f011; + | optional int64 f012; + | } + | } + | } + |} + """.stripMargin) + + testSchemaClipping( + "parquet-avro style array", + + parquetSchema = + """message root { + | required group f0 { + | optional group f00 { + | repeated binary array (UTF8); + | } + | + | optional group f01 (LIST) { + | repeated group array { + | optional int32 f010; + | optional double f011; + | } + | } + | } + |} + """.stripMargin, + + catalystSchema = { + val f11ElementType = new StructType() + .add("f011", DoubleType, nullable = true) + .add("f012", LongType, nullable = true) + + val f0Type = new StructType() + .add("f00", ArrayType(StringType, containsNull = false), nullable = false) + .add("f01", ArrayType(f11ElementType, containsNull = false), nullable = false) + + new StructType().add("f0", f0Type, nullable = false) + }, + + expectedSchema = + """message root { + | required group f0 { + | optional group f00 { + | repeated binary array (UTF8); + | } + | + | optional group f01 (LIST) { + | repeated group array { + | optional double f011; + | optional int64 f012; + | } + | } + | } + |} + """.stripMargin) + + testSchemaClipping( + "parquet-hive style array", + + parquetSchema = + """message root { + | optional group f0 { + | optional group f00 (LIST) { + | repeated group bag { + | optional binary array_element; + | } + | } + | + | optional group f01 (LIST) { + | repeated group bag { + | optional group array_element { + | optional int32 f010; + | optional double f011; + | } + | } + | } + | } + |} + """.stripMargin, + + catalystSchema = { + val f01ElementType = new StructType() + .add("f011", DoubleType, nullable = true) + .add("f012", LongType, nullable = true) + + val f0Type = new StructType() + .add("f00", ArrayType(StringType, containsNull = true), nullable = true) + .add("f01", ArrayType(f01ElementType, containsNull = true), nullable = true) + + new StructType().add("f0", f0Type, nullable = true) + }, + + expectedSchema = + """message root { + | optional group f0 { + | optional group f00 (LIST) { + | repeated group bag { + | optional binary array_element; + | } + | } + | + | optional group f01 (LIST) { + | repeated group bag { + | optional group array_element { + | optional double f011; + | optional int64 f012; + | } + | } + | } + | } + |} + """.stripMargin) + + testSchemaClipping( + "2-level list of required struct", + + parquetSchema = + s"""message root { + | required group f0 { + | required group f00 (LIST) { + | repeated group element { + | required int32 f000; + | optional int64 f001; + | } + | } + | } + |} + """.stripMargin, + + catalystSchema = { + val f00ElementType = + new StructType() + .add("f001", LongType, nullable = true) + .add("f002", DoubleType, nullable = false) + + val f00Type = ArrayType(f00ElementType, containsNull = false) + val f0Type = new StructType().add("f00", f00Type, nullable = false) + + new StructType().add("f0", f0Type, nullable = false) + }, + + expectedSchema = + s"""message root { + | required group f0 { + | required group f00 (LIST) { + | repeated group element { + | optional int64 f001; + | required double f002; + | } + | } + | } + |} + """.stripMargin) + + testSchemaClipping( + "empty requested schema", + + parquetSchema = + """message root { + | required group f0 { + | required int32 f00; + | required int64 f01; + | } + |} + """.stripMargin, + + catalystSchema = new StructType(), + + expectedSchema = "message root {}") } From e6e483cc4de740c46398385b03ffe0e662edae39 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 1 Sep 2015 10:48:57 -0700 Subject: [PATCH 170/232] [SPARK-9679] [ML] [PYSPARK] Add Python API for Stop Words Remover Add a python API for the Stop Words Remover. Author: Holden Karau Closes #8118 from holdenk/SPARK-9679-python-StopWordsRemover. --- .../spark/ml/feature/StopWordsRemover.scala | 6 +- .../ml/feature/StopWordsRemoverSuite.scala | 2 +- python/pyspark/ml/feature.py | 73 ++++++++++++++++++- python/pyspark/ml/tests.py | 20 ++++- 4 files changed, 93 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala index 5d77ea08db657..7da430c7d16df 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StopWordsRemover.scala @@ -29,14 +29,14 @@ import org.apache.spark.sql.types.{ArrayType, StringType, StructField, StructTyp /** * stop words list */ -private object StopWords { +private[spark] object StopWords { /** * Use the same default stopwords list as scikit-learn. * The original list can be found from "Glasgow Information Retrieval Group" * [[http://ir.dcs.gla.ac.uk/resources/linguistic_utils/stop_words]] */ - val EnglishStopWords = Array( "a", "about", "above", "across", "after", "afterwards", "again", + val English = Array( "a", "about", "above", "across", "after", "afterwards", "again", "against", "all", "almost", "alone", "along", "already", "also", "although", "always", "am", "among", "amongst", "amoungst", "amount", "an", "and", "another", "any", "anyhow", "anyone", "anything", "anyway", "anywhere", "are", @@ -121,7 +121,7 @@ class StopWordsRemover(override val uid: String) /** @group getParam */ def getCaseSensitive: Boolean = $(caseSensitive) - setDefault(stopWords -> StopWords.EnglishStopWords, caseSensitive -> false) + setDefault(stopWords -> StopWords.English, caseSensitive -> false) override def transform(dataset: DataFrame): DataFrame = { val outputSchema = transformSchema(dataset.schema) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala index f01306f89cb5f..e0d433f566c25 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StopWordsRemoverSuite.scala @@ -65,7 +65,7 @@ class StopWordsRemoverSuite extends SparkFunSuite with MLlibTestSparkContext { } test("StopWordsRemover with additional words") { - val stopWords = StopWords.EnglishStopWords ++ Array("python", "scala") + val stopWords = StopWords.English ++ Array("python", "scala") val remover = new StopWordsRemover() .setInputCol("raw") .setOutputCol("filtered") diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 0626281e200a1..d955307e27efd 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -22,7 +22,7 @@ from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import * from pyspark.ml.util import keyword_only -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaTransformer, _jvm from pyspark.mllib.common import inherit_doc from pyspark.mllib.linalg import _convert_to_vector @@ -30,7 +30,7 @@ 'NGram', 'Normalizer', 'OneHotEncoder', 'PolynomialExpansion', 'RegexTokenizer', 'SQLTransformer', 'StandardScaler', 'StandardScalerModel', 'StringIndexer', 'StringIndexerModel', 'Tokenizer', 'VectorAssembler', 'VectorIndexer', 'Word2Vec', - 'Word2VecModel', 'PCA', 'PCAModel', 'RFormula', 'RFormulaModel'] + 'Word2VecModel', 'PCA', 'PCAModel', 'RFormula', 'RFormulaModel', 'StopWordsRemover'] @inherit_doc @@ -933,6 +933,75 @@ class StringIndexerModel(JavaModel): """ +class StopWordsRemover(JavaTransformer, HasInputCol, HasOutputCol): + """ + .. note:: Experimental + + A feature transformer that filters out stop words from input. + Note: null values from input array are preserved unless adding null to stopWords explicitly. + """ + # a placeholder to make the stopwords show up in generated doc + stopWords = Param(Params._dummy(), "stopWords", "The words to be filtered out") + caseSensitive = Param(Params._dummy(), "caseSensitive", "whether to do a case sensitive " + + "comparison over the stop words") + + @keyword_only + def __init__(self, inputCol=None, outputCol=None, stopWords=None, + caseSensitive=False): + """ + __init__(self, inputCol=None, outputCol=None, stopWords=None,\ + caseSensitive=false) + """ + super(StopWordsRemover, self).__init__() + self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.StopWordsRemover", + self.uid) + self.stopWords = Param(self, "stopWords", "The words to be filtered out") + self.caseSensitive = Param(self, "caseSensitive", "whether to do a case " + + "sensitive comparison over the stop words") + stopWordsObj = _jvm().org.apache.spark.ml.feature.StopWords + defaultStopWords = stopWordsObj.English() + self._setDefault(stopWords=defaultStopWords) + kwargs = self.__init__._input_kwargs + self.setParams(**kwargs) + + @keyword_only + def setParams(self, inputCol=None, outputCol=None, stopWords=None, + caseSensitive=False): + """ + setParams(self, inputCol="input", outputCol="output", stopWords=None,\ + caseSensitive=false) + Sets params for this StopWordRemover. + """ + kwargs = self.setParams._input_kwargs + return self._set(**kwargs) + + def setStopWords(self, value): + """ + Specify the stopwords to be filtered. + """ + self._paramMap[self.stopWords] = value + return self + + def getStopWords(self): + """ + Get the stopwords. + """ + return self.getOrDefault(self.stopWords) + + def setCaseSensitive(self, value): + """ + Set whether to do a case sensitive comparison over the stop words + """ + self._paramMap[self.caseSensitive] = value + return self + + def getCaseSensitive(self): + """ + Get whether to do a case sensitive comparison over the stop words. + """ + return self.getOrDefault(self.caseSensitive) + + @inherit_doc @ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 60e4237293adc..b892318f50bd9 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -31,7 +31,7 @@ import unittest from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase -from pyspark.sql import DataFrame, SQLContext +from pyspark.sql import DataFrame, SQLContext, Row from pyspark.sql.functions import rand from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.param import Param, Params @@ -258,7 +258,7 @@ def test_idf(self): def test_ngram(self): sqlContext = SQLContext(self.sc) dataset = sqlContext.createDataFrame([ - ([["a", "b", "c", "d", "e"]])], ["input"]) + Row(input=["a", "b", "c", "d", "e"])]) ngram0 = NGram(n=4, inputCol="input", outputCol="output") self.assertEqual(ngram0.getN(), 4) self.assertEqual(ngram0.getInputCol(), "input") @@ -266,6 +266,22 @@ def test_ngram(self): transformedDF = ngram0.transform(dataset) self.assertEquals(transformedDF.head().output, ["a b c d", "b c d e"]) + def test_stopwordsremover(self): + sqlContext = SQLContext(self.sc) + dataset = sqlContext.createDataFrame([Row(input=["a", "panda"])]) + stopWordRemover = StopWordsRemover(inputCol="input", outputCol="output") + # Default + self.assertEquals(stopWordRemover.getInputCol(), "input") + transformedDF = stopWordRemover.transform(dataset) + self.assertEquals(transformedDF.head().output, ["panda"]) + # Custom + stopwords = ["panda"] + stopWordRemover.setStopWords(stopwords) + self.assertEquals(stopWordRemover.getInputCol(), "input") + self.assertEquals(stopWordRemover.getStopWords(), stopwords) + transformedDF = stopWordRemover.transform(dataset) + self.assertEquals(transformedDF.head().output, ["a"]) + class HasInducedError(Params): From 3f63bd6023edcc9af268933a235f34e10bc3d2ba Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 1 Sep 2015 20:06:01 +0100 Subject: [PATCH 171/232] [SPARK-10398] [DOCS] Migrate Spark download page to use new lua mirroring scripts Migrate Apache download closer.cgi refs to new closer.lua This is the bit of the change that affects the project docs; I'm implementing the changes to the Apache site separately. Author: Sean Owen Closes #8557 from srowen/SPARK-10398. --- docker/spark-mesos/Dockerfile | 2 +- docs/running-on-mesos.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/spark-mesos/Dockerfile b/docker/spark-mesos/Dockerfile index b90aef3655dee..fb3f267fe5c78 100644 --- a/docker/spark-mesos/Dockerfile +++ b/docker/spark-mesos/Dockerfile @@ -24,7 +24,7 @@ RUN apt-get update && \ apt-get install -y python libnss3 openjdk-7-jre-headless curl RUN mkdir /opt/spark && \ - curl http://www.apache.org/dyn/closer.cgi/spark/spark-1.4.0/spark-1.4.0-bin-hadoop2.4.tgz \ + curl http://www.apache.org/dyn/closer.lua/spark/spark-1.4.0/spark-1.4.0-bin-hadoop2.4.tgz \ | tar -xzC /opt ENV SPARK_HOME /opt/spark ENV MESOS_NATIVE_JAVA_LIBRARY /usr/local/lib/libmesos.so diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index cfd219ab02e26..f36921ae30c2f 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -45,7 +45,7 @@ frameworks. You can install Mesos either from source or using prebuilt packages To install Apache Mesos from source, follow these steps: 1. Download a Mesos release from a - [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/) + [mirror](http://www.apache.org/dyn/closer.lua/mesos/{{site.MESOS_VERSION}}/) 2. Follow the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for compiling and installing Mesos From ec012805337926e56343be2761a1037296446880 Mon Sep 17 00:00:00 2001 From: zhuol Date: Tue, 1 Sep 2015 11:14:59 -1000 Subject: [PATCH 172/232] [SPARK-4223] [CORE] Support * in acls. SPARK-4223. Currently we support setting view and modify acls but you have to specify a list of users. It would be nice to support * meaning all users have access. Manual tests to verify that: "*" works for any user in: a. Spark ui: view and kill stage. Done. b. Spark history server. Done. c. Yarn application killing. Done. Author: zhuol Closes #8398 from zhuoliu/4223. --- .../org/apache/spark/SecurityManager.scala | 26 ++++++++++-- .../apache/spark/SecurityManagerSuite.scala | 41 +++++++++++++++++++ docs/configuration.md | 9 ++-- 3 files changed, 69 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 673ef49e7c1c5..746d2081d4393 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -310,7 +310,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) setViewAcls(Set[String](defaultUser), allowedUsers) } - def getViewAcls: String = viewAcls.mkString(",") + /** + * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" + */ + def getViewAcls: String = { + if (viewAcls.contains("*")) { + "*" + } else { + viewAcls.mkString(",") + } + } /** * Admin acls should be set before the view or modify acls. If you modify the admin @@ -321,7 +330,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } - def getModifyAcls: String = modifyAcls.mkString(",") + /** + * Checking the existence of "*" is necessary as YARN can't recognize the "*" in "defaultuser,*" + */ + def getModifyAcls: String = { + if (modifyAcls.contains("*")) { + "*" + } else { + modifyAcls.mkString(",") + } + } /** * Admin acls should be set before the view or modify acls. If you modify the admin @@ -394,7 +412,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) def checkUIViewPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + viewAcls.mkString(",")) - !aclsEnabled || user == null || viewAcls.contains(user) + !aclsEnabled || user == null || viewAcls.contains(user) || viewAcls.contains("*") } /** @@ -409,7 +427,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) def checkModifyPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + modifyAcls.mkString(",")) - !aclsEnabled || user == null || modifyAcls.contains(user) + !aclsEnabled || user == null || modifyAcls.contains(user) || modifyAcls.contains("*") } diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index f34aefca4eb18..f29160d834082 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -125,6 +125,47 @@ class SecurityManagerSuite extends SparkFunSuite { } + test("set security with * in acls") { + val conf = new SparkConf + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.admin.acls", "user1,user2") + conf.set("spark.ui.view.acls", "*") + conf.set("spark.modify.acls", "user4") + + val securityManager = new SecurityManager(conf) + assert(securityManager.aclsEnabled() === true) + + // check for viewAcls with * + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkModifyPermissions("user7") === false) + assert(securityManager.checkModifyPermissions("user8") === false) + + // check for modifyAcls with * + securityManager.setModifyAcls(Set("user4"), "*") + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + + securityManager.setAdminAcls("user1,user2") + securityManager.setModifyAcls(Set("user1"), "user2") + securityManager.setViewAcls(Set("user1"), "user2") + assert(securityManager.checkUIViewPermissions("user5") === false) + assert(securityManager.checkUIViewPermissions("user6") === false) + assert(securityManager.checkModifyPermissions("user7") === false) + assert(securityManager.checkModifyPermissions("user8") === false) + + // check for adminAcls with * + securityManager.setAdminAcls("user1,*") + securityManager.setModifyAcls(Set("user1"), "user2") + securityManager.setViewAcls(Set("user1"), "user2") + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === true) + } + test("ssl on setup") { val conf = SSLSampleConfigs.sparkSSLConfig() val expectedAlgorithms = Set( diff --git a/docs/configuration.md b/docs/configuration.md index 77c5cbc7b3196..fb0315ce7c3cc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1286,7 +1286,8 @@ Apart from these, the following properties are also available, and may be useful Comma separated list of users/administrators that have view and modify access to all Spark jobs. This can be used if you run on a shared cluster and have a set of administrators or devs who - help debug when things work. + help debug when things work. Putting a "*" in the list means any user can have the priviledge + of admin. @@ -1327,7 +1328,8 @@ Apart from these, the following properties are also available, and may be useful Empty Comma separated list of users that have modify access to the Spark job. By default only the - user that started the Spark job has access to modify it (kill it for example). + user that started the Spark job has access to modify it (kill it for example). Putting a "*" in + the list means any user can have access to modify it. @@ -1349,7 +1351,8 @@ Apart from these, the following properties are also available, and may be useful Empty Comma separated list of users that have view access to the Spark web ui. By default only the - user that started the Spark job has view access. + user that started the Spark job has view access. Putting a "*" in the list means any user can + have view access to this Spark job. From bf550a4b551b6dd18fea3eb3f70497f9a6ad8e6c Mon Sep 17 00:00:00 2001 From: 0x0FFF Date: Tue, 1 Sep 2015 14:34:59 -0700 Subject: [PATCH 173/232] [SPARK-10162] [SQL] Fix the timezone omitting for PySpark Dataframe filter function This PR addresses [SPARK-10162](https://issues.apache.org/jira/browse/SPARK-10162) The issue is with DataFrame filter() function, if datetime.datetime is passed to it: * Timezone information of this datetime is ignored * This datetime is assumed to be in local timezone, which depends on the OS timezone setting Fix includes both code change and regression test. Problem reproduction code on master: ```python import pytz from datetime import datetime from pyspark.sql import * from pyspark.sql.types import * sqc = SQLContext(sc) df = sqc.createDataFrame([], StructType([StructField("dt", TimestampType())])) m1 = pytz.timezone('UTC') m2 = pytz.timezone('Etc/GMT+3') df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m1)).explain() df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m2)).explain() ``` It gives the same timestamp ignoring time zone: ``` >>> df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m1)).explain() Filter (dt#0 > 946713600000000) Scan PhysicalRDD[dt#0] >>> df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m2)).explain() Filter (dt#0 > 946713600000000) Scan PhysicalRDD[dt#0] ``` After the fix: ``` >>> df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m1)).explain() Filter (dt#0 > 946684800000000) Scan PhysicalRDD[dt#0] >>> df.filter(df.dt > datetime(2000, 01, 01, tzinfo=m2)).explain() Filter (dt#0 > 946695600000000) Scan PhysicalRDD[dt#0] ``` PR [8536](https://github.com/apache/spark/pull/8536) was occasionally closed by me dropping the repo Author: 0x0FFF Closes #8555 from 0x0FFF/SPARK-10162. --- python/pyspark/sql/tests.py | 26 ++++++++++++++++++-------- python/pyspark/sql/types.py | 7 +++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index cd32e26c64f22..59a891bd7c420 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -50,16 +50,17 @@ from pyspark.sql.utils import AnalysisException, IllegalArgumentException -class UTC(datetime.tzinfo): - """UTC""" - ZERO = datetime.timedelta(0) +class UTCOffsetTimezone(datetime.tzinfo): + """ + Specifies timezone in UTC offset + """ + + def __init__(self, offset=0): + self.ZERO = datetime.timedelta(hours=offset) def utcoffset(self, dt): return self.ZERO - def tzname(self, dt): - return "UTC" - def dst(self, dt): return self.ZERO @@ -841,13 +842,22 @@ def test_filter_with_datetime(self): self.assertEqual(0, df.filter(df.date > date).count()) self.assertEqual(0, df.filter(df.time > time).count()) + def test_filter_with_datetime_timezone(self): + dt1 = datetime.datetime(2015, 4, 17, 23, 1, 2, 3000, tzinfo=UTCOffsetTimezone(0)) + dt2 = datetime.datetime(2015, 4, 17, 23, 1, 2, 3000, tzinfo=UTCOffsetTimezone(1)) + row = Row(date=dt1) + df = self.sqlCtx.createDataFrame([row]) + self.assertEqual(0, df.filter(df.date == dt2).count()) + self.assertEqual(1, df.filter(df.date > dt2).count()) + self.assertEqual(0, df.filter(df.date < dt2).count()) + def test_time_with_timezone(self): day = datetime.date.today() now = datetime.datetime.now() ts = time.mktime(now.timetuple()) # class in __main__ is not serializable - from pyspark.sql.tests import UTC - utc = UTC() + from pyspark.sql.tests import UTCOffsetTimezone + utc = UTCOffsetTimezone() utcnow = datetime.datetime.utcfromtimestamp(ts) # without microseconds # add microseconds to utcnow (keeping year,month,day,hour,minute,second) utcnow = datetime.datetime(*(utcnow.timetuple()[:6] + (now.microsecond, utc))) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 94e581a78364c..f84d08d7098ad 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1290,8 +1290,11 @@ def can_convert(self, obj): def convert(self, obj, gateway_client): Timestamp = JavaClass("java.sql.Timestamp", gateway_client) - return Timestamp(int(time.mktime(obj.timetuple())) * 1000 + obj.microsecond // 1000) - + seconds = (calendar.timegm(obj.utctimetuple()) if obj.tzinfo + else time.mktime(obj.timetuple())) + t = Timestamp(int(seconds) * 1000) + t.setNanos(obj.microsecond * 1000) + return t # datetime is a subclass of date, we should register DatetimeConverter first register_input_converter(DatetimeConverter()) From 00d9af5e190475affffb8b50467fcddfc40f50dc Mon Sep 17 00:00:00 2001 From: 0x0FFF Date: Tue, 1 Sep 2015 14:58:49 -0700 Subject: [PATCH 174/232] [SPARK-10392] [SQL] Pyspark - Wrong DateType support on JDBC connection This PR addresses issue [SPARK-10392](https://issues.apache.org/jira/browse/SPARK-10392) The problem is that for "start of epoch" date (01 Jan 1970) PySpark class DateType returns 0 instead of the `datetime.date` due to implementation of its return statement Issue reproduction on master: ``` >>> from pyspark.sql.types import * >>> a = DateType() >>> a.fromInternal(0) 0 >>> a.fromInternal(1) datetime.date(1970, 1, 2) ``` Author: 0x0FFF Closes #8556 from 0x0FFF/SPARK-10392. --- python/pyspark/sql/tests.py | 5 +++++ python/pyspark/sql/types.py | 6 ++++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 59a891bd7c420..fc778631d93a3 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -168,6 +168,11 @@ def test_decimal_type(self): t3 = DecimalType(8) self.assertNotEqual(t2, t3) + # regression test for SPARK-10392 + def test_datetype_equal_zero(self): + dt = DateType() + self.assertEqual(dt.fromInternal(0), datetime.date(1970, 1, 1)) + class SQLTests(ReusedPySparkTestCase): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index f84d08d7098ad..8bd58d69eeecd 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -168,10 +168,12 @@ def needConversion(self): return True def toInternal(self, d): - return d and d.toordinal() - self.EPOCH_ORDINAL + if d is not None: + return d.toordinal() - self.EPOCH_ORDINAL def fromInternal(self, v): - return v and datetime.date.fromordinal(v + self.EPOCH_ORDINAL) + if v is not None: + return datetime.date.fromordinal(v + self.EPOCH_ORDINAL) class TimestampType(AtomicType): From c3b881a7d7e4736f7131ff002a80e25def1f63af Mon Sep 17 00:00:00 2001 From: Chuan Shao Date: Wed, 2 Sep 2015 11:02:27 -0700 Subject: [PATCH 175/232] [SPARK-7336] [HISTORYSERVER] Fix bug that applications status incorrect on JobHistory UI. Author: ArcherShao Closes #5886 from ArcherShao/SPARK-7336. --- .../deploy/history/FsHistoryProvider.scala | 27 +++++++++++++++---- 1 file changed, 22 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e573ff16c50a3..a5755eac36396 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.history import java.io.{BufferedInputStream, FileNotFoundException, InputStream, IOException, OutputStream} +import java.util.UUID import java.util.concurrent.{ExecutorService, Executors, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} @@ -73,7 +74,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // The modification time of the newest log detected during the last scan. This is used // to ignore logs that are older during subsequent scans, to avoid processing data that // is already known. - private var lastModifiedTime = -1L + private var lastScanTime = -1L // Mapping of application IDs to their metadata, in descending end time order. Apps are inserted // into the map in order, so the LinkedHashMap maintains the correct ordering. @@ -179,15 +180,14 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private[history] def checkForLogs(): Unit = { try { + val newLastScanTime = getNewLastScanTime() val statusList = Option(fs.listStatus(new Path(logDir))).map(_.toSeq) .getOrElse(Seq[FileStatus]()) - var newLastModifiedTime = lastModifiedTime val logInfos: Seq[FileStatus] = statusList .filter { entry => try { getModificationTime(entry).map { time => - newLastModifiedTime = math.max(newLastModifiedTime, time) - time >= lastModifiedTime + time >= lastScanTime }.getOrElse(false) } catch { case e: AccessControlException => @@ -224,12 +224,29 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - lastModifiedTime = newLastModifiedTime + lastScanTime = newLastScanTime } catch { case e: Exception => logError("Exception in checking for event log updates", e) } } + private def getNewLastScanTime(): Long = { + val fileName = "." + UUID.randomUUID().toString + val path = new Path(logDir, fileName) + val fos = fs.create(path) + + try { + fos.close() + fs.getFileStatus(path).getModificationTime + } catch { + case e: Exception => + logError("Exception encountered when attempting to update last scan time", e) + lastScanTime + } finally { + fs.delete(path) + } + } + override def writeEventLogs( appId: String, attemptId: Option[String], From 56c4c172e99a5e14f4bc3308e7ff36d94113b63e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 2 Sep 2015 11:13:17 -0700 Subject: [PATCH 176/232] [SPARK-10034] [SQL] add regression test for Sort on Aggregate Before #8371, there was a bug for `Sort` on `Aggregate` that we can't use aggregate expressions named `_aggOrdering` and can't use more than one ordering expressions which contains aggregate functions. The reason of this bug is that: The aggregate expression in `SortOrder` never get resolved, we alias it with `_aggOrdering` and call `toAttribute` which gives us an `UnresolvedAttribute`. So actually we are referencing aggregate expression by name, not by exprId like we thought. And if there is already an aggregate expression named `_aggOrdering` or there are more than one ordering expressions having aggregate functions, we will have conflict names and can't search by name. However, after #8371 got merged, the `SortOrder`s are guaranteed to be resolved and we are always referencing aggregate expression by exprId. The Bug doesn't exist anymore and this PR add regression tests for it. Author: Wenchen Fan Closes #8231 from cloud-fan/sort-agg. --- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 284fff184085a..a4871e247cff7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -887,4 +887,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { .select(struct($"b")) .collect() } + + test("SPARK-10034: Sort on Aggregate with aggregation expression named 'aggOrdering'") { + val df = Seq(1 -> 2).toDF("i", "j") + val query = df.groupBy('i) + .agg(max('j).as("aggOrdering")) + .orderBy(sum('j)) + checkAnswer(query, Row(1, 2)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 9e172b2c264cb..28201073a2d7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1490,6 +1490,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { """.stripMargin), Row(3) :: Row(7) :: Row(11) :: Row(15) :: Nil) + checkAnswer( + sql( + """ + |SELECT sum(b) + |FROM orderByData + |GROUP BY a + |ORDER BY sum(b), max(b) + """.stripMargin), + Row(3) :: Row(7) :: Row(11) :: Row(15) :: Nil) + checkAnswer( sql( """ From fc48307797912dc1d53893dce741ddda8630957b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 2 Sep 2015 11:32:27 -0700 Subject: [PATCH 177/232] [SPARK-10389] [SQL] support order by non-attribute grouping expression on Aggregate For example, we can write `SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1` in PostgreSQL, and we should support this in Spark SQL. Author: Wenchen Fan Closes #8548 from cloud-fan/support-order-by-non-attribute. --- .../sql/catalyst/analysis/Analyzer.scala | 72 ++++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 19 +++-- 2 files changed, 52 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1a5de15c61f86..591747b45c376 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -560,43 +560,47 @@ class Analyzer( filter } - case sort @ Sort(sortOrder, global, - aggregate @ Aggregate(grouping, originalAggExprs, child)) + case sort @ Sort(sortOrder, global, aggregate: Aggregate) if aggregate.resolved && !sort.resolved => // Try resolving the ordering as though it is in the aggregate clause. try { - val aliasedOrder = sortOrder.map(o => Alias(o.child, "aggOrder")()) - val aggregatedOrdering = Aggregate(grouping, aliasedOrder, child) - val resolvedOperator: Aggregate = execute(aggregatedOrdering).asInstanceOf[Aggregate] - def resolvedAggregateOrdering = resolvedOperator.aggregateExpressions - - // Expressions that have an aggregate can be pushed down. - val needsAggregate = resolvedAggregateOrdering.exists(containsAggregate) - - // Attribute references, that are missing from the order but are present in the grouping - // expressions can also be pushed down. - val requiredAttributes = resolvedAggregateOrdering.map(_.references).reduce(_ ++ _) - val missingAttributes = requiredAttributes -- aggregate.outputSet - val validPushdownAttributes = - missingAttributes.filter(a => grouping.exists(a.semanticEquals)) - - // If resolution was successful and we see the ordering either has an aggregate in it or - // it is missing something that is projected away by the aggregate, add the ordering - // the original aggregate operator. - if (resolvedOperator.resolved && (needsAggregate || validPushdownAttributes.nonEmpty)) { - val evaluatedOrderings: Seq[SortOrder] = sortOrder.zip(resolvedAggregateOrdering).map { - case (order, evaluated) => order.copy(child = evaluated.toAttribute) - } - val aggExprsWithOrdering: Seq[NamedExpression] = - resolvedAggregateOrdering ++ originalAggExprs - - Project(aggregate.output, - Sort(evaluatedOrderings, global, - aggregate.copy(aggregateExpressions = aggExprsWithOrdering))) - } else { - sort + val aliasedOrdering = sortOrder.map(o => Alias(o.child, "aggOrder")()) + val aggregatedOrdering = aggregate.copy(aggregateExpressions = aliasedOrdering) + val resolvedAggregate: Aggregate = execute(aggregatedOrdering).asInstanceOf[Aggregate] + val resolvedAliasedOrdering: Seq[Alias] = + resolvedAggregate.aggregateExpressions.asInstanceOf[Seq[Alias]] + + // If we pass the analysis check, then the ordering expressions should only reference to + // aggregate expressions or grouping expressions, and it's safe to push them down to + // Aggregate. + checkAnalysis(resolvedAggregate) + + val originalAggExprs = aggregate.aggregateExpressions.map( + CleanupAliases.trimNonTopLevelAliases(_).asInstanceOf[NamedExpression]) + + // If the ordering expression is same with original aggregate expression, we don't need + // to push down this ordering expression and can reference the original aggregate + // expression instead. + val needsPushDown = ArrayBuffer.empty[NamedExpression] + val evaluatedOrderings = resolvedAliasedOrdering.zip(sortOrder).map { + case (evaluated, order) => + val index = originalAggExprs.indexWhere { + case Alias(child, _) => child semanticEquals evaluated.child + case other => other semanticEquals evaluated.child + } + + if (index == -1) { + needsPushDown += evaluated + order.copy(child = evaluated.toAttribute) + } else { + order.copy(child = originalAggExprs(index).toAttribute) + } } + + Project(aggregate.output, + Sort(evaluatedOrderings, global, + aggregate.copy(aggregateExpressions = originalAggExprs ++ needsPushDown))) } catch { // Attempting to resolve in the aggregate can result in ambiguity. When this happens, // just return the original plan. @@ -605,9 +609,7 @@ class Analyzer( } protected def containsAggregate(condition: Expression): Boolean = { - condition - .collect { case ae: AggregateExpression => ae } - .nonEmpty + condition.find(_.isInstanceOf[AggregateExpression]).isDefined } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 28201073a2d7b..0ef25fe0faef0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1722,9 +1722,20 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-10130 type coercion for IF should have children resolved first") { - val df = Seq((1, 1), (-1, 1)).toDF("key", "value") - df.registerTempTable("src") - checkAnswer( - sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) + withTempTable("src") { + Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + checkAnswer( + sql("SELECT IF(a > 0, a, 0) FROM (SELECT key a FROM src) temp"), Seq(Row(1), Row(0))) + } + } + + test("SPARK-10389: order by non-attribute grouping expression on Aggregate") { + withTempTable("src") { + Seq((1, 1), (-1, 1)).toDF("key", "value").registerTempTable("src") + checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1"), + Seq(Row(1), Row(1))) + checkAnswer(sql("SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY (key + 1) * 2"), + Seq(Row(1), Row(1))) + } } } From 2da3a9e98e5d129d4507b5db01bba5ee9558d28e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 2 Sep 2015 12:53:24 -0700 Subject: [PATCH 178/232] [SPARK-10004] [SHUFFLE] Perform auth checks when clients read shuffle data. To correctly isolate applications, when requests to read shuffle data arrive at the shuffle service, proper authorization checks need to be performed. This change makes sure that only the application that created the shuffle data can read from it. Such checks are only enabled when "spark.authenticate" is enabled, otherwise there's no secure way to make sure that the client is really who it says it is. Author: Marcelo Vanzin Closes #8218 from vanzin/SPARK-10004. --- .../network/netty/NettyBlockRpcServer.scala | 3 +- .../netty/NettyBlockTransferService.scala | 2 +- network/common/pom.xml | 4 + .../spark/network/client/TransportClient.java | 22 +++ .../network/sasl/SaslClientBootstrap.java | 2 + .../spark/network/sasl/SaslRpcHandler.java | 1 + .../server/OneForOneStreamManager.java | 31 +++- .../spark/network/server/StreamManager.java | 9 + .../server/TransportRequestHandler.java | 1 + .../shuffle/ExternalShuffleBlockHandler.java | 16 +- .../network/sasl/SaslIntegrationSuite.java | 163 +++++++++++++++--- .../ExternalShuffleBlockHandlerSuite.java | 2 +- project/MimaExcludes.scala | 1 + 13 files changed, 221 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 7c170a742fb64..76968249fb625 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -38,6 +38,7 @@ import org.apache.spark.storage.{BlockId, StorageLevel} * is equivalent to one Spark-level shuffle block. */ class NettyBlockRpcServer( + appId: String, serializer: Serializer, blockManager: BlockDataManager) extends RpcHandler with Logging { @@ -55,7 +56,7 @@ class NettyBlockRpcServer( case openBlocks: OpenBlocks => val blocks: Seq[ManagedBuffer] = openBlocks.blockIds.map(BlockId.apply).map(blockManager.getBlockData) - val streamId = streamManager.registerStream(blocks.iterator.asJava) + val streamId = streamManager.registerStream(appId, blocks.iterator.asJava) logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") responseContext.onSuccess(new StreamHandle(streamId, blocks.size).toByteArray) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index ff8aae9ebe9f0..d5ad2c9ad00e8 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -49,7 +49,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage private[this] var appId: String = _ override def init(blockDataManager: BlockDataManager): Unit = { - val rpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) + val rpcHandler = new NettyBlockRpcServer(conf.getAppId, serializer, blockDataManager) var serverBootstrap: Option[TransportServerBootstrap] = None var clientBootstrap: Option[TransportClientBootstrap] = None if (authEnabled) { diff --git a/network/common/pom.xml b/network/common/pom.xml index 7dc3068ab8cb7..4141fcb8267a5 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -48,6 +48,10 @@ slf4j-api provided
+ + com.google.code.findbugs + jsr305 +