From 578d167bfccdc2d1d5ce9ca06cab7b7b753bb3eb Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 3 Dec 2014 01:33:44 +0800 Subject: [PATCH 01/24] make caseSensitive configurable --- .../src/main/scala/org/apache/spark/sql/SQLConf.scala | 8 ++++++++ .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) 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 9697beb132fb..caf5fa580394 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 @@ -46,6 +46,8 @@ private[spark] object SQLConf { // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" + val CASE_SENSITIVE = "spark.sql.caseSensitive" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -148,6 +150,12 @@ private[sql] trait SQLConf { private[spark] def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD, "_corrupt_record") + /** + * When set to true, analyzer is case sensitive + */ + private[spark] def caseSensitive: Boolean = + getConf(CASE_SENSITIVE, "true").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 31cc4170aa86..d6c48fb38733 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 @@ -65,7 +65,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, functionRegistry, caseSensitive = true) + new Analyzer(catalog, functionRegistry, caseSensitive) @transient protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer From f57f15ce72652b3a04229a860a2aba22297368b8 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 4 Dec 2014 01:48:04 +0800 Subject: [PATCH 02/24] add testcase --- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../test/TestCaseInsensitiveSQLContext.scala | 35 ++++++++++++ .../sql/SQLQueryCaseInsensitiveSuite.scala | 53 +++++++++++++++++++ 3 files changed, 89 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala 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 d6c48fb38733..a2d1f60ac7c0 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 @@ -58,7 +58,7 @@ class SQLContext(@transient val sparkContext: SparkContext) self => @transient - protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + protected[sql] lazy val catalog: Catalog = new SimpleCatalog(caseSensitive) @transient protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala new file mode 100644 index 000000000000..60077ebdd87c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala @@ -0,0 +1,35 @@ +/* + * 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.test + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.{SQLConf, SQLContext} + +/** A case insensitive SQLContext that can be used for local testing. */ +object TestCaseInsensitiveSQLContext + extends SQLContext( + new SparkContext( + "local[2]", + "CaseInsensitiveSQLContext", + new SparkConf().set(SQLConf.CASE_SENSITIVE, "false"))) { + + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala new file mode 100644 index 000000000000..270d36fd13a2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala @@ -0,0 +1,53 @@ +/* + * 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 + +import java.util.TimeZone + +import org.apache.spark.sql.test.TestCaseInsensitiveSQLContext +import org.scalatest.BeforeAndAfterAll + +/* Implicits */ + +import org.apache.spark.sql.test.TestCaseInsensitiveSQLContext._ + +object CaseInsensitiveTestData{ + case class StringData(s: String) + val table = TestCaseInsensitiveSQLContext.sparkContext.parallelize(StringData("test") :: Nil) + table.registerTempTable("caseInsensitiveTable") +} + +class SQLQueryCaseInsensitiveSuite extends QueryTest with BeforeAndAfterAll { + CaseInsensitiveTestData + + var origZone: TimeZone = _ + + override protected def beforeAll() { + origZone = TimeZone.getDefault + TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + } + + override protected def afterAll() { + TimeZone.setDefault(origZone) + } + + test("SPARK-4699 case sensitivity SQL query") { + checkAnswer(sql("SELECT S FROM CASEINSENSITIVETABLE"), "test") + } + +} From fcbf0d9162574cf6f28dc703224e23d357f0aad9 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 21 Dec 2014 01:36:44 +0800 Subject: [PATCH 03/24] fix scalastyle check --- .../scala/org/apache/spark/sql/catalyst/CatalystConf.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index 2572c010a16f..cb04cc8dca42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -50,4 +50,4 @@ object EmptyConf extends CatalystConf { def getAllConfs: immutable.Map[String, String] = { throw new UnsupportedOperationException } -} \ No newline at end of file +} 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 698bbf9a89af..2a68b5267fc3 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 @@ -254,7 +254,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this, this) with OverrideCatalog + override protected[sql] lazy val catalog = + new HiveMetastoreCatalog(this, this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient From 6332e0ffeac2180406cabfe789ef0ba697b49fa9 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 3 Jan 2015 21:56:51 +0800 Subject: [PATCH 04/24] fix bug --- .gitignore | 8 +- README.md | 2 +- core/pom.xml | 4 +- .../org/apache/spark/ui/static/webui.css | 10 + .../scala/org/apache/spark/Dependency.scala | 3 + .../org/apache/spark/MapOutputTracker.scala | 5 +- .../org/apache/spark/SecurityManager.scala | 4 +- .../scala/org/apache/spark/SparkContext.scala | 17 +- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/api/java/JavaUtils.scala | 2 +- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache/spark/deploy/DeployMessage.scala | 3 + .../apache/spark/deploy/SparkHadoopUtil.scala | 12 + .../spark/deploy/SparkSubmitArguments.scala | 31 +- .../SparkSubmitDriverBootstrapper.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 26 +- .../spark/deploy/worker/ExecutorRunner.scala | 4 +- .../apache/spark/deploy/worker/Worker.scala | 37 +- .../input/FixedLengthBinaryInputFormat.scala | 3 +- .../input/FixedLengthBinaryRecordReader.scala | 3 +- .../spark/input/PortableDataStream.scala | 4 +- .../input/WholeTextFileRecordReader.scala | 4 +- .../apache/spark/io/CompressionCodec.scala | 27 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 16 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 48 +- .../spark/scheduler/TaskResultGetter.scala | 1 + .../spark/serializer/KryoSerializer.scala | 2 + .../scala/org/apache/spark/ui/UIUtils.scala | 24 +- .../scala/org/apache/spark/util/Utils.scala | 40 +- .../scala/org/apache/spark/util/Vector.scala | 38 +- .../java/org/apache/spark/JavaAPISuite.java | 4 +- .../org/apache/spark/DistributedSuite.scala | 21 +- .../scala/org/apache/spark/DriverSuite.scala | 5 +- .../org/apache/spark/FileServerSuite.scala | 16 +- .../apache/spark/JobCancellationSuite.scala | 21 +- .../scala/org/apache/spark/ShuffleSuite.scala | 22 +- .../org/apache/spark/SparkConfSuite.scala | 51 +- .../SparkContextSchedulerCreationSuite.scala | 31 +- .../org/apache/spark/SparkContextSuite.scala | 62 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 8 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- .../spark/io/CompressionCodecSuite.scala | 6 + .../metrics/InputOutputMetricsSuite.scala | 4 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../scheduler/EventLoggingListenerSuite.scala | 4 +- .../spark/scheduler/ReplayListenerSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 9 +- .../scheduler/TaskResultGetterSuite.scala | 23 +- .../scheduler/TaskSchedulerImplSuite.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 23 +- .../apache/spark/util/AkkaUtilsSuite.scala | 2 +- .../spark/util/SizeEstimatorSuite.scala | 38 +- .../org/apache/spark/util/UtilsSuite.scala | 2 +- dev/create-release/create-release.sh | 10 +- dev/mima | 8 +- dev/run-tests | 50 +- dev/scalastyle | 4 +- docs/README.md | 6 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/building-spark.md | 47 +- docs/configuration.md | 37 +- docs/hadoop-third-party-distributions.md | 10 +- docs/job-scheduling.md | 2 +- docs/running-on-yarn.md | 10 +- docs/sql-programming-guide.md | 14 +- docs/streaming-kafka-integration.md | 2 +- docs/streaming-programming-guide.md | 4 +- docs/submitting-applications.md | 6 +- ec2/spark_ec2.py | 7 +- .../examples/streaming/KafkaWordCount.scala | 1 - .../apache/spark/examples/BroadcastTest.scala | 11 +- .../spark/examples/graphx/Analytics.scala | 4 + .../examples/streaming/CustomReceiver.scala | 1 - .../examples/streaming/HdfsWordCount.scala | 1 - .../examples/streaming/MQTTWordCount.scala | 1 - .../examples/streaming/NetworkWordCount.scala | 1 - .../examples/streaming/QueueStream.scala | 1 - .../RecoverableNetworkWordCount.scala | 1 - .../streaming/StatefulNetworkWordCount.scala | 1 - .../streaming/TwitterAlgebirdCMS.scala | 1 - .../streaming/TwitterPopularTags.scala | 1 - .../examples/streaming/ZeroMQWordCount.scala | 1 - .../clickstream/PageViewStream.scala | 1 - .../streaming/LocalJavaStreamingContext.java | 8 +- .../streaming/LocalJavaStreamingContext.java | 8 +- .../streaming/LocalJavaStreamingContext.java | 8 +- .../streaming/LocalJavaStreamingContext.java | 8 +- extras/java8-tests/README.md | 6 +- .../graphx/impl/EdgePartitionBuilder.scala | 63 +- .../impl/ShippableVertexPartition.scala | 4 +- .../spark/graphx/impl/VertexPartition.scala | 4 +- .../graphx/impl/VertexPartitionBaseOps.scala | 4 +- .../mllib/clustering/StreamingKMeans.scala | 1 - .../BinaryClassificationMetrics.scala | 59 +- .../apache/spark/mllib/feature/Word2Vec.scala | 15 +- .../apache/spark/mllib/linalg/Matrices.scala | 570 ++++++++++++++++-- .../apache/spark/mllib/linalg/Vectors.scala | 82 ++- .../linalg/distributed/IndexedRowMatrix.scala | 2 +- .../mllib/linalg/distributed/RowMatrix.scala | 4 +- .../spark/mllib/optimization/Gradient.scala | 10 +- .../apache/spark/mllib/regression/Lasso.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 2 +- .../regression/StreamingLinearAlgorithm.scala | 1 - .../spark/mllib/tree/DecisionTree.scala | 7 - .../org/apache/spark/mllib/util/MLUtils.scala | 23 +- .../spark/mllib/feature/JavaTfIdfSuite.java | 2 + .../BinaryClassificationMetricsSuite.scala | 36 ++ .../spark/mllib/linalg/MatricesSuite.scala | 172 +++++- .../StreamingLinearRegressionSuite.scala | 6 +- .../spark/mllib/util/MLUtilsSuite.scala | 15 + .../spark/mllib/util/TestingUtils.scala | 6 +- .../spark/network/util/TransportConf.java | 2 +- .../shuffle/ExternalShuffleBlockManager.java | 8 +- pom.xml | 13 +- project/MimaExcludes.scala | 8 + project/SparkBuild.scala | 20 +- python/pyspark/sql.py | 30 +- python/pyspark/streaming/dstream.py | 12 +- sbt/sbt | 136 +---- sbt/sbt-launch-lib.bash | 188 ------ sql/README.md | 4 +- .../apache/spark/sql/catalyst/SqlParser.scala | 12 +- .../sql/catalyst/analysis/Analyzer.scala | 25 +- .../catalyst/analysis/HiveTypeCoercion.scala | 16 + .../spark/sql/catalyst/dsl/package.scala | 84 +-- .../sql/catalyst/expressions/predicates.scala | 9 +- .../sql/catalyst/optimizer/Optimizer.scala | 35 +- .../plans/logical/basicOperators.scala | 11 +- .../analysis/DecimalPrecisionSuite.scala | 17 + .../spark/sql/api/java/UserDefinedType.java | 1 + .../org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/SchemaRDD.scala | 20 +- .../spark/sql/api/java/JavaSchemaRDD.scala | 6 + .../spark/sql/execution/SparkStrategies.scala | 17 +- .../sql/parquet/ParquetTableSupport.scala | 4 +- .../spark/sql/parquet/ParquetTypes.scala | 4 +- .../test/TestCaseInsensitiveSQLContext.scala | 38 -- .../spark/sql/api/java/JavaRowSuite.java | 1 + .../org/apache/spark/sql/DslQuerySuite.scala | 21 +- .../org/apache/spark/sql/JoinSuite.scala | 38 ++ .../sql/SQLQueryCaseInsensitiveSuite.scala | 55 -- .../org/apache/spark/sql/SQLQuerySuite.scala | 20 +- .../scala/org/apache/spark/sql/TestData.scala | 2 +- .../columnar/PartitionBatchPruningSuite.scala | 10 +- .../sql/parquet/ParquetFilterSuite.scala | 60 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 14 +- .../thriftserver/AbstractSparkSQLDriver.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 3 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveInspectors.scala | 4 +- .../org/apache/spark/sql/hive/HiveQl.scala | 8 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 6 - .../sql/hive/parquet/FakeParquetSerDe.scala | 2 +- .../sql/hive/execution/UDFListListInt.java | 20 +- .../spark/sql/hive/HiveInspectorSuite.scala | 28 +- .../spark/sql/hive/StatisticsSuite.scala | 50 +- .../hive/execution/HiveComparisonTest.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 7 + .../hive/execution/HiveTableScanSuite.scala | 14 +- .../sql/hive/execution/SQLQuerySuite.scala | 7 + .../spark/streaming/ContextWaiter.scala | 63 +- .../spark/streaming/StreamingContext.scala | 7 +- .../spark/streaming/StreamingSource.scala | 53 +- .../streaming/api/java/JavaDStreamLike.scala | 10 +- .../streaming/api/java/JavaPairDStream.scala | 3 +- .../spark/streaming/dstream/DStream.scala | 36 +- .../dstream/PairDStreamFunctions.scala | 4 +- .../dstream/ReducedWindowedDStream.scala | 2 - .../org/apache/spark/streaming/package.scala | 2 +- .../streaming/receiver/BlockGenerator.scala | 2 +- .../receiver/ReceivedBlockHandler.scala | 33 +- .../streaming/receiver/ReceiverMessage.scala | 2 +- .../scheduler/ReceivedBlockTracker.scala | 9 +- .../streaming/scheduler/ReceiverTracker.scala | 6 +- .../ui/StreamingJobProgressListener.scala | 19 +- .../spark/streaming/ui/StreamingPage.scala | 6 + .../streaming/util/WriteAheadLogManager.scala | 17 +- .../apache/spark/streaming/JavaAPISuite.java | 16 +- .../streaming/LocalJavaStreamingContext.java | 8 +- .../streaming/BasicOperationsSuite.scala | 1 - .../spark/streaming/CheckpointSuite.scala | 1 - .../spark/streaming/MasterFailureTest.scala | 1 - .../streaming/ReceivedBlockHandlerSuite.scala | 2 +- .../streaming/ReceivedBlockTrackerSuite.scala | 2 +- .../streaming/WindowOperationsSuite.scala | 1 - .../streaming/util/WriteAheadLogSuite.scala | 18 +- .../spark/tools/StoragePerfTester.scala | 12 +- .../spark/deploy/yarn/ApplicationMaster.scala | 4 +- .../spark/deploy/yarn/ClientArguments.scala | 5 +- .../apache/spark/deploy/yarn/ClientBase.scala | 2 +- .../deploy/yarn/YarnAllocationHandler.scala | 213 ------- .../spark/deploy/yarn/YarnAllocator.scala | 201 ++++-- .../spark/deploy/yarn/YarnRMClient.scala | 78 ++- .../spark/deploy/yarn/YarnRMClientImpl.scala | 110 ---- .../cluster/YarnClientSchedulerBackend.scala | 17 +- 197 files changed, 2598 insertions(+), 1596 deletions(-) delete mode 100755 sbt/sbt-launch-lib.bash delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala delete mode 100644 yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/.gitignore b/.gitignore index 30b1e12bf1b0..9757054a50f9 100644 --- a/.gitignore +++ b/.gitignore @@ -8,16 +8,19 @@ *.pyc .idea/ .idea_modules/ -sbt/*.jar +build/*.jar .settings .cache +cache .generated-mima* -/build/ work/ out/ .DS_Store third_party/libmesos.so third_party/libmesos.dylib +build/apache-maven* +build/zinc* +build/scala* conf/java-opts conf/*.sh conf/*.cmd @@ -55,6 +58,7 @@ dev/create-release/*final spark-*-bin-*.tgz unit-tests.log /lib/ +ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml diff --git a/README.md b/README.md index 8d57d50da96c..16628bd40677 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html). +["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell diff --git a/core/pom.xml b/core/pom.xml index 1feb00b3a7fb..c5c41b2b5de4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -352,9 +352,9 @@ - + - + diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 68c52ac09b55..5751964b792c 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -169,6 +169,16 @@ span.additional-metric-title { display: inline-block; } +.version { + line-height: 30px; + vertical-align: bottom; + font-size: 12px; + padding: 0; + margin: 0; + font-weight: bold; + color: #777; +} + /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ .scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time { diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index ab2594cfc02e..9a7cd4523e5a 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -60,6 +60,9 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will * be used. + * @param keyOrdering key ordering for RDD's shuffles + * @param aggregator map/reduce-side aggregator for RDD's shuffle + * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) */ @DeveloperApi class ShuffleDependency[K, V, C]( diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a074ab8ece1b..6e4edc7c80d7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -76,6 +76,8 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) + private val retryAttempts = AkkaUtils.numRetries(conf) + private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ @@ -108,8 +110,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ protected def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) - Await.result(future, timeout) + AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout) } catch { case e: Exception => logError("Error communicating with MapOutputTracker", e) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 49dae5231a92..ec82d09cd079 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -151,8 +151,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with private val authOn = sparkConf.getBoolean("spark.authenticate", false) // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse( - sparkConf.get("spark.ui.acls.enable", "false")).toBoolean + private var aclsOn = + sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) // admin acls should be set before view or modify acls private var adminAcls: Set[String] = diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b1c6a90d182f..df1cb3cda2db 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -172,6 +172,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = this(master, appName, sparkHome, jars, Map(), Map()) + // log out Spark Version in Spark driver log + logInfo(s"Running Spark version $SPARK_VERSION") + private[spark] val conf = config.clone() conf.validateSettings() @@ -1705,19 +1708,19 @@ object SparkContext extends Logging { // Implicit conversions to common Writable types, for saveAsSequenceFile - implicit def intToIntWritable(i: Int) = new IntWritable(i) + implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i) - implicit def longToLongWritable(l: Long) = new LongWritable(l) + implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l) - implicit def floatToFloatWritable(f: Float) = new FloatWritable(f) + implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f) - implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d) + implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d) - implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b) + implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b) - implicit def bytesToBytesWritable (aob: Array[Byte]) = new BytesWritable(aob) + implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob) - implicit def stringToText(s: String) = new Text(s) + implicit def stringToText(s: String): Text = new Text(s) private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) : ArrayWritable = { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6656df44d359..43436a169700 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -395,7 +395,7 @@ object SparkEnv extends Logging { val sparkProperties = (conf.getAll ++ schedulerMode).sorted // System properties that are not java classpaths - val systemProperties = System.getProperties.iterator.toSeq + val systemProperties = Utils.getSystemProperties.toSeq val otherProperties = systemProperties.filter { case (k, _) => k != "java.class.path" && !k.startsWith("spark.") }.sorted diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 86e94931300f..71b26737b8c0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -80,7 +80,7 @@ private[spark] object JavaUtils { prev match { case Some(k) => underlying match { - case mm: mutable.Map[a, _] => + case mm: mutable.Map[A, _] => mm remove k prev = None case _ => diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index b10b7b8d3252..65a1a8fd7e92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -23,7 +23,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, var appUiUrl: String, - val eventLogFile: Option[String] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index c46f84de8444..243d8edb72ed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -88,6 +88,8 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + case class ApplicationFinished(id: String) + // Worker internal case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders @@ -175,4 +177,5 @@ private[deploy] object DeployMessages { // Liveness checks in various places case object SendHeartbeat + } 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 60ee115e393c..57f9faf5ddd1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation @@ -183,6 +184,17 @@ class SparkHadoopUtil extends Logging { Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") statisticsDataClass.getDeclaredMethod(methodName) } + + /** + * Using reflection to get the Configuration from JobContext/TaskAttemptContext. If we directly + * call `JobContext/TaskAttemptContext.getConfiguration`, it will generate different byte codes + * for Hadoop 1.+ and Hadoop 2.+ because JobContext/TaskAttemptContext is class in Hadoop 1.+ + * while it's interface in Hadoop 2.+. + */ + def getConfigurationFromJobContext(context: JobContext): Configuration = { + val method = context.getClass.getMethod("getConfiguration") + method.invoke(context).asInstanceOf[Configuration] + } } object SparkHadoopUtil { 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 607b4df73db9..1faabe91f49a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import java.net.URI import java.util.jar.JarFile import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -125,14 +126,23 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { - try { - val jar = new JarFile(primaryResource) - // Note that this might still return null if no main-class is set; we catch that later - mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") - } catch { - case e: Exception => - SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) - return + val uri = new URI(primaryResource) + val uriScheme = uri.getScheme() + + uriScheme match { + case "file" => + try { + val jar = new JarFile(uri.getPath) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit(s"Cannot load main class from JAR $primaryResource") + } + case _ => + SparkSubmit.printErrorAndExit( + s"Cannot load main class from JAR $primaryResource with URI $uriScheme. " + + "Please specify a class through --class.") } } @@ -214,7 +224,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St """.stripMargin } - /** Fill in values by parsing user options. */ + /** + * Fill in values by parsing user options. + * NOTE: Any changes here must be reflected in YarnClientSchedulerBackend. + */ private def parseOpts(opts: Seq[String]): Unit = { val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index d2687faad62b..2eab9981845e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -151,7 +151,8 @@ private[spark] object SparkSubmitDriverBootstrapper { val isWindows = Utils.isWindows val isSubprocess = sys.env.contains("IS_SUBPROCESS") if (!isWindows) { - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin", + propagateEof = true) stdinThread.start() // Spark submit (JVM) may run as a subprocess, and so this JVM should terminate on // broken pipe, signaling that the parent process has exited. This is the case if the diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index ed5eab9f473c..e8a5cfc746fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -123,6 +123,7 @@ private[spark] class Master( override def preStart() { logInfo("Starting Spark master at " + masterUrl) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.bind() @@ -704,6 +705,11 @@ private[spark] class Master( } persistenceEngine.removeApplication(app) schedule() + + // Tell all workers that the application has finished, so they can clean up any app state. + workers.foreach { w => + w.actor ! ApplicationFinished(app.id) + } } } @@ -714,14 +720,26 @@ private[spark] class Master( def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogFile = app.desc.eventLogFile.getOrElse { - // Event logging is not enabled for this application - app.desc.appUiUrl = notFoundBasePath + val eventLogFile = app.desc.eventLogDir + .map { dir => EventLoggingListener.getLogPath(dir, app.id) } + .getOrElse { + // Event logging is not enabled for this application + app.desc.appUiUrl = notFoundBasePath + return false + } + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) + + if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { + // Event logging is enabled for this application, but the application is still in progress + val title = s"Application history not found (${app.id})" + var msg = s"Application $appName is still in progress." + logWarning(msg) + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" return false } try { - val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), 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 f4fedc6327ab..acbdf0d8bd7b 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 @@ -47,6 +47,7 @@ private[spark] class ExecutorRunner( val executorDir: File, val workerUrl: String, val conf: SparkConf, + val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { @@ -77,7 +78,7 @@ private[spark] class ExecutorRunner( /** * Kill executor process, wait for exit and notify worker to update resource status. * - * @param message the exception message which caused the executor's death + * @param message the exception message which caused the executor's death */ private def killProcess(message: Option[String]) { var exitCode: Option[Int] = None @@ -129,6 +130,7 @@ private[spark] class ExecutorRunner( logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) builder.directory(executorDir) + builder.environment.put("SPARK_LOCAL_DIRS", appLocalDirs.mkString(",")) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") 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 6863b625514c..f0f3da5eec4d 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import java.util.{UUID, Date} import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random @@ -109,6 +109,8 @@ private[spark] class Worker( val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] val finishedDrivers = new HashMap[String, DriverRunner] + val appDirectories = new HashMap[String, Seq[String]] + val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) @@ -153,6 +155,7 @@ private[spark] class Worker( assert(!registered) logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( host, port, cores, Utils.megabytesToString(memory))) + logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -294,7 +297,7 @@ private[spark] class Worker( val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir) dir.isDirectory && !isAppStillRunning && !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS) - }.foreach { dir => + }.foreach { dir => logInfo(s"Removing directory: ${dir.getPath}") Utils.deleteRecursively(dir) } @@ -339,8 +342,19 @@ private[spark] class Worker( throw new IOException("Failed to create directory " + executorDir) } + // Create local dirs for the executor. These are passed to the executor via the + // SPARK_LOCAL_DIRS environment variable, and deleted by the Worker when the + // application finishes. + val appLocalDirs = appDirectories.get(appId).getOrElse { + Utils.getOrCreateLocalRootDirs(conf).map { dir => + Utils.createDirectory(dir).getAbsolutePath() + }.toSeq + } + appDirectories(appId) = appLocalDirs + val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING) + self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs, + ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -377,6 +391,7 @@ private[spark] class Worker( message.map(" message " + _).getOrElse("") + exitStatus.map(" exitStatus " + _).getOrElse("")) } + maybeCleanupApplication(appId) } case KillExecutor(masterUrl, appId, execId) => @@ -446,6 +461,9 @@ private[spark] class Worker( case ReregisterWithMaster => reregisterWithMaster() + case ApplicationFinished(id) => + finishedApps += id + maybeCleanupApplication(id) } private def masterDisconnected() { @@ -454,6 +472,19 @@ private[spark] class Worker( registerWithMaster() } + private def maybeCleanupApplication(id: String): Unit = { + val shouldCleanup = finishedApps.contains(id) && !executors.values.exists(_.appId == id) + if (shouldCleanup) { + finishedApps -= id + appDirectories.remove(id).foreach { dirList => + logInfo(s"Cleaning up local directories for application $id") + dirList.foreach { dir => + Utils.deleteRecursively(new File(dir)) + } + } + } + } + def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index 89b29af2000c..c219d21fbefa 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.spark.deploy.SparkHadoopUtil /** * Custom Input Format for reading and splitting flat binary files that contain records, @@ -33,7 +34,7 @@ private[spark] object FixedLengthBinaryInputFormat { /** Retrieves the record length property from a Hadoop configuration */ def getRecordLength(context: JobContext): Int = { - context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt + SparkHadoopUtil.get.getConfigurationFromJobContext(context).get(RECORD_LENGTH_PROPERTY).toInt } } diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 36a1e5d475f4..67a96925da01 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.spark.deploy.SparkHadoopUtil /** * FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. @@ -82,7 +83,7 @@ private[spark] class FixedLengthBinaryRecordReader // the actual file we will be reading from val file = fileSplit.getPath // job configuration - val job = context.getConfiguration + val job = SparkHadoopUtil.get.getConfigurationFromJobContext(context) // check compression val codec = new CompressionCodecFactory(job).getCodec(file) if (codec != null) { 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 457472547fcb..593a62b3e3b3 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAt import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil /** * A general format for reading whole files in as streams, byte arrays, @@ -145,7 +146,8 @@ class PortableDataStream( private val confBytes = { val baos = new ByteArrayOutputStream() - context.getConfiguration.write(new DataOutputStream(baos)) + SparkHadoopUtil.get.getConfigurationFromJobContext(context). + write(new DataOutputStream(baos)) baos.toByteArray } diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index 1b1131b9b883..31bde8a78f3c 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, CombineFileRecordReader} import org.apache.hadoop.mapreduce.RecordReader import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.spark.deploy.SparkHadoopUtil /** @@ -51,7 +52,8 @@ private[spark] class WholeTextFileRecordReader( extends RecordReader[String, String] with Configurable { private[this] val path = split.getPath(index) - private[this] val fs = path.getFileSystem(context.getConfiguration) + private[this] val fs = path.getFileSystem( + SparkHadoopUtil.get.getConfigurationFromJobContext(context)) // True means the current file has been processed, then skip it. private[this] var processed = false diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 1ac7f4e448eb..f856890d279f 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -21,11 +21,12 @@ import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} -import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} +import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils +import org.apache.spark.Logging /** * :: DeveloperApi :: @@ -44,25 +45,33 @@ trait CompressionCodec { def compressedInputStream(s: InputStream): InputStream } - private[spark] object CompressionCodec { + private val configKey = "spark.io.compression.codec" private val shortCompressionCodecNames = Map( "lz4" -> classOf[LZ4CompressionCodec].getName, "lzf" -> classOf[LZFCompressionCodec].getName, "snappy" -> classOf[SnappyCompressionCodec].getName) def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) + createCodec(conf, conf.get(configKey, DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) - val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) - .getConstructor(classOf[SparkConf]) - ctor.newInstance(conf).asInstanceOf[CompressionCodec] + val codec = try { + val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) + .getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + } catch { + case e: ClassNotFoundException => None + case e: IllegalArgumentException => None + } + codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " + + s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC")) } + val FALLBACK_COMPRESSION_CODEC = "lzf" val DEFAULT_COMPRESSION_CODEC = "snappy" val ALL_COMPRESSION_CODECS = shortCompressionCodecNames.values.toSeq } @@ -120,6 +129,12 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { + try { + Snappy.getNativeLibraryVersion + } catch { + case e: Error => throw new IllegalArgumentException + } + override def compressedOutputStream(s: OutputStream): OutputStream = { val blockSize = conf.getInt("spark.io.compression.snappy.block.size", 32768) new SnappyOutputStream(s, blockSize) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index a157e36e2286..0001c2329c83 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -131,7 +131,7 @@ class HadoopRDD[K, V]( // used to build JobTracker ID private val createTime = new Date() - private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean + private val shouldCloneJobConf = sc.conf.getBoolean("spark.hadoop.cloneConf", false) // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { 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 fe3129b62f71..4469c89e6bb1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -483,7 +483,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues( pair => - for (v <- pair._1; w <- pair._2) yield (v, w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, w) ) } @@ -496,9 +496,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._2.isEmpty) { - pair._1.map(v => (v, None)) + pair._1.iterator.map(v => (v, None)) } else { - for (v <- pair._1; w <- pair._2) yield (v, Some(w)) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (v, Some(w)) } } } @@ -513,9 +513,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { pair => if (pair._1.isEmpty) { - pair._2.map(w => (None, w)) + pair._2.iterator.map(w => (None, w)) } else { - for (v <- pair._1; w <- pair._2) yield (Some(v), w) + for (v <- pair._1.iterator; w <- pair._2.iterator) yield (Some(v), w) } } } @@ -531,9 +531,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { - case (vs, Seq()) => vs.map(v => (Some(v), None)) - case (Seq(), ws) => ws.map(w => (None, Some(w))) - case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w)) + case (vs, Seq()) => vs.iterator.map(v => (Some(v), None)) + case (Seq(), ws) => ws.iterator.map(w => (None, Some(w))) + case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), Some(w)) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 214f22bc5b60..5118e2b91112 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1146,15 +1146,20 @@ abstract class RDD[T: ClassTag]( if (num == 0) { Array.empty } else { - mapPartitions { items => + val mapRDDs = mapPartitions { items => // Priority keeps the largest elements, so let's reverse the ordering. val queue = new BoundedPriorityQueue[T](num)(ord.reverse) queue ++= util.collection.Utils.takeOrdered(items, num)(ord) Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord) + } + if (mapRDDs.partitions.size == 0) { + Array.empty + } else { + mapRDDs.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } } } @@ -1174,7 +1179,26 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a text file, using string representations of elements. */ def saveAsTextFile(path: String) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + // + // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit + // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]` + // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an + // Ordering for `NullWritable`. That's why the compiler will generate different anonymous + // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. + // + // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate + // same bytecodes for `saveAsTextFile`. + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } @@ -1182,7 +1206,17 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a compressed text file, using string representations of elements. */ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { - this.map(x => (NullWritable.get(), new Text(x.toString))) + // https://issues.apache.org/jira/browse/SPARK-2075 + val nullWritableClassTag = implicitly[ClassTag[NullWritable]] + val textClassTag = implicitly[ClassTag[Text]] + val r = this.mapPartitions { iter => + val text = new Text() + iter.map { x => + text.set(x.toString) + (NullWritable.get(), text) + } + } + RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 819b51e12ad8..4896ec845bbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import scala.language.existentials import scala.util.control.NonFatal import org.apache.spark._ 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 621a951c27d0..d2947dcea4f7 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -26,6 +26,7 @@ import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ +import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} import org.apache.spark.scheduler.MapStatus @@ -90,6 +91,7 @@ class KryoSerializer(conf: SparkConf) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) + kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) try { // Use the default classloader when calling the user registrator. diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index d970fa30c1c3..b5022fe853c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -194,9 +194,12 @@ private[spark] object UIUtils extends Logging { - } @@ -236,8 +234,9 @@ private[spark] object UIUtils extends Logging {

- + + {org.apache.spark.SPARK_VERSION} {title}

@@ -245,11 +244,6 @@ private[spark] object UIUtils extends Logging {
{content} - } 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 5e1cb0c7a771..0d771baaa6ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -246,8 +246,11 @@ private[spark] object Utils extends Logging { retval } - /** Create a temporary directory inside the given parent directory */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + /** + * Create a directory inside the given parent directory. The directory is guaranteed to be + * newly created, and is not marked for automatic deletion. + */ + def createDirectory(root: String): File = { var attempts = 0 val maxAttempts = 10 var dir: File = null @@ -265,6 +268,15 @@ private[spark] object Utils extends Logging { } catch { case e: SecurityException => dir = null; } } + dir + } + + /** + * Create a temporary directory inside the given parent directory. The directory will be + * automatically deleted when the VM shuts down. + */ + def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { + val dir = createDirectory(root) registerShutdownDeleteDir(dir) dir } @@ -1835,19 +1847,29 @@ private[spark] object Utils extends Logging { /** * A utility class to redirect the child process's stdout or stderr. */ -private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String, + propagateEof: Boolean = false) extends Thread(name) { setDaemon(true) override def run() { scala.util.control.Exception.ignoring(classOf[IOException]) { // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) + try { + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } finally { + if (propagateEof) { + out.close() + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index c6cab82c3e54..2ed827eab46d 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -24,9 +24,9 @@ import org.apache.spark.util.random.XORShiftRandom @deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { - def length = elements.length + def length: Int = elements.length - def apply(index: Int) = elements(index) + def apply(index: Int): Double = elements(index) def + (other: Vector): Vector = { if (length != other.length) { @@ -35,7 +35,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) + other(i)) } - def add(other: Vector) = this + other + def add(other: Vector): Vector = this + other def - (other: Vector): Vector = { if (length != other.length) { @@ -44,7 +44,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) - other(i)) } - def subtract(other: Vector) = this - other + def subtract(other: Vector): Vector = this - other def dot(other: Vector): Double = { if (length != other.length) { @@ -93,19 +93,19 @@ class Vector(val elements: Array[Double]) extends Serializable { this } - def addInPlace(other: Vector) = this +=other + def addInPlace(other: Vector): Vector = this +=other def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - def multiply (d: Double) = this * d + def multiply (d: Double): Vector = this * d def / (d: Double): Vector = this * (1 / d) - def divide (d: Double) = this / d + def divide (d: Double): Vector = this / d - def unary_- = this * -1 + def unary_- : Vector = this * -1 - def sum = elements.reduceLeft(_ + _) + def sum: Double = elements.reduceLeft(_ + _) def squaredDist(other: Vector): Double = { var ans = 0.0 @@ -119,40 +119,40 @@ class Vector(val elements: Array[Double]) extends Serializable { def dist(other: Vector): Double = math.sqrt(squaredDist(other)) - override def toString = elements.mkString("(", ", ", ")") + override def toString: String = elements.mkString("(", ", ", ")") } object Vector { - def apply(elements: Array[Double]) = new Vector(elements) + def apply(elements: Array[Double]): Vector = new Vector(elements) - def apply(elements: Double*) = new Vector(elements.toArray) + def apply(elements: Double*): Vector = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { val elements: Array[Double] = Array.tabulate(length)(initializer) new Vector(elements) } - def zeros(length: Int) = new Vector(new Array[Double](length)) + def zeros(length: Int): Vector = new Vector(new Array[Double](length)) - def ones(length: Int) = Vector(length, _ => 1) + def ones(length: Int): Vector = Vector(length, _ => 1) /** * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ - def random(length: Int, random: Random = new XORShiftRandom()) = + def random(length: Int, random: Random = new XORShiftRandom()): Vector = Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { - def * (vec: Vector) = vec * num + def * (vec: Vector): Vector = vec * num } - implicit def doubleToMultiplier(num: Double) = new Multiplier(num) + implicit def doubleToMultiplier(num: Double): Multiplier = new Multiplier(num) implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { - def addInPlace(t1: Vector, t2: Vector) = t1 + t2 + def addInPlace(t1: Vector, t2: Vector): Vector = t1 + t2 - def zero(initialValue: Vector) = Vector.zeros(initialValue.length) + def zero(initialValue: Vector): Vector = Vector.zeros(initialValue.length) } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e5bdad6bda2f..5ce299d05824 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -184,6 +184,7 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @SuppressWarnings("unchecked") @Test public void repartitionAndSortWithinPartitions() { List> pairs = new ArrayList>(); @@ -491,6 +492,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @SuppressWarnings("unchecked") @Test public void aggregateByKey() { JavaPairRDD pairs = sc.parallelizePairs( @@ -1556,7 +1558,7 @@ static class Class2 {} @Test public void testRegisterKryoClasses() { SparkConf conf = new SparkConf(); - conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); + conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); Assert.assertEquals( Class1.class.getName() + "," + Class2.class.getName(), conf.get("spark.kryo.classesToRegister")); diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 998f3008ec0e..97ea3578aa8b 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -29,16 +28,10 @@ class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter - with LocalSparkContext { +class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" - after { - System.clearProperty("spark.reducer.maxMbInFlight") - System.clearProperty("spark.storage.memoryFraction") - } - test("task throws not serializable exception") { // Ensures that executors do not crash when an exn is not serializable. If executors crash, // this test will hang. Correct behavior is that executors don't crash but fail tasks @@ -84,15 +77,14 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("groupByKey where map output sizes exceed maxMbInFlight") { - System.setProperty("spark.reducer.maxMbInFlight", "1") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1") + sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000))) val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect() assert(groups.length === 16) assert(groups.map(_._2).sum === 2000) - // Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block } test("accumulators") { @@ -210,7 +202,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("compute without caching when no partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.0001") sc = new SparkContext(clusterUrl, "test") // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory @@ -218,12 +209,11 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("compute when only some partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.01") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.storage.memoryFraction", "0.01") + sc = new SparkContext(clusterUrl, "test", conf) // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions // to make sure that *some* of them do fit though @@ -231,7 +221,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("passing environment variables to cluster") { diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 5265ba904032..541d8eac8055 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -50,7 +50,10 @@ class DriverSuite extends FunSuite with Timeouts { object DriverWithoutCleanup { def main(args: Array[String]) { Utils.configTestLog4j("INFO") - val sc = new SparkContext(args(0), "DriverWithoutCleanup") + // Bind the web UI to an ephemeral port in order to avoid conflicts with other tests running on + // the same machine (we shouldn't just disable the UI here, since that might mask bugs): + val conf = new SparkConf().set("spark.ui.port", "0") + val sc = new SparkContext(args(0), "DriverWithoutCleanup", conf) sc.parallelize(1 to 100, 4).count() } } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 49426545c767..0f49ce4754fb 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -31,10 +31,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ + def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false") + override def beforeEach() { super.beforeEach() resetSparkContext() - System.setProperty("spark.authenticate", "false") } override def beforeAll() { @@ -52,7 +53,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) - System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) @@ -74,7 +74,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -108,7 +108,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test("Distributing files locally using URL as input") { // addFile("file:///....") - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -122,7 +122,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1, 1)) sc.parallelize(testData).foreach { x => @@ -133,7 +133,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -147,7 +147,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => @@ -158,7 +158,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 41ed2bce55ce..7584ae79fc92 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -40,12 +40,11 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter override def afterEach() { super.afterEach() resetSparkContext() - System.clearProperty("spark.scheduler.mode") } test("local mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[2]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -53,10 +52,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("local mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local[2]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -64,8 +63,8 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local-cluster[2,1,512]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -73,10 +72,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local-cluster[2,1,512]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 58a96245a9b5..f57921b76831 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -35,19 +35,15 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex conf.set("spark.test.noStageRetry", "true") test("groupByKey without compression") { - try { - System.setProperty("spark.shuffle.compress", "false") - sc = new SparkContext("local", "test", conf) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) - val groups = pairs.groupByKey(4).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } finally { - System.setProperty("spark.shuffle.compress", "true") - } + val myConf = conf.clone().set("spark.shuffle.compress", "false") + sc = new SparkContext("local", "test", myConf) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) + val groups = pairs.groupByKey(4).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) } test("shuffle non-zero block size") { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5d018ea9868a..790976a5ac30 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,27 +19,20 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} +import org.apache.spark.util.ResetSystemProperties import com.esotericsoftware.kryo.Kryo -class SparkConfSuite extends FunSuite with LocalSparkContext { +class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("loading from system properties") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf() - assert(conf.get("spark.test.testProperty") === "2") - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf() + assert(conf.get("spark.test.testProperty") === "2") } test("initializing without loading defaults") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf(false) - assert(!conf.contains("spark.test.testProperty")) - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf(false) + assert(!conf.contains("spark.test.testProperty")) } test("named set methods") { @@ -117,23 +110,17 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { test("nested property names") { // This wasn't supported by some external conf parsing libraries - try { - System.setProperty("spark.test.a", "a") - System.setProperty("spark.test.a.b", "a.b") - System.setProperty("spark.test.a.b.c", "a.b.c") - val conf = new SparkConf() - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "a.b") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - conf.set("spark.test.a.b", "A.B") - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "A.B") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - } finally { - System.clearProperty("spark.test.a") - System.clearProperty("spark.test.a.b") - System.clearProperty("spark.test.a.b.c") - } + System.setProperty("spark.test.a", "a") + System.setProperty("spark.test.a.b", "a.b") + System.setProperty("spark.test.a.b.c", "a.b.c") + val conf = new SparkConf() + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "a.b") + assert(conf.get("spark.test.a.b.c") === "a.b.c") + conf.set("spark.test.a.b", "A.B") + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "A.B") + assert(conf.get("spark.test.a.b.c") === "a.b.c") } test("register kryo classes through registerKryoClasses") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 0390a2e4f1db..8ae4f243ec1a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = { + def createTaskScheduler(master: String): TaskSchedulerImpl = + createTaskScheduler(master, new SparkConf()) + + def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) @@ -102,19 +105,13 @@ class SparkContextSchedulerCreationSuite } test("local-default-parallelism") { - val defaultParallelism = System.getProperty("spark.default.parallelism") - System.setProperty("spark.default.parallelism", "16") - val sched = createTaskScheduler("local") + val conf = new SparkConf().set("spark.default.parallelism", "16") + val sched = createTaskScheduler("local", conf) sched.backend match { case s: LocalBackend => assert(s.defaultParallelism() === 16) case _ => fail() } - - Option(defaultParallelism) match { - case Some(v) => System.setProperty("spark.default.parallelism", v) - case _ => System.clearProperty("spark.default.parallelism") - } } test("simr") { @@ -155,9 +152,10 @@ class SparkContextSchedulerCreationSuite testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") } - def testMesos(master: String, expectedClass: Class[_]) { + def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { + val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) try { - val sched = createTaskScheduler(master) + val sched = createTaskScheduler(master, conf) assert(sched.backend.getClass === expectedClass) } catch { case e: UnsatisfiedLinkError => @@ -168,17 +166,14 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - System.setProperty("spark.mesos.coarse", "true") - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) } test("mesos with zookeeper") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 136202210419..8b3c6871a7b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,55 +23,37 @@ import org.apache.hadoop.io.BytesWritable class SparkContextSuite extends FunSuite with LocalSparkContext { - /** Allows system properties to be changed in tests */ - private def withSystemProperty[T](property: String, value: String)(block: => T): T = { - val originalValue = System.getProperty(property) - try { - System.setProperty(property, value) - block - } finally { - if (originalValue == null) { - System.clearProperty(property) - } else { - System.setProperty(property, originalValue) - } - } - } - test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - // A SparkContext is already running, so we shouldn't be able to create a second one - intercept[SparkException] { new SparkContext(conf) } - // After stopping the running context, we should be able to create a new one - resetSparkContext() - sc = new SparkContext(conf) - } + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "false") + sc = new SparkContext(conf) + // A SparkContext is already running, so we shouldn't be able to create a second one + intercept[SparkException] { new SparkContext(conf) } + // After stopping the running context, we should be able to create a new one + resetSparkContext() + sc = new SparkContext(conf) } test("Can still construct a new SparkContext after failing to construct a previous one") { - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - // This is an invalid configuration (no app name or master URL) - intercept[SparkException] { - new SparkContext(new SparkConf()) - } - // Even though those earlier calls failed, we should still be able to create a new context - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + // This is an invalid configuration (no app name or master URL) + intercept[SparkException] { + new SparkContext(conf) } + // Even though those earlier calls failed, we should still be able to create a new context + sc = new SparkContext(conf.setMaster("local").setAppName("test")) } test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - withSystemProperty("spark.driver.allowMultipleContexts", "true") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } + var secondSparkContext: SparkContext = null + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 3f1cd0752e76..aa65f7e8915e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -119,7 +119,7 @@ class JsonProtocolSuite extends FunSuite { def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", - new SparkConf, ExecutorState.RUNNING) + new SparkConf, Seq("localDir"), ExecutorState.RUNNING) } def createDriverRunner(): DriverRunner = { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index eb7bd7ab3986..065b7534cece 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -23,11 +23,13 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.deploy.SparkSubmit._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ResetSystemProperties, Utils} import org.scalatest.FunSuite import org.scalatest.Matchers -class SparkSubmitSuite extends FunSuite with Matchers { +// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch +// of properties that neeed to be cleared after tests. +class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties { def beforeAll() { System.setProperty("spark.testing", "true") } @@ -288,6 +290,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -302,6 +305,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } 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 196217062991..6f233d7cf97a 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 @@ -33,7 +33,8 @@ class ExecutorRunnerTest extends FunSuite { val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", - new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) + new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), + ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 25be7f25c21b..8c6035fb367f 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -85,4 +85,10 @@ class CompressionCodecSuite extends FunSuite { assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("bad compression codec") { + intercept[IllegalArgumentException] { + CompressionCodec.createCodec(conf, "foobar") + } + } } diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index ca226fd4e694..f8bcde12a371 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -24,14 +24,14 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers +import org.scalatest.Matchers import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import scala.collection.mutable.ArrayBuffer -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with ShouldMatchers { +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers { test("input metrics when reading text file with single split") { val file = new File(getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(file)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 436eea4f1fdc..d6ec9e129cce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -739,7 +739,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F test("accumulator not calculated for resubmitted result stage") { //just for register - val accum = new Accumulator[Int](0, SparkContext.IntAccumulatorParam) + val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 5909811c2034..1de7e130039a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -65,11 +65,11 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val logPath = new Path(eventLogger.logPath + EventLoggingListener.IN_PROGRESS) assert(fileSystem.exists(logPath)) val logStatus = fileSystem.getFileStatus(logPath) - assert(logStatus.isFile) + assert(!logStatus.isDir) // Verify log is renamed after stop() eventLogger.stop() - assert(fileSystem.getFileStatus(new Path(eventLogger.logPath)).isFile()) + assert(!fileSystem.getFileStatus(new Path(eventLogger.logPath)).isDir) } test("Basic event logging") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7e635cb06144..7e360cc6082e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -112,7 +112,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val applications = fileSystem.listStatus(logDirPath) assert(applications != null && applications.size > 0) val eventLog = applications.sortBy(_.getModificationTime).last - assert(eventLog.isFile) + assert(!eventLog.isDir) // Replay events val (logData, version) = EventLoggingListener.openEventLog(eventLog.getPath(), fileSystem) 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 b276343cb412..24f41bf8cccd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -26,9 +26,10 @@ import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.ResetSystemProperties -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter + with BeforeAndAfterAll with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,10 +38,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll() { - System.clearProperty("spark.akka.frameSize") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 5768a3a733f0..3aab5a156ee7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId /** @@ -55,27 +55,20 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll - with LocalSparkContext { +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - override def beforeAll { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small - // as we can make it) so the tests don't take too long. - System.setProperty("spark.akka.frameSize", "1") - } - - override def afterAll { - System.clearProperty("spark.akka.frameSize") - } + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + def conf: SparkConf = new SparkConf().set("spark.akka.frameSize", "1") test("handling results smaller than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } test("handling results larger than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) @@ -89,7 +82,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. - sc = new SparkContext("local[1,2]", "test") + sc = new SparkContext("local[1,2]", "test", conf) // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 7532da88c606..40aaf9dd1f1e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -162,12 +162,12 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) val taskScheduler = new TaskSchedulerImpl(sc) val taskSet = FakeTask.createTaskSet(1) - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) schedulableBuilder.buildPools() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 5554efbcbadf..ffe6f039145e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,7 +33,7 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -44,18 +44,17 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat -import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.apache.spark.util._ -class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter - with PrivateMethodTester { +class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach + with PrivateMethodTester with ResetSystemProperties { private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null var master: BlockManagerMaster = null - var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -79,13 +78,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter manager } - before { + override def beforeEach(): Unit = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") + System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.driver.port", boundPort.toString) @@ -100,7 +99,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter SizeEstimator invokePrivate initialize() } - after { + override def afterEach(): Unit = { if (store != null) { store.stop() store = null @@ -113,14 +112,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter actorSystem.awaitTermination() actorSystem = null master = null - - if (oldArch != null) { - conf.set("os.arch", oldArch) - } else { - System.clearProperty("os.arch") - } - - System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 7bca1711ae22..6bbf72e929dc 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.BlockManagerId /** * Test the AkkaUtils with various security settings. */ -class AkkaUtilsSuite extends FunSuite with LocalSparkContext { +class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("remote fetch security bad password") { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 0ea2d13a8350..7424c2e91d4f 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.util -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite, PrivateMethodTester} class DummyClass1 {} @@ -46,20 +44,12 @@ class DummyString(val arr: Array[Char]) { } class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + extends FunSuite with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { - var oldArch: String = _ - var oldOops: String = _ - - override def beforeAll() { + override def beforeEach() { // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - } - - override def afterAll() { - resetOrClear("os.arch", oldArch) - resetOrClear("spark.test.useCompressedOops", oldOops) + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "true") } test("simple classes") { @@ -122,7 +112,7 @@ class SizeEstimatorSuite } test("32-bit arch") { - val arch = System.setProperty("os.arch", "x86") + System.setProperty("os.arch", "x86") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -131,14 +121,13 @@ class SizeEstimatorSuite assertResult(48)(SizeEstimator.estimate(DummyString("a"))) assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) - resetOrClear("os.arch", arch) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { - val arch = System.setProperty("os.arch", "amd64") - val oops = System.setProperty("spark.test.useCompressedOops", "false") + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -146,16 +135,5 @@ class SizeEstimatorSuite assertResult(64)(SizeEstimator.estimate(DummyString("a"))) assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) - - resetOrClear("os.arch", arch) - resetOrClear("spark.test.useCompressedOops", oops) - } - - def resetOrClear(prop: String, oldValue: String) { - if (oldValue != null) { - System.setProperty(prop, oldValue) - } else { - System.clearProperty(prop) - } } } 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 f9d4bea823f7..4544382094f9 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf -class UtilsSuite extends FunSuite { +class UtilsSuite extends FunSuite with ResetSystemProperties { test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 3b89aaba2960..b1b8cb44e098 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -87,8 +87,8 @@ if [[ ! "$@" =~ --package-only ]]; then git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH - git checkout -f $GIT_TAG - + git checkout -f $GIT_TAG + # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" @@ -106,7 +106,7 @@ if [[ ! "$@" =~ --package-only ]]; then clean install ./dev/change-version-to-2.11.sh - + mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dscala-2.11 -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install @@ -174,7 +174,7 @@ make_binary_release() { NAME=$1 FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME - + cd spark-$RELEASE_VERSION-bin-$NAME # TODO There should probably be a flag to make-distribution to allow 2.11 support @@ -219,7 +219,7 @@ scp spark-* \ # Docs cd spark -sbt/sbt clean +build/sbt clean cd docs # Compile docs with Java 7 to use nicer format JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build diff --git a/dev/mima b/dev/mima index 40603166c21a..bed5cd042634 100755 --- a/dev/mima +++ b/dev/mima @@ -24,13 +24,13 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -echo -e "q\n" | sbt/sbt oldDeps/update +echo -e "q\n" | build/sbt oldDeps/update rm -f .generated-mima* -# Generate Mima Ignore is called twice, first with latest built jars +# Generate Mima Ignore is called twice, first with latest built jars # on the classpath and then again with previous version jars on the classpath. # Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath -# it did not process the new classes (which are in assembly jar). +# it did not process the new classes (which are in assembly jar). ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" @@ -38,7 +38,7 @@ echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" +echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? if [ $ret_val != 0 ]; then diff --git a/dev/run-tests b/dev/run-tests index 328a73bd8b26..20603fc08923 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -59,17 +59,17 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" { if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" - else + else declare java_cmd=java fi - + # We can't use sed -r -e due to OS X / BSD compatibility; hence, all the parentheses. JAVA_VERSION=$( $java_cmd -version 2>&1 \ | grep -e "^java version" --max-count=1 \ | sed "s/java version \"\(.*\)\.\(.*\)\.\(.*\)\"/\1\2/" ) - + if [ "$JAVA_VERSION" -lt 18 ]; then echo "[warn] Java 8 tests will not run because JDK version is < 1.8." fi @@ -79,7 +79,7 @@ export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" # Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - + sql_diffs=$( git diff --name-only master \ | grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" @@ -93,7 +93,7 @@ if [ -n "$AMPLAB_JENKINS" ]; then if [ -n "$sql_diffs" ]; then echo "[info] Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true - + if [ -z "$non_sql_diffs" ]; then echo "[info] Detected no changes except in SQL. Will only run SQL tests." _SQL_TESTS_ONLY=true @@ -141,24 +141,26 @@ CURRENT_BLOCK=$BLOCK_BUILD { # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a - #+ single argument! + # single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? - # First build with 0.12 to ensure patches do not break the hive 12 build + # First build with Hive 0.12.0 to ensure patches do not break the Hive 0.12.0 build HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" - echo "[info] Compile with hive 0.12" + echo "[info] Compile with Hive 0.12.0" echo -e "q\n" \ - | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ + | build/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - # Then build with default version(0.13.1) because tests are based on this version + # Then build with default Hive version (0.13.1) because tests are based on this version + echo "[info] Compile with Hive 0.13.1" + rm -rf lib_managed echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ " -Phive -Phive-thriftserver" echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -175,27 +177,27 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi - + if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string - #+ will be interpreted as a single test, which doesn't work. + # will be interpreted as a single test, which doesn't work. SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") fi - + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" - + # NOTE: echo "q" is needed because sbt on encountering a build file with failure - #+ (either resolution or compilation) prompts the user for input either q, r, etc - #+ to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a - #+ single argument! - #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # (either resolution or compilation) prompts the user for input either q, r, etc + # to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + # single argument! + # "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | build/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -210,7 +212,7 @@ CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS echo "" echo "=========================================================================" -echo "Detecting binary incompatibilites with MiMa" +echo "Detecting binary incompatibilities with MiMa" echo "=========================================================================" CURRENT_BLOCK=$BLOCK_MIMA diff --git a/dev/scalastyle b/dev/scalastyle index 3a4df6e4bf1b..86919227ed1a 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,9 +17,9 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/README.md b/docs/README.md index 119484038083..8a54724c4bea 100644 --- a/docs/README.md +++ b/docs/README.md @@ -21,7 +21,7 @@ read those text files directly if you want. Start with index.md. The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). `Jekyll` and a few dependencies must be installed for this to work. We recommend -installing via the Ruby Gem dependency manager. Since the exact HTML output +installing via the Ruby Gem dependency manager. Since the exact HTML output varies between versions of Jekyll and its dependencies, we list specific versions here in some cases: @@ -60,7 +60,7 @@ We use Sphinx to generate Python API docs, so you will need to install it by run ## API Docs (Scaladoc and Sphinx) -You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. +You can build just the Spark scaladoc by running `build/sbt doc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as @@ -68,7 +68,7 @@ public in `__init__.py`. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `build/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 4566a2fff562..3c626a0b7f54 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -25,8 +25,8 @@ curr_dir = pwd cd("..") - puts "Running 'sbt/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt -Pkinesis-asl compile unidoc` + puts "Running 'build/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `build/sbt -Pkinesis-asl compile unidoc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/building-spark.md b/docs/building-spark.md index 70165eabca43..c1bcd91b5b85 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -9,6 +9,15 @@ redirect_from: "building-with-maven.html" Building Spark using Maven requires Maven 3.0.4 or newer and Java 6+. +# Building with `build/mvn` + +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: + +{% highlight bash %} +build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package +{% endhighlight %} + +Other build examples can be found below. # Setting up Maven's Memory Usage @@ -28,7 +37,9 @@ If you don't run this, you may see errors like the following: You can fix this by setting the `MAVEN_OPTS` variable as discussed before. -**Note:** *For Java 8 and above this step is not required.* +**Note:** +* *For Java 8 and above this step is not required.* +* *If using `build/mvn` and `MAVEN_OPTS` were not already set, the script will automate this for you.* # Specifying the Hadoop Version @@ -60,7 +71,7 @@ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". As of Spark 1.3, Spark only supports YARN versions 2.2.0 and later. +You can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". Spark only supports YARN versions 2.2.0 and later. Examples: @@ -84,7 +95,7 @@ mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 0.13.1 bindings. You can also build for +By default Spark will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using the `-Phive-0.12.0` profile. {% highlight bash %} # Apache Hadoop 2.4.X with Hive 13 support @@ -106,7 +117,7 @@ supported in Scala 2.11 builds. # Spark Tests in Maven -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: @@ -124,7 +135,7 @@ We use the scala-maven-plugin which supports incremental and continuous compilat mvn scala:cc -should run continuous compilation (i.e. wait for changes). However, this has not been tested +should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: * it only scans the paths `src/main` and `src/test` (see [docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work @@ -157,9 +168,9 @@ The debian package can then be found under assembly/target. We added the short c Running only Java 8 tests and nothing else. mvn install -DskipTests -Pjava8-tests - -Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. -For these tests to run your system must have a JDK 8 installation. + +Java 8 tests are run when `-Pjava8-tests` profile is enabled, they will run in spite of `-DskipTests`. +For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. # Building for PySpark on YARN @@ -171,7 +182,7 @@ then ship it over to the cluster. We are investigating the exact cause for this. # Packaging without Hadoop Dependencies for YARN -The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. # Building with SBT @@ -182,22 +193,22 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - sbt/sbt -Pyarn -Phadoop-2.3 assembly + build/sbt -Pyarn -Phadoop-2.3 assembly # Testing with SBT -Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: +Some of the tests require Spark to be packaged first, so always run `build/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test + build/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test # Speeding up Compilation with Zinc @@ -206,3 +217,9 @@ compiler. When run locally as a background process, it speeds up builds of Scala like Spark. Developers who regularly recompile Spark with Maven will be the most interested in Zinc. The project site gives instructions for building and running `zinc`; OS X users can install it using `brew install zinc`. + +If using the `build/mvn` package `zinc` will automatically be downloaded and leveraged for all +builds. This process will auto-start after the first time `build/mvn` is called and bind to port +3030 unless the `ZINC_PORT` environment variable is set. The `zinc` process can subsequently be +shut down at any time by running `build/zinc-/bin/zinc -shutdown` and will automatically +restart whenever `build/mvn` is called. diff --git a/docs/configuration.md b/docs/configuration.md index 2c8dea869b09..fa9d311f8506 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -452,7 +452,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedJobs 1000 - How many stages the Spark UI and status APIs remember before garbage + How many jobs the Spark UI and status APIs remember before garbage collecting. @@ -852,6 +852,41 @@ Apart from these, the following properties are also available, and may be useful between nodes leading to flooding the network with those. + + spark.shuffle.io.preferDirectBufs + true + + (Netty only) Off-heap buffers are used to reduce garbage collection during shuffle and cache + block transfer. For environments where off-heap memory is tightly limited, users may wish to + turn this off to force all allocations from Netty to be on-heap. + + + + spark.shuffle.io.numConnectionsPerPeer + 1 + + (Netty only) Connections between hosts are reused in order to reduce connection buildup for + large clusters. For clusters with many hard disks and few hosts, this may result in insufficient + concurrency to saturate all disks, and so users may consider increasing this value. + + + + spark.shuffle.io.maxRetries + 3 + + (Netty only) Fetches that fail due to IO-related exceptions are automatically retried if this is + set to a non-zero value. This retry logic helps stabilize large shuffles in the face of long GC + pauses or transient network connectivity issues. + + + + spark.shuffle.io.retryWait + 5 + + (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying + is simply maxRetries * retryWait, by default 15 seconds. + + #### Scheduling diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index dd73e9dc5444..87dcc58feb49 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -18,7 +18,7 @@ see the guide on [building with maven](building-spark.html#specifying-the-hadoop The table below lists the corresponding `hadoop.version` code for each CDH/HDP release. Note that some Hadoop releases are binary compatible across client versions. This means the pre-built Spark -distribution may "just work" without you needing to compile. That said, we recommend compiling with +distribution may "just work" without you needing to compile. That said, we recommend compiling with the _exact_ Hadoop version you are running to avoid any compatibility errors. @@ -50,7 +50,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. In SBT, the equivalent can be achieved by setting the the `hadoop.version` property: - sbt/sbt -Dhadoop.version=1.0.4 assembly + build/sbt -Dhadoop.version=1.0.4 assembly # Linking Applications to the Hadoop Version @@ -98,11 +98,11 @@ Spark can run in a variety of deployment modes: * Using dedicated set of Spark nodes in your cluster. These nodes should be co-located with your Hadoop installation. -* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and +* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and cores dedicated to Spark on each node. * Run Spark alongside Hadoop using a cluster resource manager, such as YARN or Mesos. -These options are identical for those using CDH and HDP. +These options are identical for those using CDH and HDP. # Inheriting Cluster Configuration @@ -116,5 +116,5 @@ The location of these configuration files varies across CDH and HDP versions, bu a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create configurations on-the-fly, but offer a mechanisms to download copies of them. -To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` to a location containing the configuration files. diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index dfbb871f01d2..a5425eb3557b 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -98,7 +98,7 @@ pre-packaged distribution. 2. Add this jar to the classpath of all `NodeManager`s in your cluster. 3. In the `yarn-site.xml` on each node, add `spark_shuffle` to `yarn.nodemanager.aux-services`, then set `yarn.nodemanager.aux-services.spark_shuffle.class` to -`org.apache.spark.yarn.network.YarnShuffleService`. Additionally, set all relevant +`org.apache.spark.network.yarn.YarnShuffleService`. Additionally, set all relevant `spark.shuffle.service.*` [configurations](configuration.html). 4. Restart all `NodeManager`s in your cluster. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 86276b1aa9ce..da1c8e8aa866 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -211,18 +211,18 @@ settings and a restart of all node managers. Thus, this is not applicable to hos To use a custom log4j configuration for the application master or executors, there are two options: -- upload a custom log4j.properties using spark-submit, by adding it to the "--files" list of files +- upload a custom `log4j.properties` using `spark-submit`, by adding it to the `--files` list of files to be uploaded with the application. -- add "-Dlog4j.configuration=" to "spark.driver.extraJavaOptions" - (for the driver) or "spark.executor.extraJavaOptions" (for executors). Note that if using a file, - the "file:" protocol should be explicitly provided, and the file needs to exist locally on all +- add `-Dlog4j.configuration=` to `spark.driver.extraJavaOptions` + (for the driver) or `spark.executor.extraJavaOptions` (for executors). Note that if using a file, + the `file:` protocol should be explicitly provided, and the file needs to exist locally on all the nodes. Note that for the first option, both executors and the application master will share the same log4j configuration, which may cause issues when they run on the same node (e.g. trying to write to the same log file). -If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use "${spark.yarn.app.container.log.dir}" in your log4j.properties. For example, log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log. For streaming application, configuring RollingFileAppender and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. +If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use `spark.yarn.app.container.log.dir` in your log4j.properties. For example, `log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log`. For streaming application, configuring `RollingFileAppender` and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. # Important notes diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2aea8a8aedaf..729045b81a8c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -831,13 +831,10 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using -the in-memory columnar format, and therefore `sqlContext.cacheTable(...)` is strongly recommended for this use case. - Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -1010,12 +1007,11 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that -partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed. -To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: +**NOTE:** `CACHE TABLE tbl` is now __eager__ by default not __lazy__. Don’t need to trigger cache materialization manually anymore. - CACHE TABLE logs_last_month; - SELECT COUNT(1) FROM logs_last_month; +Spark SQL newly introduced a statement to let user control table caching whether or not lazy since Spark 1.2.0: + + CACHE [LAZY] TABLE [AS SELECT] ... Several caching related features are not supported yet: diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 1c956fcb40da..4378521dcac7 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -4,7 +4,7 @@ title: Spark Streaming + Kafka Integration Guide --- [Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 1ac5b9e863ad..01450efe35e5 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -75,7 +75,7 @@ main entry point for all streaming functionality. We create a local StreamingCon {% highlight scala %} import org.apache.spark._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Create a local StreamingContext with two working thread and batch interval of 1 second. // The master requires 2 cores to prevent from a starvation scenario. @@ -107,7 +107,7 @@ each line will be split into multiple words and the stream of words is represent `words` DStream. Next, we want to count these words. {% highlight scala %} -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ // not necessary in Spark 1.3+ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 2581c9f69fa3..3bd1deaccfaf 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -10,7 +10,7 @@ through a uniform interface so you don't have to configure your application spec # Bundling Your Application's Dependencies If your code depends on other projects, you will need to package them alongside your application in order to distribute the code to a Spark cluster. To do this, -to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +create an assembly jar (or "uber" jar) containing your code and its dependencies. Both [sbt](https://github.com/sbt/sbt-assembly) and [Maven](http://maven.apache.org/plugins/maven-shade-plugin/) have assembly plugins. When creating assembly jars, list Spark and Hadoop @@ -59,7 +59,7 @@ for applications that involve the REPL (e.g. Spark shell). Alternatively, if your application is submitted from a machine far from the worker machines (e.g. locally on your laptop), it is common to use `cluster` mode to minimize network latency between the drivers and the executors. Note that `cluster` mode is currently not supported for standalone -clusters, Mesos clusters, or python applications. +clusters, Mesos clusters, or Python applications. For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. @@ -174,7 +174,7 @@ This can use up a significant amount of space over time and will need to be clea is handled automatically, and with Spark standalone, automatic cleanup can be configured with the `spark.worker.cleanup.appDataTtl` property. -For python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries +For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries to executors. # More Information diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 556d99d1027b..485eea4f5e68 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -39,10 +39,10 @@ from optparse import OptionParser from sys import stderr -DEFAULT_SPARK_VERSION = "1.1.0" +DEFAULT_SPARK_VERSION = "1.2.0" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) -MESOS_SPARK_EC2_BRANCH = "v4" +MESOS_SPARK_EC2_BRANCH = "branch-1.3" # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) @@ -251,10 +251,13 @@ def get_spark_shark_version(opts): "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", + # These are dummy versions (no Shark versions after this) "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0", + "1.1.1": "1.1.1", + "1.2.0": "1.2.0", } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index c9e1511278ed..2adc63f7ff30 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -22,7 +22,6 @@ import java.util.Properties import kafka.producer._ import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 973049b95a7b..1b53f3edbe92 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -28,20 +28,15 @@ object BroadcastTest { val bcName = if (args.length > 2) args(2) else "Http" val blockSize = if (args.length > 3) args(3) else "4096" - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + - "BroadcastFactory") - System.setProperty("spark.broadcast.blockSize", blockSize) val sparkConf = new SparkConf().setAppName("Broadcast Test") - + .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory") + .set("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(sparkConf) val slices = if (args.length > 0) args(0).toInt else 2 val num = if (args.length > 1) args(1).toInt else 1000000 - val arr1 = new Array[Int](num) - for (i <- 0 until arr1.length) { - arr1(i) = i - } + val arr1 = (0 until num).toArray for (i <- 0 until 3) { println("Iteration " + i) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 9cc47327f9a4..409721b01c8f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -33,6 +33,10 @@ object Analytics extends Logging { if (args.length < 2) { System.err.println( "Usage: Analytics --numEPart= [other options]") + System.err.println("Supported 'taskType' as follows:") + System.err.println(" pagerank Compute PageRank") + System.err.println(" cc Compute the connected components of vertices") + System.err.println(" triangles Count the number of triangles") System.exit(1) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 6bb659fbd8be..30269a7ccae9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -23,7 +23,6 @@ import java.net.Socket import org.apache.spark.{SparkConf, Logging} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.receiver.Receiver /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 6c24bc3ad09e..4b4667fec44e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ /** * Counts words in new text files created in the given directory diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index e4283e04a1b1..6ff0c47793a2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -22,7 +22,6 @@ import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index ae0a08c6cdb1..2cd8073dada1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.storage.StorageLevel /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index 4caa90659111..13ba9a43ec3c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.SynchronizedQueue import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ object QueueStream { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 19427e629f76..c3a05c89d817 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -25,7 +25,6 @@ import com.google.common.io.Files import org.apache.spark.SparkConf import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.util.IntParam /** diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index ed186ea5650c..345d0bc44135 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -20,7 +20,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.SparkConf import org.apache.spark.HashPartitioner import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index 683752ac9624..62f49530edb1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -23,7 +23,6 @@ import org.apache.spark.SparkConf import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.twitter._ // scalastyle:off diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index f55d23ab3924..f253d75b279f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -18,7 +18,6 @@ package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} -import StreamingContext._ import org.apache.spark.SparkContext._ import org.apache.spark.streaming.twitter._ import org.apache.spark.SparkConf diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 79905af381a1..6510c70bd186 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -24,7 +24,6 @@ import akka.zeromq.Subscribe import akka.util.ByteString import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ import scala.language.implicitConversions diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index 55226c0a6df6..fbacaee98690 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.examples.streaming.StreamingExamples // scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/extras/java8-tests/README.md b/extras/java8-tests/README.md index e95b73ac7702..dc9e87f2eeb9 100644 --- a/extras/java8-tests/README.md +++ b/extras/java8-tests/README.md @@ -8,7 +8,7 @@ to your Java location. The set-up depends a bit on the build system: `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically include the Java 8 test project. - `$ JAVA_HOME=/opt/jdk1.8.0/ sbt/sbt clean "test-only org.apache.spark.Java8APISuite"` + `$ JAVA_HOME=/opt/jdk1.8.0/ build/sbt clean "test-only org.apache.spark.Java8APISuite"` * For Maven users, @@ -19,6 +19,6 @@ to your Java location. The set-up depends a bit on the build system: `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests` `$ JAVA_HOME=/opt/jdk1.8.0/ mvn test -Pjava8-tests -DwildcardSuites=org.apache.spark.Java8APISuite` - Note that the above command can only be run from project root directory since this module - depends on core and the test-jars of core and streaming. This means an install step is + Note that the above command can only be run from project root directory since this module + depends on core and the test-jars of core and streaming. This means an install step is required to make the test dependencies visible to the Java 8 sub-project. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 409cf60977f6..906d42328fcb 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -129,44 +129,45 @@ private[impl] case class EdgeWithLocalIds[@specialized ED]( srcId: VertexId, dstId: VertexId, localSrcId: Int, localDstId: Int, attr: ED) private[impl] object EdgeWithLocalIds { - implicit def lexicographicOrdering[ED] = new Ordering[EdgeWithLocalIds[ED]] { - override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = { - if (a.srcId == b.srcId) { - if (a.dstId == b.dstId) 0 - else if (a.dstId < b.dstId) -1 + implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] = + new Ordering[EdgeWithLocalIds[ED]] { + override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = { + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 + else 1 + } else if (a.srcId < b.srcId) -1 else 1 - } else if (a.srcId < b.srcId) -1 - else 1 + } } - } - private[graphx] def edgeArraySortDataFormat[ED] - = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { - override def getKey( - data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { - data(pos) - } + private[graphx] def edgeArraySortDataFormat[ED] = { + new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { + override def getKey(data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { + data(pos) + } - override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { - val tmp = data(pos0) - data(pos0) = data(pos1) - data(pos1) = tmp - } + override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } - override def copyElement( - src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { - dst(dstPos) = src(srcPos) - } + override def copyElement( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } - override def copyRange( - src: Array[EdgeWithLocalIds[ED]], srcPos: Int, - dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { - System.arraycopy(src, srcPos, dst, dstPos, length) - } + override def copyRange( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } - override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { - new Array[EdgeWithLocalIds[ED]](length) + override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { + new Array[EdgeWithLocalIds[ED]](length) + } } } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index 5412d720475d..aa320088f208 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -74,8 +74,8 @@ object ShippableVertexPartition { * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a * `ShippableVertexPartition`. */ - implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) = - new ShippableVertexPartitionOps(partition) + implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) + : ShippableVertexPartitionOps[VD] = new ShippableVertexPartitionOps(partition) /** * Implicit evidence that `ShippableVertexPartition` is a member of the diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala index 55c7a19d1bda..fbe53acfc32a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala @@ -38,8 +38,8 @@ private[graphx] object VertexPartition { * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a * `VertexPartition`. */ - implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) = - new VertexPartitionOps(partition) + implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) + : VertexPartitionOps[VD] = new VertexPartitionOps(partition) /** * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor` diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index b40aa1b417a0..4fd2548b7faf 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -238,8 +238,8 @@ private[graphx] abstract class VertexPartitionBaseOps * because these methods return a `Self` and this implicit conversion re-wraps that in a * `VertexPartitionBaseOps`. This relies on the context bound on `Self`. */ - private implicit def toOps[VD2: ClassTag]( - partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = { + private implicit def toOps[VD2: ClassTag](partition: Self[VD2]) + : VertexPartitionBaseOps[VD2, Self] = { implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition) } } 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 6189dce9b27d..7752c1988fdd 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 @@ -24,7 +24,6 @@ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom 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 1af40de2c7fc..ced042e2f96c 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 @@ -28,9 +28,30 @@ import org.apache.spark.rdd.{RDD, UnionRDD} * Evaluator for binary classification. * * @param scoreAndLabels an RDD of (score, label) pairs. + * @param numBins if greater than 0, then the curves (ROC curve, PR curve) computed internally + * will be down-sampled to this many "bins". If 0, no down-sampling will occur. + * This is useful because the curve contains a point for each distinct score + * in the input, and this could be as large as the input itself -- millions of + * points or more, when thousands may be entirely sufficient to summarize + * the curve. After down-sampling, the curves will instead be made of approximately + * `numBins` points instead. Points are made from bins of equal numbers of + * consecutive points. The size of each bin is + * `floor(scoreAndLabels.count() / numBins)`, which means the resulting number + * 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. */ @Experimental -class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging { +class BinaryClassificationMetrics( + val scoreAndLabels: RDD[(Double, Double)], + val numBins: Int) extends Logging { + + require(numBins >= 0, "numBins must be nonnegative") + + /** + * Defaults `numBins` to 0. + */ + def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0) /** Unpersist intermediate RDDs used in the computation. */ def unpersist() { @@ -103,7 +124,39 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions { iter => + + val binnedCounts = + // Only down-sample if bins is > 0 + if (numBins == 0) { + // Use original directly + counts + } else { + val countsSize = counts.count() + // Group the iterator into chunks of about countsSize / numBins points, + // so that the resulting number of bins is about numBins + var grouping = countsSize / numBins + if (grouping < 2) { + // numBins was more than half of the size; no real point in down-sampling to bins + logInfo(s"Curve is too small ($countsSize) for $numBins bins to be useful") + counts + } else { + if (grouping >= Int.MaxValue) { + logWarning( + s"Curve too large ($countsSize) for $numBins bins; capping at ${Int.MaxValue}") + grouping = Int.MaxValue + } + counts.mapPartitions(_.grouped(grouping.toInt).map { pairs => + // The score of the combined point will be just the first one's score + val firstScore = pairs.head._1 + // The point will contain all counts in this chunk + val agg = new BinaryLabelCounter() + pairs.foreach(pair => agg += pair._2) + (firstScore, agg) + }) + } + } + + val agg = binnedCounts.values.mapPartitions { iter => val agg = new BinaryLabelCounter() iter.foreach(agg += _) Iterator(agg) @@ -113,7 +166,7 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) val totalCount = partitionwiseCumulativeCounts.last logInfo(s"Total counts: $totalCount") - val cumulativeCounts = counts.mapPartitionsWithIndex( + val cumulativeCounts = binnedCounts.mapPartitionsWithIndex( (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => { val cumCount = partitionwiseCumulativeCounts(index) iter.map { case (score, c) => 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 7960f3cab576..d25a7cd5b439 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 @@ -71,7 +71,8 @@ class Word2Vec extends Serializable with Logging { private var numPartitions = 1 private var numIterations = 1 private var seed = Utils.random.nextLong() - + private var minCount = 5 + /** * Sets vector size (default: 100). */ @@ -114,6 +115,15 @@ class Word2Vec extends Serializable with Logging { this } + /** + * Sets minCount, the minimum number of times a token must appear to be included in the word2vec + * model's vocabulary (default: 5). + */ + def setMinCount(minCount: Int): this.type = { + this.minCount = minCount + this + } + private val EXP_TABLE_SIZE = 1000 private val MAX_EXP = 6 private val MAX_CODE_LENGTH = 40 @@ -122,9 +132,6 @@ class Word2Vec extends Serializable with Logging { /** context words from [-window, window] */ private val window = 5 - /** minimum frequency to consider a vocabulary word */ - private val minCount = 5 - private var trainWordsCount = 0 private var vocabSize = 0 private var vocab: Array[VocabWord] = null 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 327366a1a3a8..5a7281ec6dc3 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 @@ -17,9 +17,11 @@ package org.apache.spark.mllib.linalg -import java.util.{Random, Arrays} +import java.util.{Arrays, Random} -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} +import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHashSet, ArrayBuffer} + +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} /** * Trait for a local matrix. @@ -80,6 +82,16 @@ sealed trait Matrix extends Serializable { /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() + + /** Map the values of this matrix using a function. Generates a new matrix. Performs the + * function on only the backing array. For example, an operation such as addition or + * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ + private[mllib] def map(f: Double => Double): Matrix + + /** Update all the values of this matrix using the function f. Performed in-place on the + * backing array. For example, an operation such as addition or subtraction will only be + * performed on the non-zero values in a `SparseMatrix`. */ + private[mllib] def update(f: Double => Double): Matrix } /** @@ -123,6 +135,122 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) } override def copy = new DenseMatrix(numRows, numCols, values.clone()) + + private[mllib] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f)) + + private[mllib] def update(f: Double => Double): DenseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + /** Generate a `SparseMatrix` from the given `DenseMatrix`. */ + def toSparse(): SparseMatrix = { + val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble + val colPtrs: Array[Int] = new Array[Int](numCols + 1) + val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt + var nnz = 0 + var j = 0 + while (j < numCols) { + var i = 0 + val indStart = j * numRows + while (i < numRows) { + val v = values(indStart + i) + if (v != 0.0) { + rowIndices += i + spVals += v + nnz += 1 + } + i += 1 + } + j += 1 + colPtrs(j) = nnz + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result()) + } +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.DenseMatrix]]. + */ +object DenseMatrix { + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @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 + */ + def zeros(numRows: Int, numCols: Int): DenseMatrix = + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @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 + */ + def ones(numRows: Int, numCols: Int): DenseMatrix = + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + + /** + * 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 + */ + def eye(n: Int): DenseMatrix = { + val identity = DenseMatrix.zeros(n, n) + var i = 0 + while (i < n) { + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * @param numRows number of rows of the matrix + * @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) + */ + def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @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) + */ + def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @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 + */ + def diag(vector: Vector): DenseMatrix = { + val n = vector.size + val matrix = DenseMatrix.zeros(n, n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } } /** @@ -156,6 +284,8 @@ class SparseMatrix( require(colPtrs.length == numCols + 1, "The length of the column indices should be the " + s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " + s"numCols: $numCols") + require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " + + s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}") override def toArray: Array[Double] = { val arr = new Array[Double](numRows * numCols) @@ -188,7 +318,7 @@ class SparseMatrix( private[mllib] def update(i: Int, j: Int, v: Double): Unit = { val ind = index(i, j) - if (ind == -1){ + if (ind == -1) { throw new NoSuchElementException("The given row and column indices correspond to a zero " + "value. Only non-zero elements in Sparse Matrices can be updated.") } else { @@ -197,6 +327,192 @@ class SparseMatrix( } override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) + + private[mllib] def map(f: Double => Double) = + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f)) + + private[mllib] def update(f: Double => Double): SparseMatrix = { + val len = values.length + var i = 0 + while (i < len) { + values(i) = f(values(i)) + i += 1 + } + this + } + + /** Generate a `DenseMatrix` from the given `SparseMatrix`. */ + def toDense(): DenseMatrix = { + new DenseMatrix(numRows, numCols, toArray) + } +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.SparseMatrix]]. + */ +object SparseMatrix { + + /** + * Generate a `SparseMatrix` from Coordinate List (COO) format. Input must be an array of + * (i, j, value) tuples. Entries that have duplicate values of i and j are + * added together. Tuples where value is equal to zero will be omitted. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @param entries Array of (i, j, value) tuples + * @return The corresponding `SparseMatrix` + */ + 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 + if (sortedEntries.nonEmpty) { + // Since the entries are sorted by column index, we only need to check the first and the last. + for (col <- Seq(sortedEntries.head._2, sortedEntries.last._2)) { + require(col >= 0 && col < numCols, s"Column index out of range [0, $numCols): $col.") + } + } + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = MArrayBuilder.make[Int] + rowIndices.sizeHint(numEntries) + val values = MArrayBuilder.make[Double] + values.sizeHint(numEntries) + var nnz = 0 + var prevCol = 0 + var prevRow = -1 + var prevVal = 0.0 + // Append a dummy entry to include the last one at the end of the loop. + (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) => + if (v != 0) { + if (i == prevRow && j == prevCol) { + prevVal += v + } else { + if (prevVal != 0) { + require(prevRow >= 0 && prevRow < numRows, + s"Row index out of range [0, $numRows): $prevRow.") + nnz += 1 + rowIndices += prevRow + values += prevVal + } + prevRow = i + prevVal = v + while (prevCol < j) { + colPtrs(prevCol + 1) = nnz + prevCol += 1 + } + } + } + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), values.result()) + } + + /** + * 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 + */ + def speye(n: Int): SparseMatrix = { + new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0)) + } + + /** + * Generates the skeleton of a random `SparseMatrix` with a given random number generator. + * The values of the matrix returned are undefined. + */ + private def genRandMatrix( + numRows: Int, + numCols: Int, + density: Double, + rng: Random): SparseMatrix = { + require(numRows > 0, s"numRows must be greater than 0 but got $numRows") + require(numCols > 0, s"numCols must be greater than 0 but got $numCols") + require(density >= 0.0 && density <= 1.0, + s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density") + val size = numRows.toLong * numCols + val expected = size * density + assert(expected < Int.MaxValue, + "The expected number of nonzeros cannot be greater than Int.MaxValue.") + val nnz = math.ceil(expected).toInt + if (density == 0.0) { + new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array[Int](), Array[Double]()) + } else if (density == 1.0) { + val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows) + val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows) + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](numRows * numCols)) + } else if (density < 0.34) { + // draw-by-draw, expected number of iterations is less than 1.5 * nnz + val entries = MHashSet[(Int, Int)]() + while (entries.size < nnz) { + entries += ((rng.nextInt(numRows), rng.nextInt(numCols))) + } + SparseMatrix.fromCOO(numRows, numCols, entries.map(v => (v._1, v._2, 1.0))) + } else { + // selection-rejection method + var idx = 0L + var numSelected = 0 + var j = 0 + val colPtrs = new Array[Int](numCols + 1) + val rowIndices = new Array[Int](nnz) + while (j < numCols && numSelected < nnz) { + var i = 0 + while (i < numRows && numSelected < nnz) { + if (rng.nextDouble() < 1.0 * (nnz - numSelected) / (size - idx)) { + rowIndices(numSelected) = i + numSelected += 1 + } + i += 1 + idx += 1 + } + colPtrs(j + 1) = numSelected + j += 1 + } + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](nnz)) + } + } + + /** + * Generate a `SparseMatrix` consisting of i.i.d. uniform random numbers. The number of non-zero + * elements equal the ceiling of `numRows` x `numCols` x `density` + * + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @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) + */ + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextDouble()) + } + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @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) + */ + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = { + val mat = genRandMatrix(numRows, numCols, density, rng) + mat.update(i => rng.nextGaussian()) + } + + /** + * Generate a diagonal matrix in `SparseMatrix` format from the supplied values. + * @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 + */ + def diag(vector: Vector): SparseMatrix = { + val n = vector.size + vector match { + case sVec: SparseVector => + SparseMatrix.fromCOO(n, n, sVec.indices.zip(sVec.values).map(v => (v._1, v._1, v._2))) + case dVec: DenseVector => + val entries = dVec.values.zipWithIndex + val nnzVals = entries.filter(v => v._1 != 0.0) + SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1))) + } + } } /** @@ -256,72 +572,250 @@ object Matrices { * Generate a `DenseMatrix` consisting of zeros. * @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 + * @return `Matrix` with size `numRows` x `numCols` and values of zeros */ - def zeros(numRows: Int, numCols: Int): Matrix = - new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(numRows, numCols) /** * Generate a `DenseMatrix` consisting of ones. * @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 + * @return `Matrix` with size `numRows` x `numCols` and values of ones */ - def ones(numRows: Int, numCols: Int): Matrix = - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols) /** - * Generate an Identity Matrix in `DenseMatrix` format. + * Generate a dense Identity Matrix in `Matrix` 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 + * @return `Matrix` with size `n` x `n` and values of ones on the diagonal */ - def eye(n: Int): Matrix = { - val identity = Matrices.zeros(n, n) - var i = 0 - while (i < n){ - identity.update(i, i, 1.0) - i += 1 - } - identity - } + 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 + */ + def speye(n: Int): Matrix = SparseMatrix.speye(n) /** * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. * @param numRows number of rows of the matrix * @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) + * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1) */ - def rand(numRows: Int, numCols: Int, rng: Random): Matrix = { - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) - } + def rand(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.rand(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @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) + */ + def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprand(numRows, numCols, density, rng) /** * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. * @param numRows number of rows of the matrix * @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) + * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1) */ - def randn(numRows: Int, numCols: Int, rng: Random): Matrix = { - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) - } + def randn(numRows: Int, numCols: Int, rng: Random): Matrix = + DenseMatrix.randn(numRows, numCols, rng) + + /** + * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @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) + */ + def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix = + SparseMatrix.sprandn(numRows, numCols, density, rng) /** * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. * @param vector a `Vector` tat will form the values on the diagonal of the matrix - * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * @return Square `Matrix` with size `values.length` x `values.length` and `values` * on the diagonal */ - def diag(vector: Vector): Matrix = { - val n = vector.size - val matrix = Matrices.eye(n) - val values = vector.toArray - var i = 0 - while (i < n) { - matrix.update(i, i, values(i)) - i += 1 + def diag(vector: Vector): Matrix = DenseMatrix.diag(vector) + + /** + * Horizontally concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * 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 + */ + def horzcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array[Double]()) + } else if (matrices.size == 1) { + return matrices(0) + } + val numRows = matrices(0).numRows + var hasSparse = false + var numCols = 0 + matrices.foreach { mat => + require(numRows == mat.numRows, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => hasSparse = true + case dense: DenseMatrix => // empty on purpose + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numCols += mat.numCols + } + if (!hasSparse) { + new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray)) + } else { + var startCol = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { + case spMat: SparseMatrix => + var j = 0 + val colPtrs = spMat.colPtrs + val rowIndices = spMat.rowIndices + val values = spMat.values + val data = new Array[(Int, Int, Double)](values.length) + val nCols = spMat.numCols + while (j < nCols) { + var idx = colPtrs(j) + while (idx < colPtrs(j + 1)) { + val i = rowIndices(idx) + val v = values(idx) + data(idx) = (i, j + startCol, v) + idx += 1 + } + j += 1 + } + startCol += nCols + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + var j = 0 + val nCols = dnMat.numCols + val nRows = dnMat.numRows + val values = dnMat.values + while (j < nCols) { + var i = 0 + val indStart = j * nRows + while (i < nRows) { + val v = values(indStart + i) + if (v != 0.0) { + data.append((i, j + startCol, v)) + } + i += 1 + } + j += 1 + } + startCol += nCols + data + } + SparseMatrix.fromCOO(numRows, numCols, entries) + } + } + + /** + * Vertically concatenate a sequence of matrices. The returned matrix will be in the format + * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in + * 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 + */ + def vertcat(matrices: Array[Matrix]): Matrix = { + if (matrices.isEmpty) { + return new DenseMatrix(0, 0, Array[Double]()) + } else if (matrices.size == 1) { + return matrices(0) + } + val numCols = matrices(0).numCols + var hasSparse = false + var numRows = 0 + matrices.foreach { mat => + require(numCols == mat.numCols, "The number of rows of the matrices in this sequence, " + + "don't match!") + mat match { + case sparse: SparseMatrix => + hasSparse = true + case dense: DenseMatrix => + case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " + + s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}") + } + numRows += mat.numRows + + } + if (!hasSparse) { + val allValues = new Array[Double](numRows * numCols) + var startRow = 0 + matrices.foreach { mat => + var j = 0 + val nRows = mat.numRows + val values = mat.toArray + while (j < numCols) { + var i = 0 + val indStart = j * numRows + startRow + val subMatStart = j * nRows + while (i < nRows) { + allValues(indStart + i) = values(subMatStart + i) + i += 1 + } + j += 1 + } + startRow += nRows + } + new DenseMatrix(numRows, numCols, allValues) + } else { + var startRow = 0 + val entries: Array[(Int, Int, Double)] = matrices.flatMap { + case spMat: SparseMatrix => + var j = 0 + val colPtrs = spMat.colPtrs + val rowIndices = spMat.rowIndices + val values = spMat.values + val data = new Array[(Int, Int, Double)](values.length) + while (j < numCols) { + var idx = colPtrs(j) + while (idx < colPtrs(j + 1)) { + val i = rowIndices(idx) + val v = values(idx) + data(idx) = (i + startRow, j, v) + idx += 1 + } + j += 1 + } + startRow += spMat.numRows + data + case dnMat: DenseMatrix => + val data = new ArrayBuffer[(Int, Int, Double)]() + var j = 0 + val nCols = dnMat.numCols + val nRows = dnMat.numRows + val values = dnMat.values + while (j < nCols) { + var i = 0 + val indStart = j * nRows + while (i < nRows) { + val v = values(indStart + i) + if (v != 0.0) { + data.append((i + startRow, j, v)) + } + i += 1 + } + j += 1 + } + startRow += nRows + data + } + SparseMatrix.fromCOO(numRows, numCols, entries) } - matrix } } 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 47d1a76fa361..6a782b079aac 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 @@ -268,7 +268,7 @@ object Vectors { * @param p norm. * @return norm in L^p^ space. */ - private[spark] def norm(vector: Vector, p: Double): Double = { + def norm(vector: Vector, p: Double): Double = { require(p >= 1.0) val values = vector match { case dv: DenseVector => dv.values @@ -312,6 +312,86 @@ object Vectors { math.pow(sum, 1.0 / p) } } + + /** + * Returns the squared distance between two Vectors. + * @param v1 first Vector. + * @param v2 second Vector. + * @return squared distance between two Vectors. + */ + def sqdist(v1: Vector, v2: Vector): Double = { + var squaredDistance = 0.0 + (v1, v2) match { + case (v1: SparseVector, v2: SparseVector) => + val v1Values = v1.values + val v1Indices = v1.indices + val v2Values = v2.values + val v2Indices = v2.indices + val nnzv1 = v1Indices.size + val nnzv2 = v2Indices.size + + var kv1 = 0 + var kv2 = 0 + while (kv1 < nnzv1 || kv2 < nnzv2) { + var score = 0.0 + + if (kv2 >= nnzv2 || (kv1 < nnzv1 && v1Indices(kv1) < v2Indices(kv2))) { + score = v1Values(kv1) + kv1 += 1 + } else if (kv1 >= nnzv1 || (kv2 < nnzv2 && v2Indices(kv2) < v1Indices(kv1))) { + score = v2Values(kv2) + kv2 += 1 + } else { + score = v1Values(kv1) - v2Values(kv2) + kv1 += 1 + kv2 += 1 + } + squaredDistance += score * score + } + + case (v1: SparseVector, v2: DenseVector) if v1.indices.length / v1.size < 0.5 => + squaredDistance = sqdist(v1, v2) + + case (v1: DenseVector, v2: SparseVector) if v2.indices.length / v2.size < 0.5 => + squaredDistance = sqdist(v2, v1) + + // When a SparseVector is approximately dense, we treat it as a DenseVector + case (v1, v2) => + squaredDistance = v1.toArray.zip(v2.toArray).foldLeft(0.0){ (distance, elems) => + val score = elems._1 - elems._2 + distance + score * score + } + } + squaredDistance + } + + /** + * Returns the squared distance between DenseVector and SparseVector. + */ + private[mllib] def sqdist(v1: SparseVector, v2: DenseVector): Double = { + var kv1 = 0 + var kv2 = 0 + val indices = v1.indices + var squaredDistance = 0.0 + var iv1 = indices(kv1) + val nnzv2 = v2.size + + while (kv2 < nnzv2) { + var score = 0.0 + if (kv2 != iv1) { + score = v2(kv2) + } else { + score = v1.values(kv1) - v2(kv2) + if (kv1 < indices.length - 1) { + kv1 += 1 + iv1 = indices(kv1) + } + } + squaredDistance += score * score + kv2 += 1 + } + squaredDistance + } } /** 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 5c1acca0ec53..36d8cadd2bdd 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 @@ -142,7 +142,7 @@ class IndexedRowMatrix( val mat = BDM.zeros[Double](m, n) rows.collect().foreach { case IndexedRow(rowIndex, vector) => val i = rowIndex.toInt - vector.toBreeze.activeIterator.foreach { case (j, v) => + vector.foreachActive { case (j, v) => mat(i, j) = v } } 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 10a515af8880..a3fca53929ab 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 @@ -588,8 +588,8 @@ class RowMatrix( val n = numCols().toInt val mat = BDM.zeros[Double](m, n) var i = 0 - rows.collect().foreach { v => - v.toBreeze.activeIterator.foreach { case (j, v) => + rows.collect().foreach { vector => + vector.foreachActive { case (j, v) => mat(i, j) = v } i += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 45dbf6044fcc..5a419d164029 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -94,16 +94,16 @@ class LogisticGradient extends Gradient { * :: DeveloperApi :: * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) - * L = 1/n ||A weights-y||^2 + * L = 1/2n ||A weights-y||^2 * See also the documentation for the precise formulation. */ @DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val diff = dot(data, weights) - label - val loss = diff * diff + val loss = diff * diff / 2.0 val gradient = data.copy - scal(2.0 * diff, gradient) + scal(diff, gradient) (gradient, loss) } @@ -113,8 +113,8 @@ class LeastSquaresGradient extends Gradient { weights: Vector, cumGradient: Vector): Double = { val diff = dot(data, weights) - label - axpy(2.0 * diff, data, cumGradient) - diff * diff + axpy(diff, data, cumGradient) + diff * diff / 2.0 } } 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 f9791c657178..8ecd5c6ad93c 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 @@ -45,7 +45,7 @@ class LassoModel ( /** * Train a regression model with L1-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/n ||A weights-y||^2 + regParam ||weights||_1 + * f(weights) = 1/2n ||A weights-y||^2 + regParam ||weights||_1 * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. 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 c8cad773f5ef..076ba35051c9 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 @@ -45,7 +45,7 @@ class RidgeRegressionModel ( /** * Train a regression model with L2-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/n ||A weights-y||^2 + regParam/2 ||weights||^2 + * f(weights) = 1/2n ||A weights-y||^2 + regParam/2 ||weights||^2 * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. 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 8db0442a7a56..b549b7c475fc 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,6 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** 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 73e7e32c6db3..b3e8ed9af8c5 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 @@ -64,13 +64,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val rfModel = rf.run(input) rfModel.trees(0) } - - /** - * Trains a decision tree model over an RDD. This is deprecated because it hides the static - * methods with the same name in Java. - */ - @deprecated("Please use DecisionTree.run instead.", "1.2.0") - def train(input: RDD[LabeledPoint]): DecisionTreeModel = run(input) } object DecisionTree extends Serializable with Logging { 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 b0d05ae33e1b..c7843464a750 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 @@ -19,8 +19,7 @@ package org.apache.spark.mllib.util import scala.reflect.ClassTag -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, - squaredDistance => breezeSquaredDistance} +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext @@ -28,7 +27,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliCellSampler import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext @@ -39,7 +38,7 @@ import org.apache.spark.streaming.dstream.DStream */ object MLUtils { - private[util] lazy val EPSILON = { + private[mllib] lazy val EPSILON = { var eps = 1.0 while ((1.0 + (eps / 2.0)) != 1.0) { eps /= 2.0 @@ -154,10 +153,12 @@ object MLUtils { def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { // TODO: allow to specify label precision and feature precision. val dataStr = data.map { case LabeledPoint(label, features) => - val featureStrings = features.toBreeze.activeIterator.map { case (i, v) => - s"${i + 1}:$v" + val sb = new StringBuilder(label.toString) + features.foreachActive { case (i, v) => + sb += ' ' + sb ++= s"${i + 1}:$v" } - (Iterator(label) ++ featureStrings).mkString(" ") + sb.mkString } dataStr.saveAsTextFile(dir) } @@ -264,7 +265,7 @@ object MLUtils { } Vectors.fromBreeze(vector1) } - + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: @@ -314,12 +315,10 @@ object MLUtils { val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) / (sqDist + EPSILON) if (precisionBound2 > precision) { - // TODO: breezeSquaredDistance is slow, - // so we should replace it with our own implementation. - sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + sqDist = Vectors.sqdist(v1, v2) } } else { - sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) + sqDist = Vectors.sqdist(v1, v2) } sqDist } 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 064263e02cd1..fbc26167ce66 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 @@ -49,6 +49,7 @@ public void tearDown() { 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(" ")), @@ -68,6 +69,7 @@ public void tfIdf() { 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(" ")), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 8a18e2971cab..e0224f960cc4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -124,4 +124,40 @@ class BinaryClassificationMetricsSuite extends FunSuite with MLlibTestSparkConte validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls) } + + test("binary evaluation metrics with downsampling") { + val scoreAndLabels = Seq( + (0.1, 0.0), (0.2, 0.0), (0.3, 1.0), (0.4, 0.0), (0.5, 0.0), + (0.6, 1.0), (0.7, 1.0), (0.8, 0.0), (0.9, 1.0)) + + val scoreAndLabelsRDD = sc.parallelize(scoreAndLabels, 1) + + val original = new BinaryClassificationMetrics(scoreAndLabelsRDD) + val originalROC = original.roc().collect().sorted.toList + // Add 2 for (0,0) and (1,1) appended at either end + assert(2 + scoreAndLabels.size == originalROC.size) + assert( + List( + (0.0, 0.0), (0.0, 0.25), (0.2, 0.25), (0.2, 0.5), (0.2, 0.75), + (0.4, 0.75), (0.6, 0.75), (0.6, 1.0), (0.8, 1.0), (1.0, 1.0), + (1.0, 1.0) + ) == + originalROC) + + val numBins = 4 + + val downsampled = new BinaryClassificationMetrics(scoreAndLabelsRDD, numBins) + val downsampledROC = downsampled.roc().collect().sorted.toList + assert( + // May have to add 1 if the sample factor didn't divide evenly + 2 + (numBins + (if (scoreAndLabels.size % numBins == 0) 0 else 1)) == + downsampledROC.size) + assert( + List( + (0.0, 0.0), (0.2, 0.25), (0.2, 0.75), (0.6, 0.75), (0.8, 1.0), + (1.0, 1.0), (1.0, 1.0) + ) == + downsampledROC) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 322a0e924291..a35d0fe389fd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -43,9 +43,9 @@ class MatricesSuite extends FunSuite { test("sparse matrix construction") { val m = 3 - val n = 2 + val n = 4 val values = Array(1.0, 2.0, 4.0, 5.0) - val colPtrs = Array(0, 2, 4) + val colPtrs = Array(0, 2, 2, 4, 4) val rowIndices = Array(1, 2, 1, 2) val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] assert(mat.numRows === m) @@ -53,6 +53,13 @@ class MatricesSuite extends FunSuite { assert(mat.values.eq(values), "should not copy data") assert(mat.colPtrs.eq(colPtrs), "should not copy data") assert(mat.rowIndices.eq(rowIndices), "should not copy data") + + val entries: Array[(Int, Int, Double)] = Array((2, 2, 3.0), (1, 0, 1.0), (2, 0, 2.0), + (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0)) + + val mat2 = SparseMatrix.fromCOO(m, n, entries) + assert(mat.toBreeze === mat2.toBreeze) + assert(mat2.values.length == 4) } test("sparse matrix construction with wrong number of elements") { @@ -117,6 +124,142 @@ class MatricesSuite extends FunSuite { assert(sparseMat.values(2) === 10.0) } + test("toSparse, toDense") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + + val spMat2 = deMat1.toSparse() + val deMat2 = spMat1.toDense() + + assert(spMat1.toBreeze === spMat2.toBreeze) + assert(deMat1.toBreeze === deMat2.toBreeze) + } + + test("map, update") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val deMat2 = deMat1.map(_ * 2) + val spMat2 = spMat1.map(_ * 2) + deMat1.update(_ * 2) + spMat1.update(_ * 2) + + assert(spMat1.toArray === spMat2.toArray) + assert(deMat1.toArray === deMat2.toArray) + } + + test("horzcat, vertcat, eye, speye") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(0, 1, 1, 2) + + val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) + val deMat1 = new DenseMatrix(m, n, allValues) + val deMat2 = Matrices.eye(3) + val spMat2 = Matrices.speye(3) + val deMat3 = Matrices.eye(2) + val spMat3 = Matrices.speye(2) + + val spHorz = Matrices.horzcat(Array(spMat1, spMat2)) + val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) + val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) + val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) + + val deHorz2 = Matrices.horzcat(Array[Matrix]()) + + assert(deHorz1.numRows === 3) + assert(spHorz2.numRows === 3) + assert(spHorz3.numRows === 3) + assert(spHorz.numRows === 3) + assert(deHorz1.numCols === 5) + assert(spHorz2.numCols === 5) + assert(spHorz3.numCols === 5) + assert(spHorz.numCols === 5) + assert(deHorz2.numRows === 0) + assert(deHorz2.numCols === 0) + assert(deHorz2.toArray.length === 0) + + assert(deHorz1.toBreeze.toDenseMatrix === spHorz2.toBreeze.toDenseMatrix) + assert(spHorz2.toBreeze === spHorz3.toBreeze) + assert(spHorz(0, 0) === 1.0) + assert(spHorz(2, 1) === 5.0) + assert(spHorz(0, 2) === 1.0) + assert(spHorz(1, 2) === 0.0) + assert(spHorz(1, 3) === 1.0) + assert(spHorz(2, 4) === 1.0) + assert(spHorz(1, 4) === 0.0) + assert(deHorz1(0, 0) === 1.0) + assert(deHorz1(2, 1) === 5.0) + assert(deHorz1(0, 2) === 1.0) + assert(deHorz1(1, 2) == 0.0) + assert(deHorz1(1, 3) === 1.0) + assert(deHorz1(2, 4) === 1.0) + assert(deHorz1(1, 4) === 0.0) + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(spMat1, spMat3)) + } + + intercept[IllegalArgumentException] { + Matrices.horzcat(Array(deMat1, spMat3)) + } + + val spVert = Matrices.vertcat(Array(spMat1, spMat3)) + val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) + val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) + val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) + val deVert2 = Matrices.vertcat(Array[Matrix]()) + + assert(deVert1.numRows === 5) + assert(spVert2.numRows === 5) + assert(spVert3.numRows === 5) + assert(spVert.numRows === 5) + assert(deVert1.numCols === 2) + assert(spVert2.numCols === 2) + assert(spVert3.numCols === 2) + assert(spVert.numCols === 2) + assert(deVert2.numRows === 0) + assert(deVert2.numCols === 0) + assert(deVert2.toArray.length === 0) + + assert(deVert1.toBreeze.toDenseMatrix === spVert2.toBreeze.toDenseMatrix) + assert(spVert2.toBreeze === spVert3.toBreeze) + assert(spVert(0, 0) === 1.0) + assert(spVert(2, 1) === 5.0) + assert(spVert(3, 0) === 1.0) + assert(spVert(3, 1) === 0.0) + assert(spVert(4, 1) === 1.0) + assert(deVert1(0, 0) === 1.0) + assert(deVert1(2, 1) === 5.0) + assert(deVert1(3, 0) === 1.0) + assert(deVert1(3, 1) === 0.0) + assert(deVert1(4, 1) === 1.0) + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(spMat1, spMat2)) + } + + intercept[IllegalArgumentException] { + Matrices.vertcat(Array(deMat1, spMat2)) + } + } + test("zeros") { val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix] assert(mat.numRows === 2) @@ -162,4 +305,29 @@ class MatricesSuite extends FunSuite { assert(mat.numCols === 2) assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0)) } + + test("sprand") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0) + val mat = SparseMatrix.sprand(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + val mat2 = SparseMatrix.sprand(2, 3, 1.0, rng) + assert(mat2.rowIndices.toSeq === Seq(0, 1, 0, 1, 0, 1)) + assert(mat2.colPtrs.toSeq === Seq(0, 2, 4, 6)) + } + + test("sprandn") { + val rng = mock[Random] + when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0) + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = SparseMatrix.sprandn(4, 4, 0.25, rng) + assert(mat.numRows === 4) + assert(mat.numCols === 4) + assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1)) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 03b71301e9ab..70b43ddb7daf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -52,7 +52,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data @@ -84,7 +84,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data @@ -118,7 +118,7 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // create model initialized with true weights val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(10.0, 10.0)) - .setStepSize(0.1) + .setStepSize(0.2) .setNumIterations(25) // generate sequence of simulated data for testing diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index df07987093fb..7778847f8b72 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -52,12 +52,27 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext { val values = indices.map(i => a(i)) val v2 = Vectors.sparse(n, indices, values) val norm2 = Vectors.norm(v2, 2.0) + val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5)) + val norm3 = Vectors.norm(v3, 2.0) val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") val fastSquaredDist2 = fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision) assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") + val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze) + val fastSquaredDist3 = + fastSquaredDistance(v2, norm2, v3, norm3, precision) + assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m") + if (m > 10) { + val v4 = Vectors.sparse(n, indices.slice(0, m - 10), + indices.map(i => a(i) + 0.5).slice(0, m - 10)) + val norm4 = Vectors.norm(v4, 2.0) + val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze) + val fastSquaredDist = + fastSquaredDistance(v2, norm2, v4, norm4, precision) + assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 30b906aaa3ba..e957fa5d25f4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -178,17 +178,17 @@ object TestingUtils { implicit class MatrixWithAlmostEquals(val x: Matrix) { /** - * When the difference of two vectors are within eps, returns true; otherwise, returns false. + * When the difference of two matrices are within eps, returns true; otherwise, returns false. */ def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) /** - * When the difference of two vectors are within eps, returns false; otherwise, returns true. + * When the difference of two matrices are within eps, returns false; otherwise, returns true. */ def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) /** - * Throws exception when the difference of two vectors are NOT within eps; + * Throws exception when the difference of two matrices are NOT within eps; * otherwise, returns true. */ def ~==(r: CompareMatrixRightSide): Boolean = { diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 13b37f96f8ce..7c9adf52af0f 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -40,7 +40,7 @@ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } - /** Number of concurrent connections between two nodes for fetching data. **/ + /** Number of concurrent connections between two nodes for fetching data. */ public int numConnectionsPerPeer() { return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1); } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java index dfe0ba059509..93e6fdd7161f 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -37,6 +37,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; /** @@ -49,7 +50,7 @@ * the Executor's memory, unlike the IndexShuffleBlockManager. */ public class ExternalShuffleBlockManager { - private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); + private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); // Map containing all registered executors' metadata. private final ConcurrentMap executors; @@ -60,8 +61,9 @@ public class ExternalShuffleBlockManager { private final TransportConf conf; public ExternalShuffleBlockManager(TransportConf conf) { - // TODO: Give this thread a name. - this(conf, Executors.newSingleThreadExecutor()); + this(conf, 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. diff --git a/pom.xml b/pom.xml index 4be8c220223f..05f59a9b4140 100644 --- a/pom.xml +++ b/pom.xml @@ -149,6 +149,7 @@ 2.10 ${scala.version} org.scala-lang + 1.8.8 @@ -819,10 +820,15 @@ - + org.codehaus.jackson jackson-mapper-asl - 1.8.8 + ${jackson.version} + + + org.codehaus.jackson + jackson-core-asl + ${jackson.version} @@ -1010,6 +1016,9 @@ checkpoint + + lib_managed + diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 230239aa4050..31d4c317ae56 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -53,6 +53,14 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrices.randn"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrices.rand") + ) ++ Seq( + // SPARK-3325 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.api.java.JavaDStreamLike.print"), + // SPARK-2757 + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." + + "removeAndGetProcessor") ) case v if v.startsWith("1.2") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f52074282e1f..46a54c681840 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,6 +15,8 @@ * limitations under the License. */ +import java.io.File + import scala.util.Properties import scala.collection.JavaConversions._ @@ -23,7 +25,7 @@ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.genjavadocSettings import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} +import com.typesafe.sbt.pom.{loadEffectivePom, PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings object BuildCommons { @@ -112,6 +114,17 @@ object SparkBuild extends PomBuild { override val userPropertiesMap = System.getProperties.toMap + // Handle case where hadoop.version is set via profile. + // Needed only because we read back this property in sbt + // when we create the assembly jar. + val pom = loadEffectivePom(new File("pom.xml"), + profiles = profiles, + userProps = userPropertiesMap) + if (System.getProperty("hadoop.version") == null) { + System.setProperty("hadoop.version", + pom.getProperties.get("hadoop.version").asInstanceOf[String]) + } + lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") @@ -153,7 +166,7 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink, networkCommon, networkShuffle, networkYarn).contains(x)).foreach { + networkCommon, networkShuffle, networkYarn).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } @@ -297,8 +310,7 @@ object Assembly { // This must match the same name used in maven (see network/yarn/pom.xml) "spark-" + v + "-yarn-shuffle.jar" } else { - mName + "-" + v + "-hadoop" + - Option(System.getProperty("hadoop.version")).getOrElse("1.0.4") + ".jar" + mName + "-" + v + "-hadoop" + System.getProperty("hadoop.version") + ".jar" } }, mergeStrategy in assembly := { diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 469f82473af9..0e8b398fc6b9 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1671,7 +1671,7 @@ def _ssql_ctx(self): except Py4JError as e: raise Exception("You must build Spark with Hive. " "Export 'SPARK_HIVE=true' and run " - "sbt/sbt assembly", e) + "build/sbt assembly", e) def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) @@ -2085,6 +2085,34 @@ def subtract(self, other, numPartitions=None): else: raise ValueError("Can only subtract another SchemaRDD") + def sample(self, withReplacement, fraction, seed=None): + """ + Return a sampled subset of this SchemaRDD. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.sample(False, 0.5, 97).count() + 2L + """ + assert fraction >= 0.0, "Negative fraction value: %s" % fraction + seed = seed if seed is not None else random.randint(0, sys.maxint) + rdd = self._jschema_rdd.sample(withReplacement, fraction, long(seed)) + return SchemaRDD(rdd, self.sql_ctx) + + def takeSample(self, withReplacement, num, seed=None): + """Return a fixed-size sampled subset of this SchemaRDD. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.takeSample(False, 2, 97) + [Row(field1=3, field2=u'row3'), Row(field1=1, field2=u'row1')] + """ + seed = seed if seed is not None else random.randint(0, sys.maxint) + with SCCallSiteSync(self.context) as css: + bytesInJava = self._jschema_rdd.baseSchemaRDD() \ + .takeSampleToPython(withReplacement, num, long(seed)) \ + .iterator() + cls = _create_cls(self.schema()) + return map(cls, self._collect_iterator_through_file(bytesInJava)) + def _test(): import doctest diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 0826ddc56e84..2fe39392ff08 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -157,18 +157,20 @@ def foreachRDD(self, func): api = self._ssc._jvm.PythonDStream api.callForeachRDD(self._jdstream, jfunc) - def pprint(self): + def pprint(self, num=10): """ - Print the first ten elements of each RDD generated in this DStream. + Print the first num elements of each RDD generated in this DStream. + + @param num: the number of elements from the first will be printed. """ def takeAndPrint(time, rdd): - taken = rdd.take(11) + taken = rdd.take(num + 1) print "-------------------------------------------" print "Time: %s" % time print "-------------------------------------------" - for record in taken[:10]: + for record in taken[:num]: print record - if len(taken) > 10: + if len(taken) > num: print "..." print diff --git a/sbt/sbt b/sbt/sbt index 0a251d97db95..41438251f681 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,111 +1,29 @@ #!/usr/bin/env bash -# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so -# that we can run Hive to generate the golden answer. This is not required for normal development -# or testing. -for i in "$HIVE_HOME"/lib/* -do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i" -done -export HADOOP_CLASSPATH - -realpath () { -( - TARGET_FILE="$1" - - cd "$(dirname "$TARGET_FILE")" - TARGET_FILE="$(basename "$TARGET_FILE")" - - COUNT=0 - while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] - do - TARGET_FILE="$(readlink "$TARGET_FILE")" - cd $(dirname "$TARGET_FILE") - TARGET_FILE="$(basename $TARGET_FILE)" - COUNT=$(($COUNT + 1)) - done - - echo "$(pwd -P)/"$TARGET_FILE"" -) -} - -. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash - - -declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" -declare -r sbt_opts_file=".sbtopts" -declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" - -usage() { - cat < path to global settings/plugins directory (default: ~/.sbt) - -sbt-boot path to shared boot directory (default: ~/.sbt/boot in 0.11 series) - -ivy path to local Ivy repository (default: ~/.ivy2) - -mem set memory options (default: $sbt_mem, which is $(get_mem_opts $sbt_mem)) - -no-share use all local caches; no sharing - -no-global uses global caches, but does not use global ~/.sbt directory. - -jvm-debug Turn on JVM debugging, open at the given port. - -batch Disable interactive mode - - # sbt version (default: from project/build.properties if present, else latest release) - -sbt-version use the specified version of sbt - -sbt-jar use the specified jar as the sbt launcher - -sbt-rc use an RC version of sbt - -sbt-snapshot use a snapshot version of sbt - - # java version (default: java from PATH, currently $(java -version 2>&1 | grep version)) - -java-home alternate JAVA_HOME - - # jvm options and output control - JAVA_OPTS environment variable, if unset uses "$java_opts" - SBT_OPTS environment variable, if unset uses "$default_sbt_opts" - .sbtopts if this file exists in the current directory, it is - prepended to the runner args - /etc/sbt/sbtopts if this file exists, it is prepended to the runner args - -Dkey=val pass -Dkey=val directly to the java runtime - -J-X pass option -X directly to the java runtime - (-J is stripped) - -S-X add -X to sbt's scalacOptions (-S is stripped) - -PmavenProfiles Enable a maven profile for the build. - -In the case of duplicated or conflicting options, the order above -shows precedence: JAVA_OPTS lowest, command line options highest. -EOM -} - -process_my_args () { - while [[ $# -gt 0 ]]; do - case "$1" in - -no-colors) addJava "-Dsbt.log.noformat=true" && shift ;; - -no-share) addJava "$noshare_opts" && shift ;; - -no-global) addJava "-Dsbt.global.base=$(pwd)/project/.sbtboot" && shift ;; - -sbt-boot) require_arg path "$1" "$2" && addJava "-Dsbt.boot.directory=$2" && shift 2 ;; - -sbt-dir) require_arg path "$1" "$2" && addJava "-Dsbt.global.base=$2" && shift 2 ;; - -debug-inc) addJava "-Dxsbt.inc.debug=true" && shift ;; - -batch) exec &2 +echo " Please update references to point to the new location." >&2 +echo "" >&2 +echo " Invoking 'build/sbt $@' now ..." >&2 +echo "" >&2 + +${_DIR}/../build/sbt "$@" diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash deleted file mode 100755 index fa7222d55a6d..000000000000 --- a/sbt/sbt-launch-lib.bash +++ /dev/null @@ -1,188 +0,0 @@ -#!/usr/bin/env bash -# - -# A library to simplify using the SBT launcher from other packages. -# Note: This should be used by tools like giter8/conscript etc. - -# TODO - Should we merge the main SBT script with this library? - -if test -z "$HOME"; then - declare -r script_dir="$(dirname "$script_path")" -else - declare -r script_dir="$HOME/.sbt" -fi - -declare -a residual_args -declare -a java_args -declare -a scalac_args -declare -a sbt_commands -declare -a maven_profiles - -if test -x "$JAVA_HOME/bin/java"; then - echo -e "Using $JAVA_HOME as default JAVA_HOME." - echo "Note, this will be overridden by -java-home if it is set." - declare java_cmd="$JAVA_HOME/bin/java" -else - declare java_cmd=java -fi - -echoerr () { - echo 1>&2 "$@" -} -vlog () { - [[ $verbose || $debug ]] && echoerr "$@" -} -dlog () { - [[ $debug ]] && echoerr "$@" -} - -acquire_sbt_jar () { - SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties` - URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar - URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar - JAR=sbt/sbt-launch-${SBT_VERSION}.jar - - sbt_jar=$JAR - - if [[ ! -f "$sbt_jar" ]]; then - # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f "${JAR}" ]; then - # Download - printf "Attempting to fetch sbt\n" - JAR_DL="${JAR}.part" - if hash curl 2>/dev/null; then - (curl --silent ${URL1} > "${JAR_DL}" || curl --silent ${URL2} > "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" - elif hash wget 2>/dev/null; then - (wget --quiet ${URL1} -O "${JAR_DL}" || wget --quiet ${URL2} -O "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" - else - printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 - fi - fi - if [ ! -f "${JAR}" ]; then - # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" - exit -1 - fi - printf "Launching sbt from ${JAR}\n" - fi -} - -execRunner () { - # print the arguments one to a line, quoting any containing spaces - [[ $verbose || $debug ]] && echo "# Executing command line:" && { - for arg; do - if printf "%s\n" "$arg" | grep -q ' '; then - printf "\"%s\"\n" "$arg" - else - printf "%s\n" "$arg" - fi - done - echo "" - } - - exec "$@" -} - -addJava () { - dlog "[addJava] arg = '$1'" - java_args=( "${java_args[@]}" "$1" ) -} - -enableProfile () { - dlog "[enableProfile] arg = '$1'" - maven_profiles=( "${maven_profiles[@]}" "$1" ) - export SBT_MAVEN_PROFILES="${maven_profiles[@]}" -} - -addSbt () { - dlog "[addSbt] arg = '$1'" - sbt_commands=( "${sbt_commands[@]}" "$1" ) -} -addResidual () { - dlog "[residual] arg = '$1'" - residual_args=( "${residual_args[@]}" "$1" ) -} -addDebugger () { - addJava "-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=$1" -} - -# a ham-fisted attempt to move some memory settings in concert -# so they need not be dicked around with individually. -get_mem_opts () { - local mem=${1:-2048} - local perm=$(( $mem / 4 )) - (( $perm > 256 )) || perm=256 - (( $perm < 4096 )) || perm=4096 - local codecache=$(( $perm / 2 )) - - echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m" -} - -require_arg () { - local type="$1" - local opt="$2" - local arg="$3" - if [[ -z "$arg" ]] || [[ "${arg:0:1}" == "-" ]]; then - echo "$opt requires <$type> argument" 1>&2 - exit 1 - fi -} - -is_function_defined() { - declare -f "$1" > /dev/null -} - -process_args () { - while [[ $# -gt 0 ]]; do - case "$1" in - -h|-help) usage; exit 1 ;; - -v|-verbose) verbose=1 && shift ;; - -d|-debug) debug=1 && shift ;; - - -ivy) require_arg path "$1" "$2" && addJava "-Dsbt.ivy.home=$2" && shift 2 ;; - -mem) require_arg integer "$1" "$2" && sbt_mem="$2" && shift 2 ;; - -jvm-debug) require_arg port "$1" "$2" && addDebugger $2 && shift 2 ;; - -batch) exec LogicalPlan] = - ( ORDER ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, l) } - | SORT ~ BY ~> ordering ^^ { case o => l: LogicalPlan => SortPartitions(o, l) } + ( ORDER ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, true, l) } + | SORT ~ BY ~> ordering ^^ { case o => l: LogicalPlan => Sort(o, false, l) } ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(singleOrder, ",") - | rep1sep(expression, ",") ~ direction.? ^^ { - case exps ~ d => exps.map(SortOrder(_, d.getOrElse(Ascending))) + ( rep1sep(expression ~ direction.? , ",") ^^ { + case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) - protected lazy val singleOrder: Parser[SortOrder] = - expression ~ direction ^^ { case e ~ o => SortOrder(e, o) } - protected lazy val direction: Parser[SortDirection] = ( ASC ^^^ Ascending | DESC ^^^ Descending 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 8407349295dd..ea64e8a66e85 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 @@ -45,11 +45,13 @@ class Analyzer(catalog: Catalog, maxIterations: Int = 100) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { - val resolver = if (conf.getConf(CatalystConf.CASE_SENSITIVE, "true").toBoolean) { - caseSensitiveResolution - } else { - caseInsensitiveResolution - } + def resolver: Resolver = { + if (conf.getConf(CatalystConf.CASE_SENSITIVE, "true").toBoolean) { + caseSensitiveResolution + } else { + caseInsensitiveResolution + } + } val fixedPoint = FixedPoint(maxIterations) @@ -252,7 +254,7 @@ class Analyzer(catalog: Catalog, case p: LogicalPlan if !p.childrenResolved => p // If the projection list contains Stars, expand it. - case p@Project(projectList, child) if containsStar(projectList) => + case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { case s: Star => s.expand(child.output, resolver) @@ -316,7 +318,8 @@ class Analyzer(catalog: Catalog, */ object ResolveSortReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => + case s @ Sort(ordering, global, p @ Project(projectList, child)) + if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolve(_, resolver)) val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) @@ -325,13 +328,14 @@ class Analyzer(catalog: Catalog, if (missingInProject.nonEmpty) { // Add missing attributes and then project them away after the sort. Project(projectList.map(_.toAttribute), - Sort(ordering, + Sort(ordering, global, Project(projectList ++ missingInProject, child))) } else { logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } - case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => + case s @ Sort(ordering, global, a @ Aggregate(grouping, aggs, child)) + if !s.resolved && a.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) // 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. @@ -346,8 +350,7 @@ class Analyzer(catalog: Catalog, if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. Project(a.output, - Sort(ordering, - Aggregate(grouping, aggs ++ missingInAggs, child))) + Sort(ordering, global, Aggregate(grouping, aggs ++ missingInAggs, child))) } else { s // Nothing we can do here. Return original plan. } 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 e38114ab3cf2..242f28f67029 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 @@ -361,6 +361,22 @@ trait HiveTypeCoercion { DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) ) + case LessThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case LessThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + LessThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThan(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThan(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + + case GreaterThanOrEqual(e1 @ DecimalType.Expression(p1, s1), + e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => + GreaterThanOrEqual(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)) + // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b: BinaryExpression if b.left.dataType != b.right.dataType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index fb252cdf5153..8e39f79d2ca5 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.types.decimal.Decimal - import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} @@ -29,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. @@ -119,21 +118,22 @@ package object dsl { def expr = e } - implicit def booleanToLiteral(b: Boolean) = Literal(b) - implicit def byteToLiteral(b: Byte) = Literal(b) - implicit def shortToLiteral(s: Short) = Literal(s) - implicit def intToLiteral(i: Int) = Literal(i) - implicit def longToLiteral(l: Long) = Literal(l) - implicit def floatToLiteral(f: Float) = Literal(f) - implicit def doubleToLiteral(d: Double) = Literal(d) - implicit def stringToLiteral(s: String) = Literal(s) - implicit def dateToLiteral(d: Date) = Literal(d) - implicit def bigDecimalToLiteral(d: BigDecimal) = Literal(d) - implicit def decimalToLiteral(d: Decimal) = Literal(d) - implicit def timestampToLiteral(t: Timestamp) = Literal(t) - implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) - - implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) + implicit def booleanToLiteral(b: Boolean): Literal = Literal(b) + implicit def byteToLiteral(b: Byte): Literal = Literal(b) + implicit def shortToLiteral(s: Short): Literal = Literal(s) + implicit def intToLiteral(i: Int): Literal = Literal(i) + implicit def longToLiteral(l: Long): Literal = Literal(l) + implicit def floatToLiteral(f: Float): Literal = Literal(f) + implicit def doubleToLiteral(d: Double): Literal = Literal(d) + implicit def stringToLiteral(s: String): Literal = Literal(s) + implicit def dateToLiteral(d: Date): Literal = Literal(d) + implicit def bigDecimalToLiteral(d: BigDecimal): Literal = Literal(d) + implicit def decimalToLiteral(d: Decimal): Literal = Literal(d) + implicit def timestampToLiteral(t: Timestamp): Literal = Literal(t) + implicit def binaryToLiteral(a: Array[Byte]): Literal = Literal(a) + + implicit def symbolToUnresolvedAttribute(s: Symbol): analysis.UnresolvedAttribute = + analysis.UnresolvedAttribute(s.name) def sum(e: Expression) = Sum(e) def sumDistinct(e: Expression) = SumDistinct(e) @@ -244,9 +244,9 @@ package object dsl { condition: Option[Expression] = None) = Join(logicalPlan, otherPlan, joinType, condition) - def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, logicalPlan) + def orderBy(sortExprs: SortOrder*) = Sort(sortExprs, true, logicalPlan) - def sortBy(sortExprs: SortOrder*) = SortPartitions(sortExprs, logicalPlan) + def sortBy(sortExprs: SortOrder*) = Sort(sortExprs, false, logicalPlan) def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*) = { val aliasedExprs = aggregateExprs.map { @@ -301,52 +301,52 @@ package object dsl { (1 to 22).map { x => val argTypes = Seq.fill(x)("_").mkString(", ") - s"implicit def functionToUdfBuilder[T: TypeTag](func: Function$x[$argTypes, T]) = ScalaUdfBuilder(func)" + s"implicit def functionToUdfBuilder[T: TypeTag](func: Function$x[$argTypes, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func)" } */ - implicit def functionToUdfBuilder[T: TypeTag](func: Function1[_, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function1[_, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function2[_, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function2[_, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function3[_, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function3[_, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function4[_, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function4[_, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function5[_, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function5[_, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function6[_, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function6[_, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function7[_, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function7[_, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function8[_, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function8[_, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function9[_, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function9[_, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function10[_, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function10[_, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) - implicit def functionToUdfBuilder[T: TypeTag](func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + implicit def functionToUdfBuilder[T: TypeTag](func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): ScalaUdfBuilder[T] = ScalaUdfBuilder(func) // scalastyle:on } 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 94b6fb084d38..cb5ff6795986 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,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.BooleanType @@ -48,6 +47,14 @@ trait PredicateHelper { } } + protected def splitDisjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case Or(cond1, cond2) => + splitDisjunctivePredicates(cond1) ++ splitDisjunctivePredicates(cond2) + case other => other :: Nil + } + } + /** * Returns true if `expr` can be evaluated using only the output of `plan`. This method * can be used to determine when is is acceptable to move expression evaluation within a query 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 806c1394eb15..cd3137980ca4 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 @@ -142,16 +142,16 @@ object ColumnPruning extends Rule[LogicalPlan] { case Project(projectList1, Project(projectList2, child)) => // Create a map of Aliases to their values from the child projection. // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). - val aliasMap = projectList2.collect { - case a @ Alias(e, _) => (a.toAttribute: Expression, a) - }.toMap + val aliasMap = AttributeMap(projectList2.collect { + case a @ Alias(e, _) => (a.toAttribute, a) + }) // Substitute any attributes that are produced by the child projection, so that we safely // eliminate it. // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' // TODO: Fix TransformBase to avoid the cast below. val substitutedProjection = projectList1.map(_.transform { - case a if aliasMap.contains(a) => aliasMap(a) + case a: Attribute if aliasMap.contains(a) => aliasMap(a) }).asInstanceOf[Seq[NamedExpression]] Project(substitutedProjection, child) @@ -294,11 +294,16 @@ object OptimizeIn extends Rule[LogicalPlan] { } /** - * Simplifies boolean expressions where the answer can be determined without evaluating both sides. + * Simplifies boolean expressions: + * + * 1. Simplifies expressions whose answer can be determined without evaluating both sides. + * 2. Eliminates / extracts common factors. + * 3. Removes `Not` operator. + * * Note that this rule can eliminate expressions that might otherwise have been evaluated and thus * is only safe when evaluations of expressions does not result in side effects. */ -object BooleanSimplification extends Rule[LogicalPlan] { +object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { case and @ And(left, right) => @@ -307,7 +312,9 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (l, Literal(true, BooleanType)) => l case (Literal(false, BooleanType), _) => Literal(false) case (_, Literal(false, BooleanType)) => Literal(false) - case (_, _) => and + // a && a && a ... => a + case _ if splitConjunctivePredicates(and).distinct.size == 1 => left + case _ => and } case or @ Or(left, right) => @@ -316,7 +323,19 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (_, Literal(true, BooleanType)) => Literal(true) case (Literal(false, BooleanType), r) => r case (l, Literal(false, BooleanType)) => l - case (_, _) => or + // a || a || a ... => a + case _ if splitDisjunctivePredicates(or).distinct.size == 1 => left + // (a && b && c && ...) || (a && b && d && ...) => a && b && (c || d || ...) + case _ => + val lhsSet = splitConjunctivePredicates(left).toSet + val rhsSet = splitConjunctivePredicates(right).toSet + val common = lhsSet.intersect(rhsSet) + + (lhsSet.diff(common).reduceOption(And) ++ rhsSet.diff(common).reduceOption(And)) + .reduceOption(Or) + .map(_ :: common.toList) + .getOrElse(common.toList) + .reduce(And) } case not @ Not(exp) => 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 a9282b98adfa..0b9f01cbae9e 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 @@ -130,7 +130,16 @@ case class WriteToFile( override def output = child.output } -case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { +/** + * @param order The ordering expressions + * @param global True means global sorting apply for entire data set, + * False means sorting only apply within the partition. + * @param child Child logical plan + */ +case class Sort( + order: Seq[SortOrder], + global: Boolean, + child: LogicalPlan) extends UnaryNode { override def output = child.output } 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 12c409625cac..ce90bed34f6f 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 @@ -51,6 +51,15 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { assert(analyzer(plan).schema.fields(0).dataType === expectedType) } + private def checkComparison(expression: Expression, expectedType: DataType): Unit = { + val plan = Project(Alias(expression, "c")() :: Nil, relation) + val comparison = analyzer(plan).collect { + case Project(Alias(e: BinaryComparison, _) :: Nil, _) => e + }.head + assert(comparison.left.dataType === expectedType) + assert(comparison.right.dataType === expectedType) + } + test("basic operations") { checkType(Add(d1, d2), DecimalType(6, 2)) checkType(Subtract(d1, d2), DecimalType(6, 2)) @@ -67,6 +76,14 @@ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { checkType(Add(Add(d1, d2), Add(d1, d2)), DecimalType(7, 2)) } + test("Comparison operations") { + checkComparison(LessThan(i, d1), DecimalType.Unlimited) + checkComparison(LessThanOrEqual(d1, d2), DecimalType.Unlimited) + checkComparison(GreaterThan(d2, u), DecimalType.Unlimited) + checkComparison(GreaterThanOrEqual(d1, f), DoubleType) + checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) + } + test("bringing in primitive types") { checkType(Add(d1, i), DecimalType(12, 1)) checkType(Add(d1, f), DoubleType) diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java index b751847b464f..f0d079d25b5d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java @@ -35,6 +35,7 @@ protected UserDefinedType() { } public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; + @SuppressWarnings("unchecked") UserDefinedType that = (UserDefinedType) o; return this.sqlType().equals(that.sqlType()); } 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 934c0a17ebe9..a9b99e079037 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 @@ -106,7 +106,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { + implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]): SchemaRDD = { SparkPlan.currentContext.set(self) val attributeSeq = ScalaReflection.attributesFor[A] val schema = StructType.fromAttributes(attributeSeq) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 7baf8ffcef78..80787b61ce1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -214,7 +214,7 @@ class SchemaRDD( * @group Query */ def orderBy(sortExprs: SortOrder*): SchemaRDD = - new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + new SchemaRDD(sqlContext, Sort(sortExprs, true, logicalPlan)) /** * Sorts the results by the given expressions within partition. @@ -227,7 +227,7 @@ class SchemaRDD( * @group Query */ def sortBy(sortExprs: SortOrder*): SchemaRDD = - new SchemaRDD(sqlContext, SortPartitions(sortExprs, logicalPlan)) + new SchemaRDD(sqlContext, Sort(sortExprs, false, logicalPlan)) @deprecated("use limit with integer argument", "1.1.0") def limit(limitExpr: Expression): SchemaRDD = @@ -238,7 +238,6 @@ class SchemaRDD( * {{{ * schemaRDD.limit(10) * }}} - * * @group Query */ def limit(limitNum: Int): SchemaRDD = @@ -437,6 +436,21 @@ class SchemaRDD( }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) } + /** + * Serializes the Array[Row] returned by SchemaRDD's takeSample(), using the same + * format as javaToPython and collectToPython. It is used by pyspark. + */ + private[sql] def takeSampleToPython( + withReplacement: Boolean, + num: Int, + seed: Long): JList[Array[Byte]] = { + val fieldTypes = schema.fields.map(_.dataType) + val pickle = new Pickler + new java.util.ArrayList(this.takeSample(withReplacement, num, seed).map { row => + EvaluatePython.rowToArray(row, fieldTypes) + }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) + } + /** * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value * of base RDD functions that do not change schema. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index ac4844f9b929..5b9c612487ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -218,4 +218,10 @@ class JavaSchemaRDD( */ def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD = this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD + + /** + * Return a SchemaRDD with a sampled version of the underlying dataset. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaSchemaRDD = + this.baseSchemaRDD.sample(withReplacement, fraction, seed).toJavaSchemaRDD } 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 2954d4ce7d2d..ce878c137e62 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 @@ -33,6 +33,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object LeftSemiJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) + if sqlContext.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + val semiJoin = joins.BroadcastLeftSemiJoinHash( + leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, semiJoin)).getOrElse(semiJoin) :: Nil // Find left semi joins where at least some predicates can be evaluated by matching join keys case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) => val semiJoin = joins.LeftSemiJoinHash( @@ -190,7 +196,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => + case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => execution.TakeOrdered(limit, order, planLater(child)) :: Nil case _ => Nil } @@ -257,15 +263,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Distinct(partial = false, execution.Distinct(partial = true, planLater(child))) :: Nil - case logical.Sort(sortExprs, child) if sqlContext.externalSortEnabled => - execution.ExternalSort(sortExprs, global = true, planLater(child)):: Nil - case logical.Sort(sortExprs, child) => - execution.Sort(sortExprs, global = true, planLater(child)):: Nil - case logical.SortPartitions(sortExprs, child) => // This sort only sorts tuples within a partition. Its requiredDistribution will be // an UnspecifiedDistribution. execution.Sort(sortExprs, global = false, planLater(child)) :: Nil + case logical.Sort(sortExprs, global, child) if sqlContext.externalSortEnabled => + execution.ExternalSort(sortExprs, global, planLater(child)):: Nil + case logical.Sort(sortExprs, global, child) => + execution.Sort(sortExprs, global, planLater(child)):: Nil case logical.Project(projectList, child) => execution.Project(projectList, planLater(child)) :: Nil case logical.Filter(condition, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index ef3687e69296..9049eb5932b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -130,7 +130,7 @@ private[parquet] object RowReadSupport { private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] var writer: RecordConsumer = null - private[parquet] var attributes: Seq[Attribute] = null + private[parquet] var attributes: Array[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) @@ -138,7 +138,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) if (attributes == null) { - attributes = ParquetTypesConverter.convertFromString(origAttributesStr) + attributes = ParquetTypesConverter.convertFromString(origAttributesStr).toArray } log.debug(s"write support initialized for requested schema $attributes") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 0e6fb57d57bc..97447871a11e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -24,8 +24,8 @@ import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter +import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} import parquet.hadoop.util.ContextUtil @@ -458,7 +458,7 @@ private[parquet] object ParquetTypesConverter extends Logging { // ... and fallback to "_metadata" if no such file exists (which implies the Parquet file is // empty, thus normally the "_metadata" file is expected to be fairly small). .orElse(children.find(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE)) - .map(ParquetFileReader.readFooter(conf, _)) + .map(ParquetFileReader.readFooter(conf, _, ParquetMetadataConverter.NO_FILTER)) .getOrElse( throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala deleted file mode 100644 index 806b73fc549d..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestCaseInsensitiveSQLContext.scala +++ /dev/null @@ -1,38 +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.test - -import org.apache.spark.sql.{SQLConf, SQLContext} -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.catalyst.CatalystConf - -/** A case insensitive SQLContext that can be used for local testing. */ -object TestCaseInsensitiveSQLContext - extends SQLContext( - new SparkContext( - "local[2]", - "CaseInsensitiveSQLContext", - new SparkConf())) { - - this.setConf(CatalystConf.CASE_SENSITIVE, "false") - - /** Fewer partitions to speed up testing. */ - override private[spark] def numShufflePartitions: Int = - getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt -} - diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java index bc5cd66482ad..2b5812159d07 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -141,6 +141,7 @@ public void constructComplexRow() { doubleValue, stringValue, timestampValue, null); // Complex array + @SuppressWarnings("unchecked") List> arrayOfMaps = Arrays.asList(simpleMap); List arrayOfRows = Arrays.asList(simpleStruct); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index e40d034ce4dc..c0b9cf516312 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl._ import org.apache.spark.sql.test.TestSQLContext._ +import scala.language.postfixOps + class DslQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -86,7 +88,7 @@ class DslQuerySuite extends QueryTest { Seq(Seq(6))) } - test("sorting") { + test("global sorting") { checkAnswer( testData2.orderBy('a.asc, 'b.asc), Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) @@ -120,22 +122,31 @@ class DslQuerySuite extends QueryTest { mapData.collect().sortBy(_.data(1)).reverse.toSeq) } - test("sorting #2") { + test("partition wide sorting") { + // 2 partitions totally, and + // Partition #1 with values: + // (1, 1) + // (1, 2) + // (2, 1) + // Partition #2 with values: + // (2, 2) + // (3, 1) + // (3, 2) checkAnswer( testData2.sortBy('a.asc, 'b.asc), Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) checkAnswer( testData2.sortBy('a.asc, 'b.desc), - Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1))) + Seq((1,2), (1,1), (2,1), (2,2), (3,2), (3,1))) checkAnswer( testData2.sortBy('a.desc, 'b.desc), - Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1))) + Seq((2,1), (1,2), (1,1), (3,2), (3,1), (2,2))) checkAnswer( testData2.sortBy('a.desc, 'b.asc), - Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + Seq((2,1), (1,1), (1,2), (3,1), (3,2), (2,2))) } test("limit") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 0378fd7e367f..1a4232dab86e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -48,6 +48,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: LeftSemiJoinBNL => j case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j + case j: BroadcastLeftSemiJoinHash => j } assert(operators.size === 1) @@ -382,4 +383,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { """.stripMargin), (null, 10) :: Nil) } + + test("broadcasted left semi join operator selection") { + clearCache() + sql("CACHE TABLE testData") + val tmp = autoBroadcastJoinThreshold + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=1000000000") + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", + classOf[BroadcastLeftSemiJoinHash]) + ).foreach { + case (query, joinClass) => assertJoin(query, joinClass) + } + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") + + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]) + ).foreach { + case (query, joinClass) => assertJoin(query, joinClass) + } + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, tmp.toString) + sql("UNCACHE TABLE testData") + } + + test("left semi join") { + val rdd = sql("SELECT * FROM testData2 LEFT SEMI JOIN testData ON key = a") + checkAnswer(rdd, + (1, 1) :: + (1, 2) :: + (2, 1) :: + (2, 2) :: + (3, 1) :: + (3, 2) :: Nil) + + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala deleted file mode 100644 index c21a5d9ca60d..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryCaseInsensitiveSuite.scala +++ /dev/null @@ -1,55 +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 - -import java.util.TimeZone - -import org.apache.spark.sql.test.TestCaseInsensitiveSQLContext -import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.catalyst.CatalystConf - -/* Implicits */ - -import org.apache.spark.sql.test.TestCaseInsensitiveSQLContext._ - -object CaseInsensitiveTestData{ - case class StringData(s: String) - val table = TestCaseInsensitiveSQLContext.sparkContext.parallelize(StringData("test") :: Nil) - table.registerTempTable("caseInsensitiveTable") -} - -class SQLQueryCaseInsensitiveSuite extends QueryTest with BeforeAndAfterAll { - CaseInsensitiveTestData - - var origZone: TimeZone = _ - - override protected def beforeAll() { - origZone = TimeZone.getDefault - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) - } - - override protected def afterAll() { - TimeZone.setDefault(origZone) - } - - test("SPARK-4699 case sensitivity SQL query") { - setConf(CatalystConf.CASE_SENSITIVE, "false") - checkAnswer(sql("SELECT S FROM CASEINSENSITIVETABLE"), "test") - } - -} 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 ddf4776ecf7a..2e20242c6169 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 @@ -19,10 +19,10 @@ package org.apache.spark.sql import java.util.TimeZone -import org.scalatest.BeforeAndAfterAll - import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.CatalystConf +import org.scalatest.BeforeAndAfterAll /* Implicits */ import org.apache.spark.sql.TestData._ @@ -987,6 +987,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ) } + test("oder by asc by default when not specify ascending and descending") { + checkAnswer( + sql("SELECT a, b FROM testData2 ORDER BY a desc, b"), + Seq((3, 1), (3, 2), (2, 1), (2,2), (1, 1), (1, 2)) + ) + } + test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) @@ -1005,4 +1012,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { rdd.registerTempTable("distinctData") checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), 2) } + + test("SPARK-4699 case sensitivity SQL query") { + setConf(CatalystConf.CASE_SENSITIVE, "false") + val data = TestData(1,"val_1") :: TestData(2,"val_2") :: Nil + val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) + rdd.registerTempTable("testTable1") + checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), "val_1") + setConf(CatalystConf.CASE_SENSITIVE, "true") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index bb553a0a1e50..497897c3c0d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -55,7 +55,7 @@ object TestData { TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil).toSchemaRDD + TestData2(3, 2) :: Nil, 2).toSchemaRDD testData2.registerTempTable("testData2") case class DecimalData(a: BigDecimal, b: BigDecimal) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 82afa31a99a7..1915c25392f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -105,7 +105,9 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be test(query) { val schemaRdd = sql(query) - assertResult(expectedQueryResult.toArray, "Wrong query result") { + val queryExecution = schemaRdd.queryExecution + + assertResult(expectedQueryResult.toArray, s"Wrong query result: $queryExecution") { schemaRdd.collect().map(_.head).toArray } @@ -113,8 +115,10 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head - assert(readBatches === expectedReadBatches, "Wrong number of read batches") - assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") + assert(readBatches === expectedReadBatches, s"Wrong number of read batches: $queryExecution") + assert( + readPartitions === expectedReadPartitions, + s"Wrong number of read partitions: $queryExecution") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index b17300475b6f..4c3a04506ce4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -28,11 +28,14 @@ import org.apache.spark.sql.{QueryTest, SQLConf, SchemaRDD} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. * - * Notice that `!(a cmp b)` are always transformed to its negated form `a cmp' b` by the - * `BooleanSimplification` optimization rule whenever possible. As a result, predicate `!(a < 1)` - * results a `GtEq` filter predicate rather than a `Not`. + * NOTE: * - * @todo Add test cases for `IsNull` and `IsNotNull` after merging PR #3367 + * 1. `!(a cmp b)` is always transformed to its negated form `a cmp' b` by the + * `BooleanSimplification` optimization rule whenever possible. As a result, predicate `!(a < 1)` + * results in a `GtEq` filter predicate rather than a `Not`. + * + * 2. `Tuple1(Option(x))` is used together with `AnyVal` types like `Int` to ensure the inferred + * data type is nullable. */ class ParquetFilterSuite extends QueryTest with ParquetTest { val sqlContext = TestSQLContext @@ -85,14 +88,26 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - boolean") { - withParquetRDD((true :: false :: Nil).map(Tuple1.apply)) { rdd => + withParquetRDD((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Boolean]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Boolean]]) { + Seq(Row(true), Row(false)) + } + checkFilterPushdown(rdd, '_1)('_1 === true, classOf[Eq[java.lang.Boolean]])(true) - checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.NotEq[java.lang.Boolean]])(false) + checkFilterPushdown(rdd, '_1)('_1 !== true, classOf[Operators.NotEq[java.lang.Boolean]]) { + false + } } } test("filter pushdown - integer") { - withParquetRDD((1 to 4).map(Tuple1.apply)) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[Integer]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[Integer]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[Integer]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[Integer]]) { (2 to 4).map(Row.apply(_)) @@ -118,7 +133,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - long") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toLong))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toLong)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Long]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Long]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Long]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Long]]) { (2 to 4).map(Row.apply(_)) @@ -144,7 +164,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - float") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toFloat))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Float]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Float]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Float]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Float]]) { (2 to 4).map(Row.apply(_)) @@ -170,7 +195,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } test("filter pushdown - double") { - withParquetRDD((1 to 4).map(i => Tuple1(i.toDouble))) { rdd => + withParquetRDD((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.Double]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.Double]]) { + (1 to 4).map(Row.apply(_)) + } + checkFilterPushdown(rdd, '_1)('_1 === 1, classOf[Eq[java.lang.Double]])(1) checkFilterPushdown(rdd, '_1)('_1 !== 1, classOf[Operators.NotEq[java.lang.Double]]) { (2 to 4).map(Row.apply(_)) @@ -197,6 +227,11 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { test("filter pushdown - string") { withParquetRDD((1 to 4).map(i => Tuple1(i.toString))) { rdd => + checkFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.String]])(Seq.empty[Row]) + checkFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.String]]) { + (1 to 4).map(i => Row.apply(i.toString)) + } + checkFilterPushdown(rdd, '_1)('_1 === "1", classOf[Eq[String]])("1") checkFilterPushdown(rdd, '_1)('_1 !== "1", classOf[Operators.NotEq[String]]) { (2 to 4).map(i => Row.apply(i.toString)) @@ -227,6 +262,11 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { } withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { rdd => + checkBinaryFilterPushdown(rdd, '_1)('_1.isNull, classOf[Eq[java.lang.String]])(Seq.empty[Row]) + checkBinaryFilterPushdown(rdd, '_1)('_1.isNotNull, classOf[NotEq[java.lang.String]]) { + (1 to 4).map(i => Row.apply(i.b)).toSeq + } + checkBinaryFilterPushdown(rdd, '_1)('_1 === 1.b, classOf[Eq[Array[Byte]]])(1.b) checkBinaryFilterPushdown(rdd, '_1)('_1 !== 1.b, classOf[Operators.NotEq[Array[Byte]]]) { (2 to 4).map(i => Row.apply(i.b)).toSeq diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 074855389d74..a5fe2e8da284 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import scala.reflect.ClassTag + import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} @@ -459,11 +461,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("make RecordFilter for simple predicates") { - def checkFilter[T <: FilterPredicate](predicate: Expression, defined: Boolean = true): Unit = { + def checkFilter[T <: FilterPredicate : ClassTag]( + predicate: Expression, + defined: Boolean = true): Unit = { val filter = ParquetFilters.createFilter(predicate) if (defined) { assert(filter.isDefined) - assert(filter.get.isInstanceOf[T]) + val tClass = implicitly[ClassTag[T]].runtimeClass + val filterGet = filter.get + assert( + tClass.isInstance(filterGet), + s"$filterGet of type ${filterGet.getClass} is not an instance of $tClass") } else { assert(filter.isEmpty) } @@ -484,7 +492,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA checkFilter[Operators.And]('a.int === 1 && 'a.int < 4) checkFilter[Operators.Or]('a.int === 1 || 'a.int < 4) - checkFilter[Operators.Not](!('a.int === 1)) + checkFilter[Operators.NotEq[Integer]](!('a.int === 1)) checkFilter('a.int > 'b.int, defined = false) checkFilter(('a.int > 'b.int) && ('a.int > 'b.int), defined = false) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index 6ed8fd2768f9..7a3d76c61c3a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala @@ -60,7 +60,7 @@ private[hive] abstract class AbstractSparkSQLDriver( } catch { case cause: Throwable => logError(s"Failed in [$command]", cause) - new CommandProcessorResponse(0, ExceptionUtils.getFullStackTrace(cause), null) + new CommandProcessorResponse(1, ExceptionUtils.getFullStackTrace(cause), null) } } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 1e44dd239458..23283fd3fe6b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -101,6 +101,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_comment_nonascii", "create_merge_compressed", + "create_view", "create_view_partitioned", "database_location", "database_properties", @@ -110,7 +111,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Weird DDL differences result in failures on jenkins. "create_like2", - "create_view_translate", "partitions_json", // This test is totally fine except that it includes wrong queries and expects errors, but error @@ -349,6 +349,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_nested_type", "create_skewed_table1", "create_struct_table", + "create_view_translate", "cross_join", "cross_product_check_1", "cross_product_check_2", 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 2a68b5267fc3..1f7ef89feefb 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 @@ -289,7 +289,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. */ - protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { + protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = synchronized { try { val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") 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 e1147478130d..a156d6f7e285 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 @@ -105,9 +105,9 @@ import scala.collection.JavaConversions._ {{{ public interface PrimitiveObjectInspector { // Java Primitives (java.lang.Integer, java.lang.String etc.) - Object getPrimitiveWritableObject(Object o); + Object getPrimitiveJavaObject(Object o); // Writables (hadoop.io.IntWritable, hadoop.io.Text etc.) - Object getPrimitiveJavaObject(Object o); + Object getPrimitiveWritableObject(Object o); // ObjectInspector only inspect the `writable` always return true, we need to check it // before invoking the methods above. boolean preferWritable(); 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 3f3d9e7cd4fb..8a9613cf96e5 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 @@ -680,16 +680,16 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val withSort = (orderByClause, sortByClause, distributeByClause, clusterByClause) match { case (Some(totalOrdering), None, None, None) => - Sort(totalOrdering.getChildren.map(nodeToSortOrder), withHaving) + Sort(totalOrdering.getChildren.map(nodeToSortOrder), true, withHaving) case (None, Some(perPartitionOrdering), None, None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), withHaving) + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, withHaving) case (None, None, Some(partitionExprs), None) => Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving) case (None, Some(perPartitionOrdering), Some(partitionExprs), None) => - SortPartitions(perPartitionOrdering.getChildren.map(nodeToSortOrder), + Sort(perPartitionOrdering.getChildren.map(nodeToSortOrder), false, Repartition(partitionExprs.getChildren.map(nodeToExpr), withHaving)) case (None, None, None, Some(clusterExprs)) => - SortPartitions(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), + Sort(clusterExprs.getChildren.map(nodeToExpr).map(SortOrder(_, Ascending)), false, Repartition(clusterExprs.getChildren.map(nodeToExpr), withHaving)) case (None, None, None, None) => withHaving case _ => sys.error("Unsupported set of ordering / distribution clauses.") 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 93b6ef9fbc59..7d863f9d89da 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 @@ -158,11 +158,6 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def foldable = isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector] - @transient - protected def constantReturnValue = unwrap( - returnInspector.asInstanceOf[ConstantObjectInspector].getWritableConstantValue(), - returnInspector) - @transient protected lazy val deferedObjects = argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] @@ -171,7 +166,6 @@ private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, childr override def eval(input: Row): Any = { returnInspector // Make sure initialized. - if(foldable) return constantReturnValue var i = 0 while (i < children.length) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala index abed299cd957..2a16c9d1a27c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.io.Writable * when "spark.sql.hive.convertMetastoreParquet" is set to true. */ @deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " + - "placeholder in the Hive MetaStore") + "placeholder in the Hive MetaStore", "1.2.0") class FakeParquetSerDe extends SerDe { override def getObjectInspector: ObjectInspector = new ObjectInspector { override def getCategory: Category = Category.PRIMITIVE diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java index d2d39a8c4dc2..808e2986d3b7 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java @@ -23,25 +23,21 @@ public class UDFListListInt extends UDF { /** - * * @param obj - * SQL schema: array> - * Java Type: List> - * @return + * SQL schema: array<struct<x: int, y: int, z: int>> + * Java Type: List<List<Integer>> */ + @SuppressWarnings("unchecked") public long evaluate(Object obj) { if (obj == null) { - return 0l; + return 0L; } - List listList = (List) obj; + List> listList = (List>) obj; long retVal = 0; - for (List aList : listList) { - @SuppressWarnings("unchecked") - List list = (List) aList; - @SuppressWarnings("unchecked") - Integer someInt = (Integer) list.get(1); + for (List aList : listList) { + Number someInt = (Number) aList.get(1); try { - retVal += (long) (someInt.intValue()); + retVal += someInt.longValue(); } catch (NullPointerException e) { System.out.println(e); } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index bfe608a51a30..f90d3607915a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.sql.Date import java.util +import java.util.{Locale, TimeZone} import org.apache.hadoop.hive.serde2.io.DoubleWritable import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory @@ -63,6 +64,11 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { .get()) } + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + val data = Literal(true) :: Literal(0.asInstanceOf[Byte]) :: @@ -121,11 +127,11 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { row1.zip(row2).map { - case (r1, r2) => checkValues(r1, r2) + case (r1, r2) => checkValue(r1, r2) } } - def checkValues(v1: Any, v2: Any): Unit = { + def checkValue(v1: Any, v2: Any): Unit = { (v1, v2) match { case (r1: Decimal, r2: Decimal) => // Ignore the Decimal precision @@ -195,26 +201,26 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { }) checkValues(row, unwrap(wrap(row, toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) } test("wrap / unwrap Array Type") { val dt = ArrayType(dataTypes(0)) val d = row(0) :: row(0) :: Nil - checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) } test("wrap / unwrap Map Type") { val dt = MapType(dataTypes(0), dataTypes(1)) val d = Map(row(0) -> row(1)) - checkValues(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) - checkValues(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValues(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) - checkValues(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) + checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) + checkValue(d, unwrap(wrap(d, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) + checkValue(d, unwrap(wrap(null, toInspector(Literal(d, dt))), toInspector(Literal(d, dt)))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index ff4071d8e2f1..4b6a9308b981 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.BeforeAndAfterAll import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.execution._ @@ -193,4 +193,52 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { ) } + test("auto converts to broadcast left semi join, by size estimate of a relation") { + val leftSemiJoinQuery = + """SELECT * FROM src a + |left semi JOIN src b ON a.key=86 and a.key = b.key""".stripMargin + val answer = (86, "val_86") :: Nil + + var rdd = sql(leftSemiJoinQuery) + + // Assert src has a size smaller than the threshold. + val sizes = rdd.queryExecution.analyzed.collect { + case r if implicitly[ClassTag[MetastoreRelation]].runtimeClass + .isAssignableFrom(r.getClass) => + r.statistics.sizeInBytes + } + assert(sizes.size === 2 && sizes(1) <= autoBroadcastJoinThreshold + && sizes(0) <= autoBroadcastJoinThreshold, + s"query should contain two relations, each of which has size smaller than autoConvertSize") + + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + var bhj = rdd.queryExecution.sparkPlan.collect { + case j: BroadcastLeftSemiJoinHash => j + } + assert(bhj.size === 1, + s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + + checkAnswer(rdd, answer) // check correctness of output + + TestHive.settings.synchronized { + val tmp = autoBroadcastJoinThreshold + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1") + rdd = sql(leftSemiJoinQuery) + bhj = rdd.queryExecution.sparkPlan.collect { + case j: BroadcastLeftSemiJoinHash => j + } + assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") + + val shj = rdd.queryExecution.sparkPlan.collect { + case j: LeftSemiJoinHash => j + } + assert(shj.size === 1, + "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned off") + + sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp") + } + + } } 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 8011f9b8773b..4104df8f8e02 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 @@ -132,7 +132,7 @@ abstract class HiveComparisonTest def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case PhysicalOperation(_, _, Sort(_, _)) => true + case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4d81acc753a2..fb6da33e88ef 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -56,6 +56,13 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { Locale.setDefault(originalLocale) } + test("SPARK-4908: concurent hive native commands") { + (1 to 100).par.map { _ => + sql("USE default") + sql("SHOW TABLES") + } + } + createQueryTest("constant object inspector for generic udf", """SELECT named_struct( lower("AA"), "10", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index a0ace91060a2..16f77a438e1a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.{Row, SchemaRDD} +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.Row import org.apache.spark.util.Utils @@ -76,4 +77,15 @@ class HiveTableScanSuite extends HiveComparisonTest { === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null))) TestHive.sql("DROP TABLE timestamp_query_null") } + + test("Spark-4959 Attributes are case sensitive when using a select query from a projection") { + sql("create table spark_4959 (col1 string)") + sql("""insert into table spark_4959 select "hi" from src limit 1""") + table("spark_4959").select( + 'col1.as('CaseSensitiveColName), + 'col1.as('CaseSensitiveColName2)).registerTempTable("spark_4959_2") + + assert(sql("select CaseSensitiveColName from spark_4959_2").first() === Row("hi")) + assert(sql("select casesensitivecolname from spark_4959_2").first() === Row("hi")) + } } 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 fe9fcd9dbd05..7d26ec4746a3 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 @@ -32,6 +32,13 @@ case class Nested3(f3: Int) * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("SPARK-4512 Fix attribute reference resolution error when using SORT BY") { + checkAnswer( + sql("SELECT * FROM (SELECT key + key AS a FROM src SORT BY value) t ORDER BY t.a"), + sql("SELECT key + key as a FROM src ORDER BY a").collect().toSeq + ) + } + test("CTAS with serde") { sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect sql( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala index a0aeacbc733b..fdbbe2aa6ef0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala @@ -17,30 +17,63 @@ package org.apache.spark.streaming +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock + private[streaming] class ContextWaiter { + + private val lock = new ReentrantLock() + private val condition = lock.newCondition() + + // Guarded by "lock" private var error: Throwable = null - private var stopped: Boolean = false - def notifyError(e: Throwable) = synchronized { - error = e - notifyAll() - } + // Guarded by "lock" + private var stopped: Boolean = false - def notifyStop() = synchronized { - stopped = true - notifyAll() + def notifyError(e: Throwable): Unit = { + lock.lock() + try { + error = e + condition.signalAll() + } finally { + lock.unlock() + } } - def waitForStopOrError(timeout: Long = -1) = synchronized { - // If already had error, then throw it - if (error != null) { - throw error + def notifyStop(): Unit = { + lock.lock() + try { + stopped = true + condition.signalAll() + } finally { + lock.unlock() } + } - // If not already stopped, then wait - if (!stopped) { - if (timeout < 0) wait() else wait(timeout) + /** + * Return `true` if it's stopped; or throw the reported error if `notifyError` has been called; or + * `false` if the waiting time detectably elapsed before return from the method. + */ + def waitForStopOrError(timeout: Long = -1): Boolean = { + lock.lock() + try { + if (timeout < 0) { + while (!stopped && error == null) { + condition.await() + } + } else { + var nanos = TimeUnit.MILLISECONDS.toNanos(timeout) + while (!stopped && error == null && nanos > 0) { + nanos = condition.awaitNanos(nanos) + } + } + // If already had error, then throw it if (error != null) throw error + // already stopped or timeout + stopped + } finally { + lock.unlock() } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ecab5510a8e7..8ef078713784 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.mutable.Queue -import scala.language.implicitConversions import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} @@ -523,9 +522,11 @@ object StreamingContext extends Logging { private[streaming] val DEFAULT_CLEANER_TTL = 3600 - implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + @deprecated("Replaced by implicit functions in the DStream companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") + def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { - new PairDStreamFunctions[K, V](stream) + DStream.toPairDStreamFunctions(stream)(kt, vt, ord) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index e35a568ddf11..9697437dd2fe 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -29,9 +29,17 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { private val streamingListener = ssc.progressListener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, - defaultValue: T) { + defaultValue: T): Unit = { + registerGaugeWithOption[T](name, + (l: StreamingJobProgressListener) => Option(f(streamingListener)), defaultValue) + } + + private def registerGaugeWithOption[T]( + name: String, + f: StreamingJobProgressListener => Option[T], + defaultValue: T): Unit = { metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] { - override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue) + override def getValue: T = f(streamingListener).getOrElse(defaultValue) }) } @@ -41,6 +49,12 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for number of total completed batches registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L) + // Gauge for number of total received records + registerGauge("totalReceivedRecords", _.numTotalReceivedRecords, 0L) + + // Gauge for number of total processed records + registerGauge("totalProcessedRecords", _.numTotalProcessedRecords, 0L) + // Gauge for number of unprocessed batches registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L) @@ -55,19 +69,30 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { // Gauge for last completed batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. - registerGauge("lastCompletedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) - registerGauge("lastCompletedBatch_processStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) - registerGauge("lastCompletedBatch_processEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + registerGaugeWithOption("lastCompletedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime), -1L) + registerGaugeWithOption("lastCompletedBatch_processingStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + registerGaugeWithOption("lastCompletedBatch_processingEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + + // Gauge for last completed batch's delay information. + registerGaugeWithOption("lastCompletedBatch_processingDelay", + _.lastCompletedBatch.flatMap(_.processingDelay), -1L) + registerGaugeWithOption("lastCompletedBatch_schedulingDelay", + _.lastCompletedBatch.flatMap(_.schedulingDelay), -1L) + registerGaugeWithOption("lastCompletedBatch_totalDelay", + _.lastCompletedBatch.flatMap(_.totalDelay), -1L) // Gauge for last received batch, useful for monitoring the streaming job's running status, // displayed data -1 for any abnormal condition. - registerGauge("lastReceivedBatch_submissionTime", - _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) - registerGauge("lastReceivedBatch_processStartTime", - _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) - registerGauge("lastReceivedBatch_processEndTime", - _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + registerGaugeWithOption("lastReceivedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime), -1L) + registerGaugeWithOption("lastReceivedBatch_processingStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime), -1L) + registerGaugeWithOption("lastReceivedBatch_processingEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime), -1L) + + // Gauge for last received batch records. + registerGauge("lastReceivedBatch_records", _.lastReceivedBatchRecords.values.sum, 0L) } 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 2a7004e56ef5..e0542eda1383 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 @@ -51,7 +51,15 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * operator, so this DStream will be registered as an output stream and there materialized. */ def print(): Unit = { - dstream.print() + print(10) + } + + /** + * Print the first num elements of each RDD generated in this DStream. This is an output + * operator, so this DStream will be registered as an output stream and there materialized. + */ + def print(num: Int): Unit = { + dstream.print(num) } /** 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 bb44b906d738..de124cf40eff 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 @@ -36,7 +36,6 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** @@ -815,6 +814,6 @@ object JavaPairDStream { def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long]) : JavaPairDStream[K, JLong] = { - StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) + DStream.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index dbf1ebbaf653..28fc00cf3944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -20,8 +20,8 @@ package org.apache.spark.streaming.dstream import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import scala.deprecated import scala.collection.mutable.HashMap +import scala.language.implicitConversions import scala.reflect.ClassTag import scala.util.matching.Regex @@ -29,7 +29,7 @@ import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext.rddToFileName import org.apache.spark.streaming.scheduler.Job import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} @@ -48,8 +48,7 @@ import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} * `window`. In addition, [[org.apache.spark.streaming.dstream.PairDStreamFunctions]] contains * operations available only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and * `join`. These operations are automatically available on any DStream of pairs - * (e.g., DStream[(Int, Int)] through implicit conversions when - * `org.apache.spark.streaming.StreamingContext._` is imported. + * (e.g., DStream[(Int, Int)] through implicit conversions. * * DStreams internally is characterized by a few basic properties: * - A list of other DStreams that the DStream depends on @@ -606,13 +605,21 @@ abstract class DStream[T: ClassTag] ( * operator, so this DStream will be registered as an output stream and there materialized. */ def print() { + print(10) + } + + /** + * Print the first num elements of each RDD generated in this DStream. This is an output + * operator, so this DStream will be registered as an output stream and there materialized. + */ + def print(num: Int) { def foreachFunc = (rdd: RDD[T], time: Time) => { - val first11 = rdd.take(11) + val firstNum = rdd.take(num + 1) println ("-------------------------------------------") println ("Time: " + time) println ("-------------------------------------------") - first11.take(10).foreach(println) - if (first11.size > 10) println("...") + firstNum.take(num).foreach(println) + if (firstNum.size > num) println("...") println() } new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() @@ -802,10 +809,21 @@ abstract class DStream[T: ClassTag] ( } } -private[streaming] object DStream { +object DStream { + + // `toPairDStreamFunctions` was in SparkContext before 1.3 and users had to + // `import StreamingContext._` to enable it. Now we move it here to make the compiler find + // it automatically. However, we still keep the old function in StreamingContext for backward + // compatibility and forward to the following function directly. + + implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): + PairDStreamFunctions[K, V] = { + new PairDStreamFunctions[K, V](stream) + } /** Get the creation site of a DStream from the stack trace of when the DStream is created. */ - def getCreationSite(): CallSite = { + private[streaming] def getCreationSite(): CallSite = { val SPARK_CLASS_REGEX = """^org\.apache\.spark""".r val SPARK_STREAMING_TESTCLASS_REGEX = """^org\.apache\.spark\.streaming\.test""".r val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 98539e06b4e2..8a5857163244 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -27,12 +27,10 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner, SerializableWritable} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, Time} -import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext.rddToFileName /** * Extra functions available on DStream of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use - * these functions. */ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 1a47089e513c..c0a5af0b65cc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -17,8 +17,6 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext._ - import org.apache.spark.rdd.RDD import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} import org.apache.spark.Partitioner diff --git a/streaming/src/main/scala/org/apache/spark/streaming/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/package.scala index 4dd985cf5a17..2153ae0d3418 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/package.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/package.scala @@ -26,7 +26,7 @@ package org.apache.spark * available only on DStreams * of key-value pairs, such as `groupByKey` and `reduceByKey`. These operations are automatically * available on any DStream of the right type (e.g. DStream[(Int, Int)] through implicit - * conversions when you `import org.apache.spark.streaming.StreamingContext._`. + * conversions. * * For the Java API of Spark Streaming, take a look at the * [[org.apache.spark.streaming.api.java.JavaStreamingContext]] which serves as the entry point, and diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 55765dc90698..79263a718397 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -116,7 +116,7 @@ private[streaming] class BlockGenerator( /** * Push a single data item into the buffer. After buffering the data, the - * `BlockGeneratorListnere.onAddData` callback will be called. All received data items + * `BlockGeneratorListener.onAddData` callback will be called. All received data items * will be periodically pushed into BlockManager. */ def addDataWithCallback(data: Any, metadata: Any) = synchronized { 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 fdf995320beb..f7a8ebee8a54 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 @@ -42,7 +42,7 @@ private[streaming] trait ReceivedBlockHandler { def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult /** Cleanup old blocks older than the given threshold time */ - def cleanupOldBlock(threshTime: Long) + def cleanupOldBlocks(threshTime: Long) } @@ -82,7 +82,7 @@ private[streaming] class BlockManagerBasedBlockHandler( BlockManagerBasedStoreResult(blockId) } - def cleanupOldBlock(threshTime: Long) { + def cleanupOldBlocks(threshTime: Long) { // this is not used as blocks inserted into the BlockManager are cleared by DStream's clearing // of BlockRDDs. } @@ -121,6 +121,24 @@ private[streaming] class WriteAheadLogBasedBlockHandler( private val maxFailures = conf.getInt( "spark.streaming.receiver.writeAheadLog.maxFailures", 3) + private val effectiveStorageLevel = { + if (storageLevel.deserialized) { + logWarning(s"Storage level serialization ${storageLevel.deserialized} is not supported when" + + s" write ahead log is enabled, change to serialization false") + } + if (storageLevel.replication > 1) { + logWarning(s"Storage level replication ${storageLevel.replication} is unnecessary when " + + s"write ahead log is enabled, change to replication 1") + } + + StorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.useOffHeap, false, 1) + } + + if (storageLevel != effectiveStorageLevel) { + logWarning(s"User defined storage level $storageLevel is changed to effective storage level " + + s"$effectiveStorageLevel when write ahead log is enabled") + } + // Manages rolling log files private val logManager = new WriteAheadLogManager( checkpointDirToLogDir(checkpointDir, streamId), @@ -156,7 +174,7 @@ private[streaming] class WriteAheadLogBasedBlockHandler( // Store the block in block manager val storeInBlockManagerFuture = Future { val putResult = - blockManager.putBytes(blockId, serializedBlock, storageLevel, tellMaster = true) + blockManager.putBytes(blockId, serializedBlock, effectiveStorageLevel, tellMaster = true) if (!putResult.map { _._1 }.contains(blockId)) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") @@ -169,16 +187,13 @@ private[streaming] class WriteAheadLogBasedBlockHandler( } // Combine the futures, wait for both to complete, and return the write ahead log segment - val combinedFuture = for { - _ <- storeInBlockManagerFuture - fileSegment <- storeInWriteAheadLogFuture - } yield fileSegment + val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2) val segment = Await.result(combinedFuture, blockStoreTimeout) WriteAheadLogBasedStoreResult(blockId, segment) } - def cleanupOldBlock(threshTime: Long) { - logManager.cleanupOldLogs(threshTime) + def cleanupOldBlocks(threshTime: Long) { + logManager.cleanupOldLogs(threshTime, waitForCompletion = false) } def stop() { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala index bf39d1e891ca..ab9fa192191a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming.receiver /** Messages sent to the NetworkReceiver. */ -private[streaming] sealed trait ReceiverMessage +private[streaming] sealed trait ReceiverMessage extends Serializable private[streaming] object StopReceiver extends ReceiverMessage 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 02758e0bca6c..2ce458cddec1 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 @@ -139,14 +139,17 @@ private[streaming] class ReceivedBlockTracker( getReceivedBlockQueue(streamId).toSeq } - /** Clean up block information of old batches. */ - def cleanupOldBatches(cleanupThreshTime: Time): Unit = synchronized { + /** + * Clean up block information of old batches. If waitForCompletion is true, this method + * returns only after the files are cleaned up. + */ + def cleanupOldBatches(cleanupThreshTime: Time, waitForCompletion: Boolean): Unit = synchronized { assert(cleanupThreshTime.milliseconds < clock.currentTime()) val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq logInfo("Deleting batches " + timesToCleanup) writeToLog(BatchCleanupEvent(timesToCleanup)) timeToAllocatedBlocks --= timesToCleanup - logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds)) + logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds, waitForCompletion)) log } 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 32e481dabc8c..8dbb42a86e3b 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 @@ -121,7 +121,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** Clean up metadata older than the given threshold time */ def cleanupOldMetadata(cleanupThreshTime: Time) { - receivedBlockTracker.cleanupOldBatches(cleanupThreshTime) + receivedBlockTracker.cleanupOldBatches(cleanupThreshTime, waitForCompletion = false) } /** Register a receiver */ @@ -150,8 +150,8 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false logWarning("No prior receiver info") ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) } - receiverInfo(streamId) = newReceiverInfo - listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) + receiverInfo -= streamId + listenerBus.post(StreamingListenerReceiverStopped(newReceiverInfo)) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { 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 f61069b56db5..5ee53a5c5f56 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 @@ -25,7 +25,6 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution -import org.apache.spark.Logging private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) @@ -36,6 +35,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private val completedaBatchInfos = new Queue[BatchInfo] private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L + private var totalReceivedRecords = 0L + private var totalProcessedRecords = 0L private val receiverInfos = new HashMap[Int, ReceiverInfo] val batchDuration = ssc.graph.batchDuration.milliseconds @@ -65,6 +66,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + + batchStarted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => + totalReceivedRecords += infos.map(_.numRecords).sum + } } override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { @@ -73,6 +78,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) completedaBatchInfos.enqueue(batchCompleted.batchInfo) if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() totalCompletedBatches += 1L + + batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => + totalProcessedRecords += infos.map(_.numRecords).sum + } } def numReceivers = synchronized { @@ -83,6 +92,14 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) totalCompletedBatches } + def numTotalReceivedRecords: Long = synchronized { + totalReceivedRecords + } + + def numTotalProcessedRecords: Long = synchronized { + totalProcessedRecords + } + def numUnprocessedBatches: Long = synchronized { waitingBatchInfos.size + runningBatchInfos.size } 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 1353e487c72c..98e9a2e639e2 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 @@ -67,6 +67,12 @@ private[ui] class StreamingPage(parent: StreamingTab)
  • Waiting batches: {listener.numUnprocessedBatches}
  • +
  • + Received records: {listener.numTotalReceivedRecords} +
  • +
  • + Processed records: {listener.numTotalProcessedRecords} +
  • } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala index 70d234320be7..166661b7496d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -19,11 +19,11 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.{Await, ExecutionContext, Future} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.Logging import org.apache.spark.util.Utils import WriteAheadLogManager._ @@ -124,8 +124,12 @@ private[streaming] class WriteAheadLogManager( * files, which is usually based on the local system time. So if there is coordination necessary * between the node calculating the threshTime (say, driver node), and the local system time * (say, worker node), the caller has to take account of possible time skew. + * + * If waitForCompletion is set to true, this method will return only after old logs have been + * deleted. This should be set to true only for testing. Else the files will be deleted + * asynchronously. */ - def cleanupOldLogs(threshTime: Long): Unit = { + def cleanupOldLogs(threshTime: Long, waitForCompletion: Boolean): Unit = { val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") @@ -146,10 +150,15 @@ private[streaming] class WriteAheadLogManager( logInfo(s"Cleared log files in $logDirectory older than $threshTime") } if (!executionContext.isShutdown) { - Future { deleteFiles() } + val f = Future { deleteFiles() } + if (waitForCompletion) { + import scala.concurrent.duration._ + Await.ready(f, 1 second) + } } } + /** Stop the manager, close any open log writer */ def stop(): Unit = synchronized { if (currentLogWriter != null) { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index ce645fccba1d..12cc0de7509d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -57,7 +57,7 @@ public void equalIterable(Iterable a, Iterable b) { @Test public void testInitialization() { - Assert.assertNotNull(ssc.sc()); + Assert.assertNotNull(ssc.sparkContext()); } @SuppressWarnings("unchecked") @@ -662,7 +662,7 @@ public void testStreamingContextTransform(){ listOfDStreams1, new Function2>, Time, JavaRDD>() { public JavaRDD call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 2); + Assert.assertEquals(2, listOfRDDs.size()); return null; } } @@ -675,7 +675,7 @@ public JavaRDD call(List> listOfRDDs, Time time) { listOfDStreams2, new Function2>, Time, JavaPairRDD>>() { public JavaPairRDD> call(List> listOfRDDs, Time time) { - assert(listOfRDDs.size() == 3); + Assert.assertEquals(3, listOfRDDs.size()); JavaRDD rdd1 = (JavaRDD)listOfRDDs.get(0); JavaRDD rdd2 = (JavaRDD)listOfRDDs.get(1); JavaRDD> rdd3 = (JavaRDD>)listOfRDDs.get(2); @@ -969,7 +969,7 @@ public Integer call(Tuple2 in) throws Exception { }); JavaTestUtils.attachTestOutputStream(reversed); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1012,7 +1012,7 @@ public Iterable> call(Tuple2 in) throws } }); JavaTestUtils.attachTestOutputStream(flatMapped); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -1163,9 +1163,9 @@ public void testGroupByKeyAndWindow() { JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); - assert(result.size() == expected.size()); + Assert.assertEquals(expected.size(), result.size()); for (int i = 0; i < result.size(); i++) { - assert(convert(result.get(i)).equals(convert(expected.get(i)))); + Assert.assertEquals(convert(expected.get(i)), convert(result.get(i))); } } @@ -1383,7 +1383,7 @@ public JavaPairRDD call(JavaPairRDD in) thro }); JavaTestUtils.attachTestOutputStream(sorted); - List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 86b96785d7b8..199f5e716112 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} import org.apache.spark.HashPartitioner diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index c97998add8ff..72d055eb2ea3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -30,7 +30,6 @@ import org.apache.hadoop.io.{IntWritable, Text} import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 5dbb7232009e..e0f14fd95428 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import org.apache.spark.Logging import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import org.apache.spark.streaming.StreamingContext._ import scala.util.Random import scala.collection.mutable.ArrayBuffer diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 3661e16a9ef2..132ff2443fc0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -168,7 +168,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche manualClock.currentTime() shouldEqual 5000L val cleanupThreshTime = 3000L - handler.cleanupOldBlock(cleanupThreshTime) + handler.cleanupOldBlocks(cleanupThreshTime) eventually(timeout(10000 millis), interval(10 millis)) { getWriteAheadLogFiles().size should be < preCleanupLogFiles.size } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index 01a09b67b99d..de7e9d624bf6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -166,7 +166,7 @@ class ReceivedBlockTrackerSuite // Cleanup first batch but not second batch val oldestLogFile = getWriteAheadLogFiles().head incrementTime() - tracker3.cleanupOldBatches(batchTime2) + tracker3.cleanupOldBatches(batchTime2, waitForCompletion = true) // Verify that the batch allocations have been cleaned, and the act has been written to log tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index 471c99fab468..a5d2bb2fde16 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream import org.apache.spark.storage.StorageLevel 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 8f69bcb64279..7ce9499dc614 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 @@ -182,15 +182,29 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { } test("WriteAheadLogManager - cleanup old logs") { + logCleanUpTest(waitForCompletion = false) + } + + test("WriteAheadLogManager - cleanup old logs synchronously") { + logCleanUpTest(waitForCompletion = true) + } + + private def logCleanUpTest(waitForCompletion: Boolean): Unit = { // Write data with manager, recover with new manager and verify val manualClock = new ManualClock val dataToWrite = generateRandomData() manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) val logFiles = getLogFilesInDirectory(testDir) assert(logFiles.size > 1) - manager.cleanupOldLogs(manualClock.currentTime() / 2) - eventually(timeout(1 second), interval(10 milliseconds)) { + + manager.cleanupOldLogs(manualClock.currentTime() / 2, waitForCompletion) + + if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } else { + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } } } diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index db58eb642b56..15ee95070a3d 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.util.Utils @@ -49,13 +49,13 @@ object StoragePerfTester { val writeData = "1" * recordLength val executor = Executors.newFixedThreadPool(numMaps) - System.setProperty("spark.shuffle.compress", "false") - System.setProperty("spark.shuffle.sync", "true") - System.setProperty("spark.shuffle.manager", - "org.apache.spark.shuffle.hash.HashShuffleManager") + val conf = new SparkConf() + .set("spark.shuffle.compress", "false") + .set("spark.shuffle.sync", "true") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") // This is only used to instantiate a BlockManager. All thread scheduling is done manually. - val sc = new SparkContext("local[4]", "Write Tester") + val sc = new SparkContext("local[4]", "Write Tester", conf) val hashShuffleManager = sc.env.shuffleManager.asInstanceOf[HashShuffleManager] def writeOutputBytes(mapId: Int, total: AtomicLong) = { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index dc7a07844632..9c77dff48dc8 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -311,7 +311,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def cleanupStagingDir(fs: FileSystem) { var stagingDirPath: Path = null try { - val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean + val preserveFiles = sparkConf.getBoolean("spark.yarn.preserve.staging.files", false) if (!preserveFiles) { stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) if (stagingDirPath == null) { @@ -511,7 +511,7 @@ object ApplicationMaster extends Logging { SignalLogger.register(log) val amArgs = new ApplicationMasterArguments(args) SparkHadoopUtil.get.runAsSparkUser { () => - master = new ApplicationMaster(amArgs, new YarnRMClientImpl(amArgs)) + master = new ApplicationMaster(amArgs, new YarnRMClient(amArgs)) System.exit(master.run()) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index c439969510fc..39f1021c9d94 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -39,6 +39,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var appName: String = "Spark" var priority = 0 + parseArgs(args.toList) + // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", @@ -50,7 +52,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private val isDynamicAllocationEnabled = sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) - parseArgs(args.toList) loadEnvironmentArgs() validateArgs() @@ -197,6 +198,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | to work with. | --files files Comma separated list of files to be distributed with the job. | --archives archives Comma separated list of archives to be distributed with the job. - """ + """.stripMargin } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 5f0c67f05c9d..eb97a7b3c59a 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -693,7 +693,7 @@ private[spark] object ClientBase extends Logging { addClasspathEntry(Environment.PWD.$(), env) // Normally the users app.jar is last in case conflicts with spark jars - if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + if (sparkConf.getBoolean("spark.yarn.user.classpath.first", false)) { addUserClasspath(args, sparkConf, env) addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) populateHadoopClasspath(conf, env) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala deleted file mode 100644 index 2bbf5d7db866..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ /dev/null @@ -1,213 +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.deploy.yarn - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -import org.apache.hadoop.yarn.client.api.AMRMClient -import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -import org.apache.hadoop.yarn.util.Records - -/** - * Acquires resources for executors from a ResourceManager and launches executors in new containers. - */ -private[yarn] class YarnAllocationHandler( - conf: Configuration, - sparkConf: SparkConf, - amClient: AMRMClient[ContainerRequest], - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]], - securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { - - override protected def releaseContainer(container: Container) = { - amClient.releaseAssignedContainer(container.getId()) - } - - // pending isn't used on stable as the AMRMClient handles incremental asks - override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { - addResourceRequests(count) - - // We have already set the container request. Poll the ResourceManager for a response. - // This doubles as a heartbeat if there are no pending container requests. - val progressIndicator = 0.1f - new StableAllocateResponse(amClient.allocate(progressIndicator)) - } - - private def createRackResourceRequests( - hostContainers: ArrayBuffer[ContainerRequest] - ): ArrayBuffer[ContainerRequest] = { - // Generate modified racks and new set of hosts under it before issuing requests. - val rackToCounts = new HashMap[String, Int]() - - for (container <- hostContainers) { - val candidateHost = container.getNodes.last - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += 1 - rackToCounts.put(rack, count) - } - } - - val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts) { - requestedContainers ++= createResourceRequests( - AllocationType.RACK, - rack, - count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers - } - - private def addResourceRequests(numExecutors: Int) { - val containerRequests: List[ContainerRequest] = - if (numExecutors <= 0) { - logDebug("numExecutors: " + numExecutors) - List() - } else if (preferredHostToCount.isEmpty) { - logDebug("host preferences is empty") - createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY).toList - } else { - // Request for all hosts in preferred nodes and for numExecutors - - // candidates.size, request by default allocation policy. - val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests ++= createResourceRequests( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - } - val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( - hostContainerRequests).toList - - val anyContainerRequests = createResourceRequests( - AllocationType.ANY, - resource = null, - numExecutors, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - - val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( - hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size) - - containerRequestBuffer ++= hostContainerRequests - containerRequestBuffer ++= rackContainerRequests - containerRequestBuffer ++= anyContainerRequests - containerRequestBuffer.toList - } - - for (request <- containerRequests) { - amClient.addContainerRequest(request) - } - - for (request <- containerRequests) { - val nodes = request.getNodes - var hostStr = if (nodes == null || nodes.isEmpty) { - "Any" - } else { - nodes.last - } - logInfo("Container request (host: %s, priority: %s, capability: %s".format( - hostStr, - request.getPriority().getPriority, - request.getCapability)) - } - } - - private def createResourceRequests( - requestType: AllocationType.AllocationType, - resource: String, - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - // If hostname is specified, then we need at least two requests - node local and rack local. - // There must be a third request, which is ANY. That will be specially handled. - requestType match { - case AllocationType.HOST => { - assert(YarnSparkHadoopUtil.ANY_HOST != resource) - val hostname = resource - val nodeLocal = constructContainerRequests( - Array(hostname), - racks = null, - numExecutors, - priority) - - // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. - YarnSparkHadoopUtil.populateRackInfo(conf, hostname) - nodeLocal - } - case AllocationType.RACK => { - val rack = resource - constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) - } - case AllocationType.ANY => constructContainerRequests( - hosts = null, racks = null, numExecutors, priority) - case _ => throw new IllegalArgumentException( - "Unexpected/unsupported request type: " + requestType) - } - } - - private def constructContainerRequests( - hosts: Array[String], - racks: Array[String], - numExecutors: Int, - priority: Int - ): ArrayBuffer[ContainerRequest] = { - - val memoryRequest = executorMemory + memoryOverhead - val resource = Resource.newInstance(memoryRequest, executorCores) - - val prioritySetting = Records.newRecord(classOf[Priority]) - prioritySetting.setPriority(priority) - - val requests = new ArrayBuffer[ContainerRequest]() - for (i <- 0 until numExecutors) { - requests += new ContainerRequest(resource, hosts, racks, prioritySetting) - } - requests - } - - private class StableAllocateResponse(response: AllocateResponse) extends YarnAllocateResponse { - override def getAllocatedContainers() = response.getAllocatedContainers() - override def getAvailableResources() = response.getAvailableResources() - override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() - } - -} 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 b32e15738f28..de65ef23ad1c 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 @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.util.{List => JList} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.regex.Pattern @@ -25,17 +24,20 @@ import java.util.regex.Pattern import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import com.google.common.util.concurrent.ThreadFactoryBuilder + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse +import org.apache.hadoop.yarn.client.api.AMRMClient +import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ - object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -52,12 +54,12 @@ object AllocationType extends Enumeration { // more info on how we are requesting for containers. /** - * Common code for the Yarn container allocator. Contains all the version-agnostic code to - * manage container allocation for a running Spark application. + * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ -private[yarn] abstract class YarnAllocator( +private[yarn] class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], @@ -67,7 +69,7 @@ private[yarn] abstract class YarnAllocator( import YarnAllocator._ // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]() @@ -161,8 +163,6 @@ private[yarn] abstract class YarnAllocator( def allocateResources(): Unit = synchronized { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() - // this is needed by alpha, do it here since we add numPending right after this - val executorsPending = numPendingAllocate.get() if (missing > 0) { val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) @@ -172,7 +172,7 @@ private[yarn] abstract class YarnAllocator( logDebug("Empty allocation request ...") } - val allocateResponse = allocateContainers(missing, executorsPending) + val allocateResponse = allocateContainers(missing) val allocatedContainers = allocateResponse.getAllocatedContainers() if (allocatedContainers.size > 0) { @@ -368,7 +368,7 @@ private[yarn] abstract class YarnAllocator( val containerId = completedContainer.getContainerId if (releasedContainers.containsKey(containerId)) { - // YarnAllocationHandler already marked the container for release, so remove it from + // Already marked the container for release, so remove it from // `releasedContainers`. releasedContainers.remove(containerId) } else { @@ -441,20 +441,16 @@ private[yarn] abstract class YarnAllocator( } } - protected def allocatedContainersOnHost(host: String): Int = { - var retval = 0 + private def allocatedContainersOnHost(host: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + allocatedHostToContainersMap.getOrElse(host, Set()).size } - retval } - protected def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 + private def allocatedContainersOnRack(rack: String): Int = { allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) + allocatedRackCount.getOrElse(rack, 0) } - retval } private def isResourceConstraintSatisfied(container: Container): Boolean = { @@ -464,9 +460,8 @@ private[yarn] abstract class YarnAllocator( // A simple method to copy the split info map. private def generateNodeToWeight( conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]] - ): (Map[String, Int], Map[String, Int]) = { - + input: collection.Map[String, collection.Set[SplitInfo]]) + : (Map[String, Int], Map[String, Int]) = { if (input == null) { return (Map[String, Int](), Map[String, Int]()) } @@ -488,9 +483,9 @@ private[yarn] abstract class YarnAllocator( (hostToCount.toMap, rackToCount.toMap) } - private def internalReleaseContainer(container: Container) = { + private def internalReleaseContainer(container: Container): Unit = { releasedContainers.put(container.getId(), true) - releaseContainer(container) + amClient.releaseAssignedContainer(container.getId()) } /** @@ -498,26 +493,158 @@ private[yarn] abstract class YarnAllocator( * * @param count Number of containers to allocate. * If zero, should still contact RM (as a heartbeat). - * @param pending Number of containers pending allocate. Only used on alpha. * @return Response to the allocation request. */ - protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse + private def allocateContainers(count: Int): AllocateResponse = { + addResourceRequests(count) + + // We have already set the container request. Poll the ResourceManager for a response. + // This doubles as a heartbeat if there are no pending container requests. + val progressIndicator = 0.1f + amClient.allocate(progressIndicator) + } - /** Called to release a previously allocated container. */ - protected def releaseContainer(container: Container): Unit + private def createRackResourceRequests(hostContainers: ArrayBuffer[ContainerRequest]) + : ArrayBuffer[ContainerRequest] = { + // Generate modified racks and new set of hosts under it before issuing requests. + val rackToCounts = new HashMap[String, Int]() - /** - * Defines the interface for an allocate response from the RM. This is needed since the alpha - * and stable interfaces differ here in ways that cannot be fixed using other routes. - */ - protected trait YarnAllocateResponse { + for (container <- hostContainers) { + val candidateHost = container.getNodes.last + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += 1 + rackToCounts.put(rack, count) + } + } + + val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts) { + requestedContainers ++= createResourceRequests( + AllocationType.RACK, + rack, + count, + RM_REQUEST_PRIORITY) + } + + requestedContainers + } + + private def addResourceRequests(numExecutors: Int): Unit = { + val containerRequests: List[ContainerRequest] = + if (numExecutors <= 0) { + logDebug("numExecutors: " + numExecutors) + List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") + createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY).toList + } else { + // Request for all hosts in preferred nodes and for numExecutors - + // candidates.size, request by default allocation policy. + val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) + for ((candidateHost, candidateCount) <- preferredHostToCount) { + val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) + + if (requiredCount > 0) { + hostContainerRequests ++= createResourceRequests( + AllocationType.HOST, + candidateHost, + requiredCount, + RM_REQUEST_PRIORITY) + } + } + val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests( + hostContainerRequests).toList + + val anyContainerRequests = createResourceRequests( + AllocationType.ANY, + resource = null, + numExecutors, + RM_REQUEST_PRIORITY) + + val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( + hostContainerRequests.size + rackContainerRequests.size + anyContainerRequests.size) + + containerRequestBuffer ++= hostContainerRequests + containerRequestBuffer ++= rackContainerRequests + containerRequestBuffer ++= anyContainerRequests + containerRequestBuffer.toList + } - def getAllocatedContainers(): JList[Container] + for (request <- containerRequests) { + amClient.addContainerRequest(request) + } - def getAvailableResources(): Resource + for (request <- containerRequests) { + val nodes = request.getNodes + val hostStr = if (nodes == null || nodes.isEmpty) { + "Any" + } else { + nodes.last + } + logInfo("Container request (host: %s, priority: %s, capability: %s".format( + hostStr, + request.getPriority().getPriority, + request.getCapability)) + } + } - def getCompletedContainersStatuses(): JList[ContainerStatus] + private def createResourceRequests( + requestType: AllocationType.AllocationType, + resource: String, + numExecutors: Int, + priority: Int): ArrayBuffer[ContainerRequest] = { + // If hostname is specified, then we need at least two requests - node local and rack local. + // There must be a third request, which is ANY. That will be specially handled. + requestType match { + case AllocationType.HOST => { + assert(YarnSparkHadoopUtil.ANY_HOST != resource) + val hostname = resource + val nodeLocal = constructContainerRequests( + Array(hostname), + racks = null, + numExecutors, + priority) + + // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. + YarnSparkHadoopUtil.populateRackInfo(conf, hostname) + nodeLocal + } + case AllocationType.RACK => { + val rack = resource + constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) + } + case AllocationType.ANY => constructContainerRequests( + hosts = null, racks = null, numExecutors, priority) + case _ => throw new IllegalArgumentException( + "Unexpected/unsupported request type: " + requestType) + } + } + private def constructContainerRequests( + hosts: Array[String], + racks: Array[String], + numExecutors: Int, + priority: Int + ): ArrayBuffer[ContainerRequest] = { + val memoryRequest = executorMemory + memoryOverhead + val resource = Resource.newInstance(memoryRequest, executorCores) + + val prioritySetting = Records.newRecord(classOf[Priority]) + prioritySetting.setPriority(priority) + + val requests = new ArrayBuffer[ContainerRequest]() + for (i <- 0 until numExecutors) { + requests += new ContainerRequest(resource, hosts, racks, prioritySetting) + } + requests } } 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 2510b9c9cef6..bf4e15908bb4 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 @@ -17,19 +17,33 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} + +import scala.collection.JavaConversions._ import scala.collection.{Map, Set} +import scala.util.Try -import org.apache.hadoop.yarn.conf.YarnConfiguration +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.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.util.Utils /** - * Interface that defines a Yarn RM client. Abstracts away Yarn version-specific functionality that - * is used by Spark's AM. + * Handles registering and unregistering the application with the YARN ResourceManager. */ -trait YarnRMClient { +private[spark] class YarnRMClient(args: ApplicationMasterArguments) extends Logging { + + private var amClient: AMRMClient[ContainerRequest] = _ + private var uiHistoryAddress: String = _ + private var registered: Boolean = false /** * Registers the application master with the RM. @@ -46,7 +60,21 @@ trait YarnRMClient { preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, uiHistoryAddress: String, - securityMgr: SecurityManager): YarnAllocator + securityMgr: SecurityManager + ): YarnAllocator = { + amClient = AMRMClient.createAMRMClient() + amClient.init(conf) + amClient.start() + this.uiHistoryAddress = uiHistoryAddress + + logInfo("Registering the ApplicationMaster") + synchronized { + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + registered = true + } + new YarnAllocator(conf, sparkConf, amClient, getAttemptId(), args, + preferredNodeLocations, securityMgr) + } /** * Unregister the AM. Guaranteed to only be called once. @@ -54,15 +82,45 @@ trait YarnRMClient { * @param status The final status of the AM. * @param diagnostics Diagnostics message to include in the final status. */ - def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit + def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit = synchronized { + if (registered) { + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + } + } /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId + def getAttemptId(): ApplicationAttemptId = { + val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) + val containerId = ConverterUtils.toContainerId(containerIdString) + containerId.getApplicationAttemptId() + } /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ - def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] + def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { + // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, + // so not all stable releases have it. + val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) + .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") + + // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. + try { + 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 } + Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) + } catch { + case e: NoSuchMethodException => + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = prefix + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } + } /** Returns the maximum number of attempts to register the AM. */ - def getMaxRegAttempts(conf: YarnConfiguration): Int + def getMaxRegAttempts(conf: YarnConfiguration): Int = + conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala deleted file mode 100644 index 8d4b96ed7993..000000000000 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ /dev/null @@ -1,110 +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.deploy.yarn - -import java.util.{List => JList} - -import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ -import scala.util._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -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} -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - - -/** - * YarnRMClient implementation for the Yarn stable API. - */ -private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { - - private var amClient: AMRMClient[ContainerRequest] = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - override def register( - conf: YarnConfiguration, - sparkConf: SparkConf, - preferredNodeLocations: Map[String, Set[SplitInfo]], - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager) = { - amClient = AMRMClient.createAMRMClient() - amClient.init(conf) - amClient.start() - this.uiHistoryAddress = uiHistoryAddress - - logInfo("Registering the ApplicationMaster") - synchronized { - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) - registered = true - } - new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, - preferredNodeLocations, securityMgr) - } - - override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { - if (registered) { - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) - } - } - - override def getAttemptId() = { - val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { - // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, - // so not all stable releases have it. - val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) - .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") - - // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. - try { - 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 } - Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) - } catch { - case e: NoSuchMethodException => - val proxy = WebAppUtils.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = prefix + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - } - - override def getMaxRegAttempts(conf: YarnConfiguration) = - conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) - -} diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 2923e6729cd6..09597bd0e6ab 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -65,7 +65,8 @@ private[spark] class YarnClientSchedulerBackend( */ private def getExtraClientArguments: Seq[String] = { val extraArgs = new ArrayBuffer[String] - val optionTuples = // List of (target Client argument, environment variable, Spark property) + // List of (target Client argument, environment variable, Spark property) + val optionTuples = List( ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), @@ -78,11 +79,25 @@ private[spark] class YarnClientSchedulerBackend( ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) + // Warn against the following deprecated environment variables: env var -> suggestion + val deprecatedEnvVars = Map( + "SPARK_MASTER_MEMORY" -> "SPARK_DRIVER_MEMORY or --driver-memory through spark-submit", + "SPARK_WORKER_INSTANCES" -> "SPARK_WORKER_INSTANCES or --num-executors through spark-submit", + "SPARK_WORKER_MEMORY" -> "SPARK_EXECUTOR_MEMORY or --executor-memory through spark-submit", + "SPARK_WORKER_CORES" -> "SPARK_EXECUTOR_CORES or --executor-cores through spark-submit") + // Do the same for deprecated properties: property -> suggestion + val deprecatedProps = Map("spark.master.memory" -> "--driver-memory through spark-submit") optionTuples.foreach { case (optionName, envVar, sparkProp) => if (System.getenv(envVar) != null) { extraArgs += (optionName, System.getenv(envVar)) + if (deprecatedEnvVars.contains(envVar)) { + logWarning(s"NOTE: $envVar is deprecated. Use ${deprecatedEnvVars(envVar)} instead.") + } } else if (sc.getConf.contains(sparkProp)) { extraArgs += (optionName, sc.getConf.get(sparkProp)) + if (deprecatedProps.contains(sparkProp)) { + logWarning(s"NOTE: $sparkProp is deprecated. Use ${deprecatedProps(sparkProp)} instead.") + } } } extraArgs From 9bf4cc7dbb069c4969c5f317590e3e9ddc4efd4f Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 3 Jan 2015 22:39:10 +0800 Subject: [PATCH 05/24] fix bug in catalyst --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 ea64e8a66e85..cc8e13bacb1d 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 @@ -25,14 +25,14 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.CatalystConf -import org.apache.spark.sql.catalyst.EmptyConf +import org.apache.spark.sql.catalyst.test.SimpleConf /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing * when all relations are already filled in and the analyser needs only to resolve attribute * references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, EmptyConf) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleConf) /** * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and From 73c16b13b23e2b9e98ac6fb1864d8c98a3813dfb Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 4 Jan 2015 01:02:17 +0800 Subject: [PATCH 06/24] fix bug in sql/hive --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 1f7ef89feefb..f391f99e4d92 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 @@ -73,6 +73,9 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + /* By default it should be case insensitive to match Hive */ + setConf(CatalystConf.CASE_SENSITIVE, "false") + // Change the default SQL dialect to HiveQL override private[spark] def dialect: String = getConf(SQLConf.DIALECT, "hiveql") @@ -249,9 +252,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { runSqlHive(s"SET $key=$value") } - /* By default it should be case insensitive to match Hive */ - this.setConf(CatalystConf.CASE_SENSITIVE, "false") - /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient override protected[sql] lazy val catalog = From dee56e9ae71ebd9c8464cf6be763895e8bcdf2e6 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 19 Jan 2015 01:55:50 -0800 Subject: [PATCH 07/24] fix test case failure --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 363dfaba4caf..90daa5f6d611 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 @@ -51,13 +51,13 @@ import org.apache.spark.sql.types._ class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - /* By default it should be case insensitive to match Hive */ - setConf(CatalystConf.CASE_SENSITIVE, "false") - protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") } + /* By default it should be case insensitive to match Hive */ + conf.setConf(CatalystConf.CASE_SENSITIVE, "false") + /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive From 5472b0832213aa0d7f092c06f54095477e695c93 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:17:28 +0800 Subject: [PATCH 08/24] fix compile issue --- pom.xml | 14 +++++++------- .../apache/spark/sql/catalyst/CatalystConf.scala | 10 ++++++---- .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/test/SimpleConf.scala | 4 +++- .../catalyst/analysis/DecimalPrecisionSuite.scala | 2 +- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 ------ 6 files changed, 18 insertions(+), 20 deletions(-) diff --git a/pom.xml b/pom.xml index c85c5feeaf38..eaed47a27aef 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ central Maven Repository - https://repo1.maven.org/maven2 + http://repo1.maven.org/maven2 true @@ -203,7 +203,7 @@ apache-repo Apache Repository - https://repository.apache.org/content/repositories/releases + http://repository.apache.org/content/repositories/releases true @@ -214,7 +214,7 @@ jboss-repo JBoss Repository - https://repository.jboss.org/nexus/content/repositories/releases + http://repository.jboss.org/nexus/content/repositories/releases true @@ -225,7 +225,7 @@ mqtt-repo MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases + http://repo.eclipse.org/content/repositories/paho-releases true @@ -236,7 +236,7 @@ cloudera-repo Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos + http://repository.cloudera.com/artifactory/cloudera-repos true @@ -258,7 +258,7 @@ spring-releases Spring Release Repository - https://repo.spring.io/libs-release + http://repo.spring.io/libs-release true @@ -270,7 +270,7 @@ central - https://repo1.maven.org/maven2 + http://repo1.maven.org/maven2 true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index bd47eee4fa09..19fd6942f3ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -36,19 +36,21 @@ private[spark] trait CatalystConf { * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyConf extends CatalystConf { - def setConf(key: String, value: String) : Unit = { + override def caseSensitiveAnalysis: Boolean = true + + override def setConf(key: String, value: String) : Unit = { throw new UnsupportedOperationException } - def getConf(key: String) : String = { + override def getConf(key: String) : String = { throw new UnsupportedOperationException } - def getConf(key: String, defaultValue: String) : String = { + override def getConf(key: String, defaultValue: String) : String = { throw new UnsupportedOperationException } - def getAllConfs: immutable.Map[String, String] = { + override def getAllConfs: immutable.Map[String, String] = { throw new UnsupportedOperationException } } 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 e5b8fb0ab545..b5e50a109fbd 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleConf) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleConf(true)) /** * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala index 6c10b46726db..b9d17d99343d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala @@ -23,9 +23,11 @@ import scala.collection.immutable import scala.collection.mutable /** A CatalystConf that can be used for local testing. */ -class SimpleConf(caseSensitiveAnalysis: Boolean) extends CatalystConf { +class SimpleConf(caseSensitive: Boolean) extends CatalystConf { val map = mutable.Map[String, String]() + def caseSensitiveAnalysis: Boolean = caseSensitive + def setConf(key: String, value: String) : Unit = { map.put(key, value) } 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 390ab861f2af..82ed3640b48b 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.test.SimpleConf import org.scalatest.{BeforeAndAfter, FunSuite} class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { - val conf = new SimpleConf + val conf = new SimpleConf(true) val catalog = new SimpleCatalog(conf) val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf) 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 50615e2bc61f..46eacf7e8a79 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 @@ -21,13 +21,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ -<<<<<<< HEAD import org.apache.spark.sql.catalyst.CatalystConf -import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types._ -======= ->>>>>>> 254e0509762937acc9c72b432d5d953bf72c3c52 import org.apache.spark.sql.TestData._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} From 69b3b708c2b78ed2e1061d69ef3e7c3b5e2d94c6 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:32:35 +0800 Subject: [PATCH 09/24] fix AnalysisSuite --- .../spark/sql/catalyst/analysis/AnalysisSuite.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 82a53c124f78..3e849ad709db 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 @@ -36,9 +36,13 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseInsensitiveCatalog = new SimpleCatalog(caseInsensitiveConf) val caseSensitiveAnalyzer = - new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitiveConf) + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitiveConf) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } val caseInsensitiveAnalyzer = - new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseInsensitiveConf) + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseInsensitiveConf) { + override val extendedResolutionRules = EliminateSubQueries :: Nil + } def caseSensitiveAnalyze(plan: LogicalPlan): Unit = caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer.execute(plan)) From fd30e25f84e569769519282cc3ec39e58a200e87 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:34:34 +0800 Subject: [PATCH 10/24] added override --- .../sql/catalyst/analysis/Analyzer.scala | 4 +- ...pleConf.scala => SimpleCatalystConf.scala} | 86 +++++++++---------- .../sql/catalyst/analysis/AnalysisSuite.scala | 6 +- .../analysis/DecimalPrecisionSuite.scala | 4 +- 4 files changed, 50 insertions(+), 50 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/{SimpleConf.scala => SimpleCatalystConf.scala} (73%) 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 b5e50a109fbd..57eb211b34c0 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 @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.test.SimpleConf +import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.apache.spark.sql.types._ /** @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleConf(true)) +object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleCatalystConf(true)) /** * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala similarity index 73% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala index b9d17d99343d..beb8ba03bfab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala @@ -1,43 +1,43 @@ -/* - * 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.test - -import org.apache.spark.sql.catalyst.CatalystConf - -import scala.collection.immutable -import scala.collection.mutable - -/** A CatalystConf that can be used for local testing. */ -class SimpleConf(caseSensitive: Boolean) extends CatalystConf { - val map = mutable.Map[String, String]() - - def caseSensitiveAnalysis: Boolean = caseSensitive - - def setConf(key: String, value: String) : Unit = { - map.put(key, value) - } - def getConf(key: String) : String ={ - map.get(key).get - } - def getConf(key: String, defaultValue: String) : String = { - map.getOrElse(key, defaultValue) - } - def getAllConfs: immutable.Map[String, String] = { - map.toMap - } -} +/* + * 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.test + +import org.apache.spark.sql.catalyst.CatalystConf + +import scala.collection.immutable +import scala.collection.mutable + +/** A CatalystConf that can be used for local testing. */ +class SimpleCatalystConf(caseSensitive: Boolean) extends CatalystConf { + val map = mutable.Map[String, String]() + + override def caseSensitiveAnalysis: Boolean = caseSensitive + + override def setConf(key: String, value: String) : Unit = { + map.put(key, value) + } + override def getConf(key: String) : String ={ + map.get(key).get + } + override def getConf(key: String, defaultValue: String) : String = { + map.getOrElse(key, defaultValue) + } + override def getAllConfs: immutable.Map[String, String] = { + map.toMap + } +} 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 3e849ad709db..054550d35a25 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 @@ -24,13 +24,13 @@ 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.CatalystConf -import org.apache.spark.sql.catalyst.test.SimpleConf +import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ class AnalysisSuite extends FunSuite with BeforeAndAfter { - val caseSensitiveConf = new SimpleConf(true) - val caseInsensitiveConf = new SimpleConf(false) + val caseSensitiveConf = new SimpleCatalystConf(true) + val caseInsensitiveConf = new SimpleCatalystConf(false) val caseSensitiveCatalog = new SimpleCatalog(caseSensitiveConf) val caseInsensitiveCatalog = new SimpleCatalog(caseInsensitiveConf) 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 82ed3640b48b..19eb6aa30cf3 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 @@ -20,11 +20,11 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Union, Project, LocalRelation} import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.test.SimpleConf +import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.scalatest.{BeforeAndAfter, FunSuite} class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { - val conf = new SimpleConf(true) + val conf = new SimpleCatalystConf(true) val catalog = new SimpleCatalog(conf) val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, conf) From 966e719b77e3e8e3e715e58f3a0aeed3b4aba009 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:41:13 +0800 Subject: [PATCH 11/24] set CASE_SENSITIVE false in hivecontext --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 5 +++-- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 5 +---- 2 files changed, 4 insertions(+), 6 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 9331916da41f..aae1e26f369e 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 @@ -17,12 +17,12 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.CatalystConf +import java.util.Properties import scala.collection.immutable import scala.collection.JavaConversions._ -import java.util.Properties +import org.apache.spark.sql.catalyst.CatalystConf private[spark] object SQLConf { val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" @@ -266,3 +266,4 @@ private[sql] class SQLConf extends Serializable with CatalystConf { settings.clear() } } + 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 174ed204e8fe..e243c60dad28 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 @@ -50,10 +50,6 @@ import org.apache.spark.sql.types._ class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - protected[sql] override lazy val conf: SQLConf = new SQLConf { - override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - } - /** * When true, enables an experimental feature where metastore tables that use the parquet SerDe * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive @@ -266,6 +262,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[hive] class SQLSession extends super.SQLSession { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + setConf(CatalystConf.CASE_SENSITIVE, "false") } protected[hive] lazy val hiveconf: HiveConf = { From 5d7c45618bcc0ba1195e406230972a9c237016c7 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:46:35 +0800 Subject: [PATCH 12/24] set CASE_SENSITIVE false in TestHive --- .../main/scala/org/apache/spark/sql/hive/test/TestHive.scala | 2 ++ 1 file changed, 2 insertions(+) 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 9f17bca083d1..fce1b40c14c6 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 @@ -39,6 +39,7 @@ import org.apache.spark.{SparkConf, SparkContext} import scala.collection.mutable import scala.language.implicitConversions +import org.apache.spark.sql.catalyst.CatalystConf /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -108,6 +109,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + setConf(CatalystConf.CASE_SENSITIVE, "false") } } From 6ef31cfb5269e6298349cf97fbe28fcfa43c26ec Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:53:07 +0800 Subject: [PATCH 13/24] revert pom changes --- pom.xml | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index eaed47a27aef..c85c5feeaf38 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ central Maven Repository - http://repo1.maven.org/maven2 + https://repo1.maven.org/maven2 true @@ -203,7 +203,7 @@ apache-repo Apache Repository - http://repository.apache.org/content/repositories/releases + https://repository.apache.org/content/repositories/releases true @@ -214,7 +214,7 @@ jboss-repo JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases + https://repository.jboss.org/nexus/content/repositories/releases true @@ -225,7 +225,7 @@ mqtt-repo MQTT Repository - http://repo.eclipse.org/content/repositories/paho-releases + https://repo.eclipse.org/content/repositories/paho-releases true @@ -236,7 +236,7 @@ cloudera-repo Cloudera Repository - http://repository.cloudera.com/artifactory/cloudera-repos + https://repository.cloudera.com/artifactory/cloudera-repos true @@ -258,7 +258,7 @@ spring-releases Spring Release Repository - http://repo.spring.io/libs-release + https://repo.spring.io/libs-release true @@ -270,7 +270,7 @@ central - http://repo1.maven.org/maven2 + https://repo1.maven.org/maven2 true From eee75bad4d7eacb73cfc57ea733aed1dcd97ec11 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:55:34 +0800 Subject: [PATCH 14/24] fix EmptyConf --- .../scala/org/apache/spark/sql/catalyst/CatalystConf.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index 19fd6942f3ff..7d253544f6fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -36,7 +36,9 @@ private[spark] trait CatalystConf { * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyConf extends CatalystConf { - override def caseSensitiveAnalysis: Boolean = true + override def caseSensitiveAnalysis: Boolean = { + throw new UnsupportedOperationException + } override def setConf(key: String, value: String) : Unit = { throw new UnsupportedOperationException From d5a99337c86c92e705098b95f844b928e5129213 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 30 Apr 2015 16:59:04 +0800 Subject: [PATCH 15/24] fix style --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 57eb211b34c0..2d5bd48f0204 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 @@ -31,7 +31,8 @@ import org.apache.spark.sql.types._ * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ -object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleCatalystConf(true)) +object SimpleAnalyzer + extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, new SimpleCatalystConf(true)) /** * Provides a logical query plan analyzer, which translates [[UnresolvedAttribute]]s and From 7fc4a987426974c07a10a5cd2ba2f920c5578da0 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 30 Apr 2015 22:21:36 +0800 Subject: [PATCH 16/24] fix test case --- .../main/scala/org/apache/spark/sql/hive/test/TestHive.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 fce1b40c14c6..02dacd0cafc1 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 @@ -109,7 +109,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - setConf(CatalystConf.CASE_SENSITIVE, "false") + override def caseSensitiveAnalysis: Boolean = + getConf(CatalystConf.CASE_SENSITIVE, "false").toBoolean } } From 6db4bf5448a9c6b7c1401b329c0fc55d6593faec Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 30 Apr 2015 22:26:16 +0800 Subject: [PATCH 17/24] also fix for HiveContext --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 e243c60dad28..fbe89e906f8f 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 @@ -262,7 +262,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[hive] class SQLSession extends super.SQLSession { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") - setConf(CatalystConf.CASE_SENSITIVE, "false") + override def caseSensitiveAnalysis: Boolean = + getConf(CatalystConf.CASE_SENSITIVE, "false").toBoolean } protected[hive] lazy val hiveconf: HiveConf = { From a3f76599431c1eb00f152aef83c7a7754714fb42 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 3 May 2015 09:09:58 +0800 Subject: [PATCH 18/24] remove unsed imports --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 1 - .../test/scala/org/apache/spark/sql/hive/parquetSuites.scala | 4 +--- 3 files changed, 2 insertions(+), 5 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 2d5bd48f0204..d26dce6d59aa 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.expressions._ @@ -25,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.apache.spark.sql.types._ +import org.apache.spark.util.collection.OpenHashSet /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index ac10b173307d..d8b2403ea249 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.{SparkFiles, SparkException} import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.plans.logical.Project -import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index d5dd0bf58e70..bf1121ddf027 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -21,14 +21,12 @@ import java.io.File import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.{QueryTest, SQLConf, SaveMode} +import org.apache.spark.sql.{QueryTest, SQLConf} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan -import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ -import org.apache.spark.sql.json.JSONRelation import org.apache.spark.sql.sources.{InsertIntoDataSource, LogicalRelation} import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.SaveMode From b35529e5ed790e252364758d259d0631d6d08092 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 3 May 2015 09:10:49 +0800 Subject: [PATCH 19/24] minor style --- .../scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c30e999d8f25..e68028f57b63 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,9 +19,9 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.EmptyConf +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} /** * Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception From 9e11752a5a899f5baabe5c2f2f0dd69363e335b9 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 3 May 2015 09:16:34 +0800 Subject: [PATCH 20/24] improve SimpleCatalystConf --- .../catalyst/test/SimpleCatalystConf.scala | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala index beb8ba03bfab..f7b7df60738b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala @@ -17,27 +17,31 @@ package org.apache.spark.sql.catalyst.test -import org.apache.spark.sql.catalyst.CatalystConf - import scala.collection.immutable -import scala.collection.mutable +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.catalyst.CatalystConf /** A CatalystConf that can be used for local testing. */ class SimpleCatalystConf(caseSensitive: Boolean) extends CatalystConf { - val map = mutable.Map[String, String]() + val settings = java.util.Collections.synchronizedMap( + new java.util.HashMap[String, String]()) override def caseSensitiveAnalysis: Boolean = caseSensitive override def setConf(key: String, value: String) : Unit = { - map.put(key, value) + settings.put(key, value) } + override def getConf(key: String) : String ={ - map.get(key).get + settings.get(key) } + override def getConf(key: String, defaultValue: String) : String = { - map.getOrElse(key, defaultValue) + Option(settings.get(key)).getOrElse(defaultValue) } - override def getAllConfs: immutable.Map[String, String] = { - map.toMap + + override def getAllConfs: immutable.Map[String, String] = settings.synchronized { + settings.toMap } } From b73df6cd22490e9eb2a9c336571a139bebf4c444 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 3 May 2015 09:18:52 +0800 Subject: [PATCH 21/24] style issue --- .../spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala | 5 +++-- .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 6 +++--- 2 files changed, 6 insertions(+), 5 deletions(-) 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 19eb6aa30cf3..f96456ff39ed 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 @@ -17,11 +17,12 @@ package org.apache.spark.sql.catalyst.analysis +import org.scalatest.{BeforeAndAfter, FunSuite} + import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Union, Project, LocalRelation} -import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.test.SimpleCatalystConf -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.sql.types._ class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { val conf = new SimpleCatalystConf(true) 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 d6a7912a5dba..015c24976565 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 @@ -25,21 +25,21 @@ import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContaine import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.parse.VariableSubstitution import org.apache.hadoop.hive.ql.processors._ -import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe -import org.apache.spark.sql.SQLConf + +import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.execution.HiveNativeCommand +import org.apache.spark.sql.SQLConf import org.apache.spark.util.Utils import org.apache.spark.{SparkConf, SparkContext} import scala.collection.mutable import scala.language.implicitConversions -import org.apache.spark.sql.catalyst.CatalystConf /* Implicit conversions */ import scala.collection.JavaConversions._ From af512c7902113b537343b0de4ff5b24420612f2f Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 8 May 2015 11:20:12 +0800 Subject: [PATCH 22/24] fix conflicts --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 21 +++++-------------- 1 file changed, 5 insertions(+), 16 deletions(-) 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 22c8209c51aa..1bcb8a249959 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,26 +17,18 @@ package org.apache.spark.sql.hive -import java.io.IOException -import java.util.{List => JList} - import com.google.common.base.Objects import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Partition => TPartition, Table => TTable} -import org.apache.hadoop.hive.metastore.{TableType, Warehouse} +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.metastore.Warehouse import org.apache.hadoop.hive.ql.metadata._ -import org.apache.hadoop.hive.ql.plan.CreateTableDesc -import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe -import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} -import org.apache.hadoop.util.ReflectionUtils +import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.Logging -import org.apache.spark.sql.hive.client.IsolatedClientLoader import org.apache.spark.sql.{SaveMode, AnalysisException, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NoSuchTableException, Catalog, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical @@ -44,9 +36,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.hive.client._ import org.apache.spark.sql.parquet.{ParquetRelation2, Partition => ParquetPartition, PartitionSpec} -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, DDLParser, LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.util.Utils /* Implicit conversions */ @@ -55,8 +46,6 @@ import scala.collection.JavaConversions._ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) extends Catalog with Logging { - import org.apache.spark.sql.hive.HiveMetastoreTypes._ - /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) From d4b724f174e18b035a708f2a7fd9a690942861b8 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 8 May 2015 11:45:33 +0800 Subject: [PATCH 23/24] address michael's comment --- .../spark/sql/catalyst/CatalystConf.scala | 29 ++---------- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../catalyst/test/SimpleCatalystConf.scala | 47 ------------------- .../sql/catalyst/analysis/AnalysisSuite.scala | 3 +- .../analysis/DecimalPrecisionSuite.scala | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 3 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +- .../spark/sql/sources/DataSourceTest.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- 10 files changed, 13 insertions(+), 84 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index 7d253544f6fe..3f351b07b37d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -17,18 +17,8 @@ package org.apache.spark.sql.catalyst -import scala.collection.immutable - -private[spark] object CatalystConf{ - val CASE_SENSITIVE = "spark.sql.caseSensitive" -} - private[spark] trait CatalystConf { def caseSensitiveAnalysis: Boolean - def setConf(key: String, value: String) : Unit - def getConf(key: String) : String - def getConf(key: String, defaultValue: String) : String - def getAllConfs: immutable.Map[String, String] } /** @@ -39,20 +29,7 @@ object EmptyConf extends CatalystConf { override def caseSensitiveAnalysis: Boolean = { throw new UnsupportedOperationException } - - override def setConf(key: String, value: String) : Unit = { - throw new UnsupportedOperationException - } - - override def getConf(key: String) : String = { - throw new UnsupportedOperationException - } - - override def getConf(key: String, defaultValue: String) : String = { - throw new UnsupportedOperationException - } - - override def getAllConfs: immutable.Map[String, String] = { - throw new UnsupportedOperationException - } } + +/** A CatalystConf that can be used for local testing. */ +case class SimpleCatalystConf(caseSensitiveAnalysis: Boolean) extends CatalystConf 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 a9a4c0f12cb7..073cfe91344b 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 @@ -20,11 +20,10 @@ package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.sql.catalyst.{SimpleCatalystConf, CatalystConf} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala deleted file mode 100644 index f7b7df60738b..000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/test/SimpleCatalystConf.scala +++ /dev/null @@ -1,47 +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.catalyst.test - -import scala.collection.immutable -import scala.collection.JavaConversions._ - -import org.apache.spark.sql.catalyst.CatalystConf - -/** A CatalystConf that can be used for local testing. */ -class SimpleCatalystConf(caseSensitive: Boolean) extends CatalystConf { - val settings = java.util.Collections.synchronizedMap( - new java.util.HashMap[String, String]()) - - override def caseSensitiveAnalysis: Boolean = caseSensitive - - override def setConf(key: String, value: String) : Unit = { - settings.put(key, value) - } - - override def getConf(key: String) : String ={ - settings.get(key) - } - - override def getConf(key: String, defaultValue: String) : String = { - Option(settings.get(key)).getOrElse(defaultValue) - } - - override def getAllConfs: immutable.Map[String, String] = settings.synchronized { - settings.toMap - } -} 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 054550d35a25..6f2f35564d12 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 @@ -23,8 +23,7 @@ import org.apache.spark.sql.AnalysisException 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.CatalystConf -import org.apache.spark.sql.catalyst.test.SimpleCatalystConf +import org.apache.spark.sql.catalyst.SimpleCatalystConf import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ 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 f96456ff39ed..565b1cfe019c 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 @@ -21,8 +21,8 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{Union, Project, LocalRelation} -import org.apache.spark.sql.catalyst.test.SimpleCatalystConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.catalyst.SimpleCatalystConf class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { val conf = new SimpleCatalystConf(true) 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 4faaf2eb1de6..98a75bb4ed2d 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 @@ -34,6 +34,7 @@ private[spark] object SQLConf { val CODEGEN_ENABLED = "spark.sql.codegen" val UNSAFE_ENABLED = "spark.sql.unsafe.enabled" val DIALECT = "spark.sql.dialect" + val CASE_SENSITIVE = "spark.sql.caseSensitive" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_INT96_AS_TIMESTAMP = "spark.sql.parquet.int96AsTimestamp" @@ -164,7 +165,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf { /** * caseSensitive analysis true by default */ - def caseSensitiveAnalysis: Boolean = getConf(CatalystConf.CASE_SENSITIVE, "true").toBoolean + def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, "true").toBoolean /** * When set to true, Spark SQL will use managed memory for certain operations. This option only 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 62b63792c75c..b44eb223c80c 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 @@ -1279,12 +1279,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-4699 case sensitivity SQL query") { - setConf(CatalystConf.CASE_SENSITIVE, "false") + setConf(SQLConf.CASE_SENSITIVE, "false") val data = TestData(1, "val_1") :: TestData(2, "val_2") :: Nil val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) rdd.toDF().registerTempTable("testTable1") checkAnswer(sql("SELECT VALUE FROM TESTTABLE1 where KEY = 1"), Row("val_1")) - setConf(CatalystConf.CASE_SENSITIVE, "true") + setConf(SQLConf.CASE_SENSITIVE, "true") } test("SPARK-6145: ORDER BY test for nested fields") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala index a023788c8ab7..9d3090c19b4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -26,5 +26,5 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { // We want to test some edge cases. implicit val caseInsensisitiveContext = new SQLContext(TestSQLContext.sparkContext) - caseInsensisitiveContext.setConf(CatalystConf.CASE_SENSITIVE, "false") + caseInsensisitiveContext.setConf(SQLConf.CASE_SENSITIVE, "false") } 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 8c0250890bbc..3bab648e31d1 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 @@ -352,7 +352,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] override lazy val conf: SQLConf = new SQLConf { override def dialect: String = getConf(SQLConf.DIALECT, "hiveql") override def caseSensitiveAnalysis: Boolean = - getConf(CatalystConf.CASE_SENSITIVE, "false").toBoolean + getConf(SQLConf.CASE_SENSITIVE, "false").toBoolean } /** 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 5bf2e21445d8..1598d4bd4755 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 @@ -108,7 +108,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // The super.getConf(SQLConf.DIALECT) is "sql" by default, we need to set it as "hiveql" override def dialect: String = super.getConf(SQLConf.DIALECT, "hiveql") override def caseSensitiveAnalysis: Boolean = - getConf(CatalystConf.CASE_SENSITIVE, "false").toBoolean + getConf(SQLConf.CASE_SENSITIVE, "false").toBoolean } } From cd517126992e14439cfaeaeb5b843e0e50003205 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 8 May 2015 12:06:19 +0800 Subject: [PATCH 24/24] fix compile --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) 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 1bcb8a249959..f5398605bc7e 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 @@ -46,6 +46,8 @@ import scala.collection.JavaConversions._ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: HiveContext) extends Catalog with Logging { + val conf = hive.conf + /** Usages should lock on `this`. */ protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) @@ -138,7 +140,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive val tableProperties = new scala.collection.mutable.HashMap[String, String] tableProperties.put("spark.sql.sources.provider", provider) if (userSpecifiedSchema.isDefined) { - val threshold = hive.conf.schemaStringLengthThreshold + val threshold = conf.schemaStringLengthThreshold val schemaJsonString = userSpecifiedSchema.get.json // Split the JSON string. val parts = schemaJsonString.grouped(threshold).toSeq @@ -345,7 +347,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && - hive.conf.parquetUseDataSourceApi && + conf.parquetUseDataSourceApi && relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) @@ -356,7 +358,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && - hive.conf.parquetUseDataSourceApi && + conf.parquetUseDataSourceApi && relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) @@ -365,7 +367,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive // Read path case p @ PhysicalOperation(_, _, relation: MetastoreRelation) if hive.convertMetastoreParquet && - hive.conf.parquetUseDataSourceApi && + conf.parquetUseDataSourceApi && relation.tableDesc.getSerdeClassName.toLowerCase.contains("parquet") => val parquetRelation = convertToParquetRelation(relation) val attributedRewrites = relation.output.zip(parquetRelation.output) @@ -425,7 +427,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( desc.name, - hive.conf.defaultDataSourceName, + conf.defaultDataSourceName, temporary = false, mode, options = Map.empty[String, String], @@ -454,7 +456,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive val mode = if (allowExisting) SaveMode.Ignore else SaveMode.ErrorIfExists CreateTableUsingAsSelect( tblName, - hive.conf.defaultDataSourceName, + conf.defaultDataSourceName, temporary = false, mode, options = Map.empty[String, String],