From 05dcbdfb248aec0e43534d6ca64a396804467d26 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Jan 2016 17:55:58 -0800 Subject: [PATCH 1/3] Merge with apache --- LICENSE | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../sort/BypassMergeSortShuffleWriter.java | 25 +- .../unsafe/sort/UnsafeInMemorySorter.java | 6 +- .../scala/org/apache/spark/CacheManager.scala | 5 - .../scala/org/apache/spark/TaskContext.scala | 3 +- .../org/apache/spark/TaskContextImpl.scala | 3 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- .../org/apache/spark/rdd/CartesianRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 16 +- .../org/apache/spark/scheduler/Task.scala | 3 +- .../status/api/v1/ExecutorListResource.scala | 10 +- .../org/apache/spark/status/api/v1/api.scala | 2 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 13 +- .../spark/util/logging/FileAppender.scala | 28 +- .../org/apache/spark/CacheManagerSuite.scala | 9 - .../org/apache/spark/rdd/DoubleRDDSuite.scala | 2 +- .../mesos/MesosSchedulerBackendSuite.scala | 2 +- .../BypassMergeSortShuffleWriterSuite.scala | 38 +- .../sort/IndexShuffleBlockResolverSuite.scala | 51 ++- .../spark/storage/BlockManagerSuite.scala | 71 ++-- .../apache/spark/util/FileAppenderSuite.scala | 77 ++++ dev/deps/spark-deps-hadoop-2.2 | 10 +- dev/deps/spark-deps-hadoop-2.3 | 10 +- dev/deps/spark-deps-hadoop-2.4 | 10 +- dev/deps/spark-deps-hadoop-2.6 | 10 +- dev/merge_spark_pr.py | 19 +- dev/run-tests.py | 9 +- dev/sparktestsupport/modules.py | 2 +- dev/test-dependencies.sh | 17 +- docs/mllib-clustering.md | 6 +- docs/mllib-linear-methods.md | 8 +- docs/running-on-mesos.md | 12 +- docs/sql-programming-guide.md | 5 + .../python/mllib/gaussian_mixture_model.py | 4 + examples/src/main/python/sort.py | 2 +- .../examples/mllib/DenseGaussianMixture.scala | 6 + .../clickstream/PageViewGenerator.scala | 14 +- .../spark/streaming/flume/sink/Logging.scala | 8 +- .../streaming/flume/FlumeInputDStream.scala | 18 +- .../kafka/DirectKafkaInputDStream.scala | 4 +- .../spark/streaming/kafka/KafkaCluster.scala | 2 +- .../streaming/kafka/KafkaInputDStream.scala | 4 +- .../spark/streaming/kafka/KafkaRDD.scala | 2 +- .../kafka/ReliableKafkaStreamSuite.scala | 2 +- .../streaming/mqtt/MQTTInputDStream.scala | 4 +- .../twitter/TwitterInputDStream.scala | 4 +- .../streaming/KinesisWordCountASL.scala | 2 +- .../kinesis/KinesisInputDStream.scala | 2 +- .../streaming/kinesis/KinesisReceiver.scala | 2 +- .../streaming/kinesis/KinesisUtils.scala | 2 +- .../kinesis/KinesisBackedBlockRDDSuite.scala | 6 +- .../kinesis/KinesisCheckpointerSuite.scala | 4 +- .../kinesis/KinesisReceiverSuite.scala | 2 +- .../kinesis/KinesisStreamSuite.scala | 4 +- .../org/apache/spark/graphx/GraphOps.scala | 5 +- .../graphx/impl/ReplicatedVertexView.scala | 4 +- .../impl/ShippableVertexPartition.scala | 4 +- .../graphx/impl/VertexPartitionBaseOps.scala | 2 +- .../apache/spark/graphx/lib/PageRank.scala | 2 +- .../launcher/SparkSubmitCommandBuilder.java | 13 +- .../org/apache/spark/ml/tree/treeParams.scala | 2 +- .../python/GaussianMixtureModelWrapper.scala | 4 + .../clustering/GaussianMixtureModel.scala | 2 +- .../spark/mllib/clustering/LDAModel.scala | 26 ++ .../apache/spark/mllib/feature/Word2Vec.scala | 15 +- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 7 +- .../spark/mllib/stat/test/ChiSqTest.scala | 6 +- .../spark/mllib/tree/DecisionTree.scala | 2 +- .../mllib/tree/GradientBoostedTrees.scala | 2 +- .../spark/mllib/tree/RandomForest.scala | 2 +- .../spark/mllib/clustering/LDASuite.scala | 15 +- pom.xml | 27 +- project/MimaBuild.scala | 7 +- project/MimaExcludes.scala | 12 + python/docs/Makefile | 2 +- python/lib/py4j-0.9-src.zip | Bin 44846 -> 0 bytes python/lib/py4j-0.9.1-src.zip | Bin 0 -> 47035 bytes python/pyspark/mllib/clustering.py | 35 +- python/pyspark/shell.py | 4 +- python/pyspark/sql/column.py | 24 +- python/pyspark/sql/dataframe.py | 26 +- python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/group.py | 6 +- python/pyspark/streaming/context.py | 89 +---- python/pyspark/streaming/util.py | 3 +- .../scala/org/apache/spark/repl/Main.scala | 6 +- sbin/spark-config.sh | 2 +- scalastyle-config.xml | 34 +- .../sql/catalyst/parser/SelectClauseParser.g | 7 + .../sql/catalyst/expressions/UnsafeRow.java | 4 - .../spark/sql/catalyst/CatalystQl.scala | 63 ++-- .../apache/spark/sql/catalyst/SqlParser.scala | 5 +- .../sql/catalyst/analysis/Analyzer.scala | 38 +- .../catalyst/analysis/FunctionRegistry.scala | 2 + .../catalyst/analysis/HiveTypeCoercion.scala | 72 ++-- .../sql/catalyst/analysis/unresolved.scala | 5 + .../catalyst/encoders/ExpressionEncoder.scala | 10 + .../catalyst/expressions/BoundAttribute.scala | 4 +- .../expressions/aggregate/Covariance.scala | 198 +++++++++++ .../expressions/conditionalExpressions.scala | 334 ++++++------------ .../sql/catalyst/expressions/literals.scala | 6 +- .../spark/sql/catalyst/expressions/misc.scala | 251 ++++++++++++- .../expressions/namedExpressions.scala | 6 + .../sql/catalyst/expressions/objects.scala | 4 + .../sql/catalyst/optimizer/Optimizer.scala | 217 ++++++------ .../sql/catalyst/parser/ParseDriver.scala | 24 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 10 - .../plans/logical/basicOperators.scala | 121 ------- .../sql/catalyst/plans/logical/object.scala | 185 ++++++++++ .../plans/physical/partitioning.scala | 7 +- .../spark/sql/catalyst/trees/TreeNode.scala | 9 + .../sql/catalyst/util/NumberConverter.scala | 2 +- .../apache/spark/sql/types/StructType.scala | 3 +- .../spark/sql/catalyst/CatalystQlSuite.scala | 151 +++++++- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../ExpressionTypeCheckingSuite.scala | 4 +- .../analysis/HiveTypeCoercionSuite.scala | 45 ++- .../ConditionalExpressionSuite.scala | 51 +-- .../expressions/MiscFunctionsSuite.scala | 6 +- .../BooleanSimplificationSuite.scala | 2 +- .../optimizer/CombiningLimitsSuite.scala | 3 +- .../EliminateSerializationSuite.scala | 76 ++++ .../optimizer/SimplifyConditionalSuite.scala | 50 +++ ...> SimplifyStringCaseConversionSuite.scala} | 3 +- .../execution/vectorized/ColumnVector.java | 176 +++++++++ .../execution/vectorized/ColumnarBatch.java | 296 ++++++++++++++++ .../vectorized/OffHeapColumnVector.java | 179 ++++++++++ .../vectorized/OnHeapColumnVector.java | 175 +++++++++ .../scala/org/apache/spark/sql/Column.scala | 19 +- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../scala/org/apache/spark/sql/Dataset.scala | 9 +- .../spark/sql/ExperimentalMethods.scala | 5 + .../org/apache/spark/sql/GroupedDataset.scala | 6 +- .../org/apache/spark/sql/SQLContext.scala | 6 +- .../apache/spark/sql/execution/Exchange.scala | 14 +- .../spark/sql/execution/SparkOptimizer.scala | 27 ++ .../apache/spark/sql/execution/SparkQl.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 19 +- .../apache/spark/sql/execution/Window.scala | 2 +- .../spark/sql/execution/basicOperators.scala | 127 ------- .../datasources/DataSourceStrategy.scala | 16 +- .../datasources/LogicalRelation.scala | 8 +- .../datasources/WriterContainer.scala | 212 +++-------- .../sql/execution/datasources/bucket.scala | 3 +- .../datasources/json/JacksonParser.scala | 2 +- .../datasources/parquet/ParquetRelation.scala | 14 +- .../sql/execution/datasources/rules.scala | 18 +- .../apache/spark/sql/execution/objects.scala | 182 ++++++++++ .../org/apache/spark/sql/functions.scala | 42 ++- .../apache/spark/sql/sources/interfaces.scala | 3 - .../apache/spark/sql/JavaDataFrameSuite.java | 4 +- .../apache/spark/sql/JavaDatasetSuite.java | 33 +- .../org/apache/spark/sql/DataFrameSuite.scala | 21 +- .../apache/spark/sql/DatasetCacheSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 40 ++- .../org/apache/spark/sql/QueryTest.scala | 8 +- .../apache/spark/sql/SQLContextSuite.scala | 12 + .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../datasources/json/JsonSuite.scala | 19 +- .../datasources/json/TestJsonData.scala | 4 + .../parquet/ParquetFilterSuite.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../parquet/ParquetReadBenchmark.scala | 2 +- .../vectorized/ColumnarBatchBenchmark.scala | 320 +++++++++++++++++ .../vectorized/ColumnarBatchSuite.scala | 317 +++++++++++++++++ .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../HiveThriftServer2Suites.scala | 4 +- .../spark/sql/hive/ExtendedHiveQlParser.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 10 +- .../org/apache/spark/sql/hive/HiveQl.scala | 24 +- .../org/apache/spark/sql/hive/HiveShim.scala | 23 ++ .../apache/spark/sql/hive/SQLBuilder.scala | 14 +- .../spark/sql/hive/execution/commands.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 8 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../spark/sql/hive/ErrorPositionSuite.scala | 7 +- .../apache/spark/sql/hive/HiveQlSuite.scala | 2 +- .../sql/hive/InsertIntoHiveTableSuite.scala | 4 +- .../sql/hive/LogicalPlanToSQLSuite.scala | 15 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/SQLBuilderTest.scala | 2 +- .../execution/AggregationQuerySuite.scala | 32 ++ .../hive/execution/HiveComparisonTest.scala | 4 +- .../sql/hive/execution/HiveUDFSuite.scala | 7 + .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../spark/sql/hive/orc/OrcFilterSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 8 +- .../sql/sources/BucketedWriteSuite.scala | 29 +- .../sql/sources/hadoopFsRelationSuites.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../spark/streaming/StreamingContext.scala | 36 +- .../streaming/api/java/JavaDStreamLike.scala | 2 +- .../streaming/api/python/PythonDStream.scala | 10 - .../dstream/ConstantInputDStream.scala | 4 +- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 18 +- .../streaming/dstream/InputDStream.scala | 6 +- .../dstream/PluggableInputDStream.scala | 4 +- .../streaming/dstream/RawInputDStream.scala | 4 +- .../dstream/ReceiverInputDStream.scala | 6 +- .../dstream/SocketInputDStream.scala | 4 +- .../streaming/dstream/StateDStream.scala | 6 +- .../spark/streaming/receiver/Receiver.scala | 8 +- .../spark/streaming/util/StateMap.scala | 2 +- .../streaming/BasicOperationsSuite.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/MasterFailureTest.scala | 4 +- .../spark/streaming/StateMapSuite.scala | 7 +- .../streaming/StreamingContextSuite.scala | 2 +- .../spark/streaming/TestSuiteBase.scala | 4 +- .../scheduler/ReceiverTrackerSuite.scala | 4 +- .../streaming/util/WriteAheadLogSuite.scala | 2 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 28 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 4 +- 221 files changed, 4114 insertions(+), 1606 deletions(-) rename core/src/test/{java => scala}/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala (87%) delete mode 100644 python/lib/py4j-0.9-src.zip create mode 100644 python/lib/py4j-0.9.1-src.zip create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/{SimplifyCaseConversionExpressionsSuite.scala => SimplifyStringCaseConversionSuite.scala} (96%) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala diff --git a/LICENSE b/LICENSE index a2f75b817ab37..9c944ac610afe 100644 --- a/LICENSE +++ b/LICENSE @@ -264,7 +264,7 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.9 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.9.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (BSD licence) sbt and sbt-launch-lib.bash (BSD 3 Clause) d3.min.js (https://github.com/mbostock/d3/blob/master/LICENSE) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 97625b94a0e23..40d5066a93f4c 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1173,7 +1173,7 @@ test_that("group by, agg functions", { expect_equal(3, count(mean(gd))) expect_equal(3, count(max(gd))) - expect_equal(30, collect(max(gd))[1, 2]) + expect_equal(30, collect(max(gd))[2, 2]) expect_equal(1, collect(count(gd))[1, 2]) mockLines2 <- c("{\"name\":\"ID1\", \"value\": \"10\"}", diff --git a/bin/pyspark b/bin/pyspark index 5eaa17d3c2016..2ac4a8be250d6 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -67,7 +67,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a97d884f0bf39..51d6d15f66c69 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.9.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index 34ecb19654f1a..3bec5debc2968 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -350,7 +350,7 @@ net.sf.py4j py4j - 0.9 + 0.9.1 org.apache.spark diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a1a1fb01426a0..56cdc22f36261 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -138,7 +138,7 @@ public void write(Iterator> records) throws IOException { final File file = tempShuffleBlockIdPlusFile._2(); final BlockId blockId = tempShuffleBlockIdPlusFile._1(); partitionWriters[i] = - blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics).open(); + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); } // Creating the file to write to and creating a disk writer both involve interacting with // the disk, and can take a long time in aggregate when we open many files, so should be @@ -185,16 +185,19 @@ private long[] writePartitionedFile(File outputFile) throws IOException { boolean threwException = true; try { for (int i = 0; i < numPartitions; i++) { - final FileInputStream in = new FileInputStream(partitionWriters[i].fileSegment().file()); - boolean copyThrewException = true; - try { - lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); - copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); - } - if (!partitionWriters[i].fileSegment().file().delete()) { - logger.error("Unable to delete file for partition {}", i); + final File file = partitionWriters[i].fileSegment().file(); + if (file.exists()) { + final FileInputStream in = new FileInputStream(file); + boolean copyThrewException = true; + try { + lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + if (!file.delete()) { + logger.error("Unable to delete file for partition {}", i); + } } } threwException = false; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index f71b8d154cc24..d1b0bc5d11f46 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -116,8 +116,10 @@ public UnsafeInMemorySorter( * Free the memory used by pointer array. */ public void free() { - consumer.freeArray(array); - array = null; + if (consumer != null) { + consumer.freeArray(array); + array = null; + } } public void reset() { diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 4d20c7369376e..36b536e89c3a4 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -68,11 +68,6 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { logInfo(s"Partition $key not found, computing it") val computedValues = rdd.computeOrReadCheckpoint(partition, context) - // If the task is running locally, do not persist the result - if (context.isRunningLocally) { - return computedValues - } - // Otherwise, cache the values and keep track of any updates in block statuses val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index e25ed0fdd7fd2..7704abc134096 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -97,8 +97,9 @@ abstract class TaskContext extends Serializable { /** * Returns true if the task is running locally in the driver program. - * @return + * @return false */ + @deprecated("Local execution was removed, so this always returns false", "2.0.0") def isRunningLocally(): Boolean /** diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 6c493630997eb..94ff884b742b8 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -33,7 +33,6 @@ private[spark] class TaskContextImpl( override val taskMemoryManager: TaskMemoryManager, @transient private val metricsSystem: MetricsSystem, internalAccumulators: Seq[Accumulator[Long]], - val runningLocally: Boolean = false, val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext with Logging { @@ -85,7 +84,7 @@ private[spark] class TaskContextImpl( override def isCompleted(): Boolean = completed - override def isRunningLocally(): Boolean = runningLocally + override def isRunningLocally(): Boolean = false override def isInterrupted(): Boolean = interrupted diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 2d97cd9a9a208..bda872746c8b8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -32,7 +32,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.9.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 18e8cddbc40db..57108dcedcf0c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -50,7 +50,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) - extends RDD[Pair[T, U]](sc, Nil) + extends RDD[(T, U)](sc, Nil) with Serializable { val numPartitionsInRdd2 = rdd2.partitions.length 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 de7102f5b6245..9dad7944144d8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -95,7 +95,7 @@ abstract class RDD[T: ClassTag]( /** Construct an RDD with just a one-to-one dependency on one parent */ def this(@transient oneParent: RDD[_]) = - this(oneParent.context , List(new OneToOneDependency(oneParent))) + this(oneParent.context, List(new OneToOneDependency(oneParent))) private[spark] def conf = sc.conf // ======================================================================= @@ -970,6 +970,13 @@ abstract class RDD[T: ClassTag]( * apply the fold to each element sequentially in some defined ordering. For functions * that are not commutative, the result may differ from that of a fold applied to a * non-distributed collection. + * + * @param zeroValue the initial value for the accumulated result of each partition for the `op` + * operator, and also the initial value for the combine results from different + * partitions for the `op` operator - this will typically be the neutral + * element (e.g. `Nil` for list concatenation or `0` for summation) + * @param op an operator used to both accumulate results within a partition and combine results + * from different partitions */ def fold(zeroValue: T)(op: (T, T) => T): T = withScope { // Clone the zero value since we will also be serializing it as part of tasks @@ -988,6 +995,13 @@ abstract class RDD[T: ClassTag]( * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are * allowed to modify and return their first argument instead of creating a new U to avoid memory * allocation. + * + * @param zeroValue the initial value for the accumulated result of each partition for the + * `seqOp` operator, and also the initial value for the combine results from + * different partitions for the `combOp` operator - this will typically be the + * neutral element (e.g. `Nil` for list concatenation or `0` for summation) + * @param seqOp an operator used to accumulate results within a partition + * @param combOp an associative operator used to combine results from different partitions */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = withScope { // Clone the zero value since we will also be serializing it as part of tasks diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 0379ca2af6ab3..fca57928eca1b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -74,8 +74,7 @@ private[spark] abstract class Task[T]( attemptNumber, taskMemoryManager, metricsSystem, - internalAccumulators, - runningLocally = false) + internalAccumulators) TaskContext.setTaskContext(context) context.taskMetrics.setHostname(Utils.localHostName()) context.taskMetrics.setAccumulatorsUpdater(context.collectInternalAccumulators) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala index 8ad4656b4dada..3bdba922328c2 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ExecutorListResource.scala @@ -28,9 +28,13 @@ private[v1] class ExecutorListResource(ui: SparkUI) { @GET def executorList(): Seq[ExecutorSummary] = { val listener = ui.executorsListener - val storageStatusList = listener.storageStatusList - (0 until storageStatusList.size).map { statusId => - ExecutorsPage.getExecInfo(listener, statusId) + listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + val storageStatusList = listener.storageStatusList + (0 until storageStatusList.size).map { statusId => + ExecutorsPage.getExecInfo(listener, statusId) + } } } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5feb1dc2e5b74..9cd52d6c2bef5 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -115,7 +115,7 @@ class StageData private[spark]( val status: StageStatus, val stageId: Int, val attemptId: Int, - val numActiveTasks: Int , + val numActiveTasks: Int, val numCompleteTasks: Int, val numFailedTasks: Int, diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 1a29b0f412603..7072a152d6b69 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -52,12 +52,19 @@ private[ui] class ExecutorsPage( private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = listener.storageStatusList + val (storageStatusList, execInfo) = listener.synchronized { + // The follow codes should be protected by `listener` to make sure no executors will be + // removed before we query their status. See SPARK-12784. + val _storageStatusList = listener.storageStatusList + val _execInfo = { + for (statusId <- 0 until _storageStatusList.size) + yield ExecutorsPage.getExecInfo(listener, statusId) + } + (_storageStatusList, _execInfo) + } val maxMem = storageStatusList.map(_.maxMem).sum val memUsed = storageStatusList.map(_.memUsed).sum val diskUsed = storageStatusList.map(_.diskUsed).sum - val execInfo = for (statusId <- 0 until storageStatusList.size) yield - ExecutorsPage.getExecInfo(listener, statusId) val execInfoSorted = execInfo.sortBy(_.id) val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 58c8560a3d049..86bbaa20f6cf2 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.logging -import java.io.{File, FileOutputStream, InputStream} +import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.{IntParam, Utils} @@ -58,20 +58,28 @@ private[spark] class FileAppender(inputStream: InputStream, file: File, bufferSi protected def appendStreamToFile() { try { logDebug("Started appending thread") - openFile() - val buf = new Array[Byte](bufferSize) - var n = 0 - while (!markedForStop && n != -1) { - n = inputStream.read(buf) - if (n != -1) { - appendToFile(buf, n) + Utils.tryWithSafeFinally { + openFile() + val buf = new Array[Byte](bufferSize) + var n = 0 + while (!markedForStop && n != -1) { + try { + n = inputStream.read(buf) + } catch { + // An InputStream can throw IOException during read if the stream is closed + // asynchronously, so once appender has been flagged to stop these will be ignored + case _: IOException if markedForStop => // do nothing and proceed to stop appending + } + if (n > 0) { + appendToFile(buf, n) + } } + } { + closeFile() } } catch { case e: Exception => logError(s"Error writing stream to file $file", e) - } finally { - closeFile() } } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index cb8bd04e496a7..30aa94c8a5971 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -82,15 +82,6 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before assert(value.toList === List(5, 6, 7)) } - test("get uncached local rdd") { - // Local computation should not persist the resulting value, so don't expect a put(). - when(blockManager.get(RDDBlockId(0, 0))).thenReturn(None) - - val context = new TaskContextImpl(0, 0, 0, 0, null, null, Seq.empty, runningLocally = true) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } - test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager val context = TaskContext.empty() diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 4e72b89bfcc40..76451788d2406 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -178,7 +178,7 @@ class DoubleRDDSuite extends SparkFunSuite with SharedSparkContext { test("WorksWithOutOfRangeWithInfiniteBuckets") { // Verify that out of range works with two buckets val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN)) - val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0) + val buckets = Array(-1.0/0.0, 0.0, 1.0/0.0) val histogramResults = rdd.histogram(buckets) val expectedHistogramResults = Array(1, 1) assert(histogramResults === expectedHistogramResults) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 504e5780f3d8a..e111e2e9f6163 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -76,7 +76,7 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi test("check spark-class location correctly") { val conf = new SparkConf - conf.set("spark.mesos.executor.home" , "/mesos-home") + conf.set("spark.mesos.executor.home", "/mesos-home") val listenerBus = mock[LiveListenerBus] listenerBus.post( diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index e33408b94e2cf..ef6ce04e3ff28 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -105,7 +105,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte new Answer[(TempShuffleBlockId, File)] { override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = File.createTempFile(blockId.toString, null, tempDir) + val file = new File(tempDir, blockId.name) blockIdToFileMap.put(blockId, file) temporaryFilesCreated.append(file) (blockId, file) @@ -166,6 +166,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ true) assert(temporaryFilesCreated.nonEmpty) assert(writer.getPartitionLengths.sum === outputFile.length()) + assert(writer.getPartitionLengths.filter(_ == 0L).size === 4) // should be 4 zero length files assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get assert(shuffleWriteMetrics.shuffleBytesWritten === outputFile.length()) @@ -174,6 +175,41 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(taskMetrics.memoryBytesSpilled === 0) } + test("only generate temp shuffle file for non-empty partition") { + // Using exception to test whether only non-empty partition creates temp shuffle file, + // because temp shuffle file will only be cleaned after calling stop(false) in the failure + // case, so we could use it to validate the temp shuffle files. + def records: Iterator[(Int, Int)] = + Iterator((1, 1), (5, 5)) ++ + (0 until 100000).iterator.map { i => + if (i == 99990) { + throw new SparkException("intentional failure") + } else { + (2, 2) + } + } + + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + blockResolver, + shuffleHandle, + 0, // MapId + taskContext, + conf + ) + + intercept[SparkException] { + writer.write(records) + } + + assert(temporaryFilesCreated.nonEmpty) + // Only 3 temp shuffle files will be created + assert(temporaryFilesCreated.count(_.exists()) === 3) + + writer.stop( /* success = */ false) + assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted + } + test("cleanup of intermediate files after errors") { val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala similarity index 87% rename from core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index f200ff36c7dd5..d21ce73f4021e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -19,18 +19,18 @@ package org.apache.spark.shuffle.sort import java.io.{File, FileInputStream, FileOutputStream} +import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.mockito.{Mock, MockitoAnnotations} import org.scalatest.BeforeAndAfterEach +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.storage._ import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkFunSuite} class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -64,12 +64,15 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa } test("commit shuffle files multiple times") { - val lengths = Array[Long](10, 0, 20) val resolver = new IndexShuffleBlockResolver(conf, blockManager) + val lengths = Array[Long](10, 0, 20) val dataTmp = File.createTempFile("shuffle", null, tempDir) val out = new FileOutputStream(dataTmp) - out.write(new Array[Byte](30)) - out.close() + Utils.tryWithSafeFinally { + out.write(new Array[Byte](30)) + } { + out.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths, dataTmp) val dataFile = resolver.getDataFile(1, 2) @@ -77,12 +80,15 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(dataFile.length() === 30) assert(!dataTmp.exists()) + val lengths2 = new Array[Long](3) val dataTmp2 = File.createTempFile("shuffle", null, tempDir) val out2 = new FileOutputStream(dataTmp2) - val lengths2 = new Array[Long](3) - out2.write(Array[Byte](1)) - out2.write(new Array[Byte](29)) - out2.close() + Utils.tryWithSafeFinally { + out2.write(Array[Byte](1)) + out2.write(new Array[Byte](29)) + } { + out2.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths2, dataTmp2) assert(lengths2.toSeq === lengths.toSeq) assert(dataFile.exists()) @@ -90,20 +96,27 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(!dataTmp2.exists()) // The dataFile should be the previous one - val in = new FileInputStream(dataFile) val firstByte = new Array[Byte](1) - in.read(firstByte) + val in = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in.read(firstByte) + } { + in.close() + } assert(firstByte(0) === 0) // remove data file dataFile.delete() + val lengths3 = Array[Long](10, 10, 15) val dataTmp3 = File.createTempFile("shuffle", null, tempDir) val out3 = new FileOutputStream(dataTmp3) - val lengths3 = Array[Long](10, 10, 15) - out3.write(Array[Byte](2)) - out3.write(new Array[Byte](34)) - out3.close() + Utils.tryWithSafeFinally { + out3.write(Array[Byte](2)) + out3.write(new Array[Byte](34)) + } { + out3.close() + } resolver.writeIndexFileAndCommit(1, 2, lengths3, dataTmp3) assert(lengths3.toSeq != lengths.toSeq) assert(dataFile.exists()) @@ -111,9 +124,13 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa assert(!dataTmp2.exists()) // The dataFile should be the previous one - val in2 = new FileInputStream(dataFile) val firstByte2 = new Array[Byte](1) - in2.read(firstByte2) + val in2 = new FileInputStream(dataFile) + Utils.tryWithSafeFinally { + in2.read(firstByte2) + } { + in2.close() + } assert(firstByte2(0) === 2) } } 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 67210e5d4c50e..62e6c4f7932df 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -45,20 +45,18 @@ import org.apache.spark.util._ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { - private val conf = new SparkConf(false).set("spark.app.id", "test") + var conf: SparkConf = null var store: BlockManager = null var store2: BlockManager = null var store3: BlockManager = null var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null - conf.set("spark.authenticate", "false") - val securityMgr = new SecurityManager(conf) - val mapOutputTracker = new MapOutputTrackerMaster(conf) - val shuffleManager = new HashShuffleManager(conf) + val securityMgr = new SecurityManager(new SparkConf(false)) + val mapOutputTracker = new MapOutputTrackerMaster(new SparkConf(false)) + val shuffleManager = new HashShuffleManager(new SparkConf(false)) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - conf.set("spark.kryoserializer.buffer", "1m") - val serializer = new KryoSerializer(conf) + val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) @@ -79,15 +77,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def beforeEach(): Unit = { super.beforeEach() - rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") - conf.set("os.arch", "amd64") - conf.set("spark.test.useCompressedOops", "true") + conf = new SparkConf(false) + .set("spark.app.id", "test") + .set("spark.kryoserializer.buffer", "1m") + .set("spark.test.useCompressedOops", "true") + .set("spark.storage.unrollFraction", "0.4") + .set("spark.storage.unrollMemoryThreshold", "512") + + rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) conf.set("spark.driver.port", rpcEnv.address.port.toString) - conf.set("spark.storage.unrollFraction", "0.4") - conf.set("spark.storage.unrollMemoryThreshold", "512") master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, new LiveListenerBus)), conf, true) @@ -98,6 +98,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { + conf = null if (store != null) { store.stop() store = null @@ -473,34 +474,22 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-9591: getRemoteBytes from another location when Exception throw") { - val origTimeoutOpt = conf.getOption("spark.network.timeout") - try { - conf.set("spark.network.timeout", "2s") - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") - store3 = makeBlockManager(8000, "executor3") - val list1 = List(new Array[Byte](4000)) - store2.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store3.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - var list1Get = store.getRemoteBytes("list1") - assert(list1Get.isDefined, "list1Get expected to be fetched") - // block manager exit - store2.stop() - store2 = null - list1Get = store.getRemoteBytes("list1") - // get `list1` block - assert(list1Get.isDefined, "list1Get expected to be fetched") - store3.stop() - store3 = null - // exception throw because there is no locations - intercept[BlockFetchException] { - list1Get = store.getRemoteBytes("list1") - } - } finally { - origTimeoutOpt match { - case Some(t) => conf.set("spark.network.timeout", t) - case None => conf.remove("spark.network.timeout") - } + conf.set("spark.shuffle.io.maxRetries", "0") + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + store3 = makeBlockManager(8000, "executor3") + val list1 = List(new Array[Byte](4000)) + store2.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store3.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") + store2.stop() + store2 = null + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") + store3.stop() + store3 = null + // exception throw because there is no locations + intercept[BlockFetchException] { + store.getRemoteBytes("list1") } } diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 98d1b28d5a167..b367cc8358342 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,12 +18,17 @@ package org.apache.spark.util import java.io._ +import java.util.concurrent.CountDownLatch import scala.collection.mutable.HashSet import scala.reflect._ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files +import org.apache.log4j.{Appender, Level, Logger} +import org.apache.log4j.spi.LoggingEvent +import org.mockito.ArgumentCaptor +import org.mockito.Mockito.{atLeast, mock, verify} import org.scalatest.BeforeAndAfter import org.apache.spark.{Logging, SparkConf, SparkFunSuite} @@ -188,6 +193,67 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { testAppenderSelection[FileAppender, Any](rollingStrategy("xyz")) } + test("file appender async close stream abruptly") { + // Test FileAppender reaction to closing InputStream using a mock logging appender + val mockAppender = mock(classOf[Appender]) + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] + + // Make sure only logging errors + val logger = Logger.getRootLogger + logger.setLevel(Level.ERROR) + logger.addAppender(mockAppender) + + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) + + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) + + appender.awaitTermination() + + // If InputStream was closed without first stopping the appender, an exception will be logged + verify(mockAppender, atLeast(1)).doAppend(loggingEventCaptor.capture) + val loggingEvent = loggingEventCaptor.getValue + assert(loggingEvent.getThrowableInformation !== null) + assert(loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } + + test("file appender async close stream gracefully") { + // Test FileAppender reaction to closing InputStream using a mock logging appender + val mockAppender = mock(classOf[Appender]) + val loggingEventCaptor = new ArgumentCaptor[LoggingEvent] + + // Make sure only logging errors + val logger = Logger.getRootLogger + logger.setLevel(Level.ERROR) + logger.addAppender(mockAppender) + + val testOutputStream = new PipedOutputStream() + val testInputStream = new PipedInputStream(testOutputStream) with LatchedInputStream + + // Close the stream before appender tries to read will cause an IOException + testInputStream.close() + testOutputStream.close() + val appender = FileAppender(testInputStream, testFile, new SparkConf) + + // Stop the appender before an IOException is called during read + testInputStream.latchReadStarted.await() + appender.stop() + testInputStream.latchReadProceed.countDown() + + appender.awaitTermination() + + // Make sure no IOException errors have been logged as a result of appender closing gracefully + verify(mockAppender, atLeast(0)).doAppend(loggingEventCaptor.capture) + import scala.collection.JavaConverters._ + loggingEventCaptor.getAllValues.asScala.foreach { loggingEvent => + assert(loggingEvent.getThrowableInformation === null + || !loggingEvent.getThrowableInformation.getThrowable.isInstanceOf[IOException]) + } + } + /** * Run the rolling file appender with data and see whether all the data was written correctly * across rolled over files. @@ -228,4 +294,15 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { file.getName.startsWith(testFile.getName) }.foreach { _.delete() } } + + /** Used to synchronize when read is called on a stream */ + private trait LatchedInputStream extends PipedInputStream { + val latchReadStarted = new CountDownLatch(1) + val latchReadProceed = new CountDownLatch(1) + abstract override def read(): Int = { + latchReadStarted.countDown() + latchReadProceed.await() + super.read() + } + } } diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index e4373f79f7922..53034a25d46ab 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -84,13 +84,13 @@ hadoop-yarn-server-web-proxy-2.2.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar @@ -160,7 +160,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 7478181406d07..a23e260641aeb 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.3.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar @@ -151,7 +151,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index faffb8bf398a5..6bedbed1e3355 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -79,13 +79,13 @@ hadoop-yarn-server-web-proxy-2.4.0.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar @@ -152,7 +152,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index e703c7acd3876..7bfad57b4a4a6 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -85,13 +85,13 @@ htrace-core-3.0.4.jar httpclient-4.3.2.jar httpcore-4.3.2.jar ivy-2.4.0.jar -jackson-annotations-2.4.4.jar -jackson-core-2.4.4.jar +jackson-annotations-2.5.3.jar +jackson-core-2.5.3.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.4.4.jar +jackson-databind-2.5.3.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-scala_2.10-2.4.4.jar +jackson-module-scala_2.10-2.5.3.jar jackson-xc-1.9.13.jar janino-2.7.8.jar jansi-1.4.jar @@ -158,7 +158,7 @@ pmml-agent-1.2.7.jar pmml-model-1.2.7.jar pmml-schema-1.2.7.jar protobuf-java-2.5.0.jar -py4j-0.9.jar +py4j-0.9.1.jar pyrolite-4.9.jar quasiquotes_2.10-2.0.0-M8.jar reflectasm-1.07-shaded.jar diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index bf1a000f46791..5ab285eae99b7 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -355,11 +355,21 @@ def standardize_jira_ref(text): return clean_text + +def get_current_ref(): + ref = run_cmd("git rev-parse --abbrev-ref HEAD").strip() + if ref == 'HEAD': + # The current ref is a detached HEAD, so grab its SHA. + return run_cmd("git rev-parse HEAD").strip() + else: + return ref + + def main(): global original_head os.chdir(SPARK_HOME) - original_head = run_cmd("git rev-parse HEAD")[:8] + original_head = get_current_ref() branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) @@ -449,5 +459,8 @@ def main(): (failure_count, test_count) = doctest.testmod() if failure_count: exit(-1) - - main() + try: + main() + except: + clean_up() + raise diff --git a/dev/run-tests.py b/dev/run-tests.py index 8726889cbc777..795db0dcfbab9 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -529,9 +529,14 @@ def main(): run_apache_rat_checks() # style checks - if not changed_files or any(f.endswith(".scala") for f in changed_files): + if not changed_files or any(f.endswith(".scala") + or f.endswith("scalastyle-config.xml") + for f in changed_files): run_scala_style_checks() - if not changed_files or any(f.endswith(".java") for f in changed_files): + if not changed_files or any(f.endswith(".java") + or f.endswith("checkstyle.xml") + or f.endswith("checkstyle-suppressions.xml") + for f in changed_files): # run_java_style_checks() pass if not changed_files or any(f.endswith(".py") for f in changed_files): diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 1fc6596164124..93a8c15e3ec30 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -426,7 +426,7 @@ def contains_file(self, filename): # No other modules should directly depend on this module. root = Module( name="root", - dependencies=[], + dependencies=[build], # Changes to build should trigger all tests. source_file_regexes=[], # In order to run all of the tests, enable every test profile: build_profile_flags=list(set( diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 424ce6ad7663c..3cb5d2be2a91a 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -70,19 +70,10 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_PROFILE in "${HADOOP_PROFILES[@]}"; do echo "Performing Maven install for $HADOOP_PROFILE" - $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar install:install -q \ - -pl '!assembly' \ - -pl '!examples' \ - -pl '!external/flume-assembly' \ - -pl '!external/kafka-assembly' \ - -pl '!external/twitter' \ - -pl '!external/flume' \ - -pl '!external/mqtt' \ - -pl '!external/mqtt-assembly' \ - -pl '!external/zeromq' \ - -pl '!external/kafka' \ - -pl '!tags' \ - -DskipTests + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE jar:jar jar:test-jar install:install clean -q + + echo "Performing Maven validate for $HADOOP_PROFILE" + $MVN $HADOOP2_MODULE_PROFILES -P$HADOOP_PROFILE validate -q echo "Generating dependency manifest for $HADOOP_PROFILE" mkdir -p dev/pr-deps diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 93cd0c1c61ae9..d0be03286849a 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -152,11 +152,7 @@ clusters = KMeans.train(parsedData, 2, maxIterations=10, runs=10, initializationMode="random") # Evaluate clustering by computing Within Set Sum of Squared Errors -def error(point): - center = clusters.centers[clusters.predict(point)] - return sqrt(sum([x**2 for x in (point - center)])) - -WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) +WSSSE = clusters.computeCost(parsedData) print("Within Set Sum of Squared Error = " + str(WSSSE)) # Save and load model diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 20b35612cab95..aac8f7560a4f8 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -590,7 +590,8 @@ val parsedData = data.map { line => // Building the model val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) +val stepSize = 0.00000001 +val model = LinearRegressionWithSGD.train(parsedData, numIterations, stepSize) // Evaluate model on training examples and compute training error val valuesAndPreds = parsedData.map { point => @@ -655,8 +656,9 @@ public class LinearRegression { // Building the model int numIterations = 100; + double stepSize = 0.00000001; final LinearRegressionModel model = - LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); + LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations, stepSize); // Evaluate model on training examples and compute training error JavaRDD> valuesAndPreds = parsedData.map( @@ -706,7 +708,7 @@ data = sc.textFile("data/mllib/ridge-data/lpsa.data") parsedData = data.map(parsePoint) # Build the model -model = LinearRegressionWithSGD.train(parsedData) +model = LinearRegressionWithSGD.train(parsedData, iterations=100, step=0.00000001) # Evaluate the model on training data valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 3193e17853483..ed720f1039f94 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -202,7 +202,7 @@ where each application gets more or fewer machines as it ramps up and down, but additional overhead in launching each task. This mode may be inappropriate for low-latency requirements like interactive queries or serving web requests. -To run in coarse-grained mode, set the `spark.mesos.coarse` property to false in your +To run in fine-grained mode, set the `spark.mesos.coarse` property to false in your [SparkConf](configuration.html#spark-properties): {% highlight scala %} @@ -266,13 +266,11 @@ See the [configuration page](configuration.html) for information on Spark config Property NameDefaultMeaning spark.mesos.coarse - false + true - If set to true, runs over Mesos clusters in - "coarse-grained" sharing mode, - where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per - Spark task. This gives lower-latency scheduling for short queries, but leaves resources in use - for the whole duration of the Spark job. + If set to true, runs over Mesos clusters in "coarse-grained" sharing mode, where Spark acquires one long-lived Mesos task on each machine. + If set to false, runs over Mesos cluster in "fine-grained" sharing mode, where one Mesos task is created per Spark task. + Detailed information in 'Mesos Run Modes'. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b058833616433..bc89c781562bd 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2151,6 +2151,11 @@ options. ... {% endhighlight %} + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + ## Upgrading From Spark SQL 1.4 to 1.5 - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index 2cb8010cdc07f..69e836fc1d06a 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -62,5 +62,9 @@ def parseVector(line): for i in range(args.k): print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, "sigma = ", model.gaussians[i].sigma.toArray())) + print("\n") + print(("The membership value of each vector to all mixture components (first 100): ", + model.predictSoft(data).take(100))) + print("\n") print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index f6b0ecb02c100..b6c2916254056 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -30,7 +30,7 @@ lines = sc.textFile(sys.argv[1], 1) sortedCount = lines.flatMap(lambda x: x.split(' ')) \ .map(lambda x: (int(x), 1)) \ - .sortByKey(lambda x: x) + .sortByKey() # This is just a demo on how to bring all the sorted data back to a single node. # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala index 1fce4ba7efd60..90b817b23e156 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala @@ -58,6 +58,12 @@ object DenseGaussianMixture { (clusters.weights(i), clusters.gaussians(i).mu, clusters.gaussians(i).sigma)) } + println("The membership value of each vector to all mixture components (first <= 100):") + val membership = clusters.predictSoft(data) + membership.take(100).foreach { x => + print(" " + x.mkString(",")) + } + println() println("Cluster labels (first <= 100):") val clusterLabels = clusters.predict(data) clusterLabels.take(100).foreach { x => diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index ce1a62060ef6c..50216b9bd40f2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -23,15 +23,15 @@ import java.net.ServerSocket import java.util.Random /** Represents a page view on a website with associated dimension data. */ -class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) +class PageView(val url: String, val status: Int, val zipCode: Int, val userID: Int) extends Serializable { - override def toString() : String = { + override def toString(): String = { "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID) } } object PageView extends Serializable { - def fromString(in : String) : PageView = { + def fromString(in: String): PageView = { val parts = in.split("\t") new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt) } @@ -58,9 +58,9 @@ object PageViewGenerator { 404 -> .05) val userZipCode = Map(94709 -> .5, 94117 -> .5) - val userID = Map((1 to 100).map(_ -> .01) : _*) + val userID = Map((1 to 100).map(_ -> .01): _*) - def pickFromDistribution[T](inputMap : Map[T, Double]) : T = { + def pickFromDistribution[T](inputMap: Map[T, Double]): T = { val rand = new Random().nextDouble() var total = 0.0 for ((item, prob) <- inputMap) { @@ -72,7 +72,7 @@ object PageViewGenerator { inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0 } - def getNextClickEvent() : String = { + def getNextClickEvent(): String = { val id = pickFromDistribution(userID) val page = pickFromDistribution(pages) val status = pickFromDistribution(httpStatus) @@ -80,7 +80,7 @@ object PageViewGenerator { new PageView(page, status, zipCode, id).toString() } - def main(args : Array[String]) { + def main(args: Array[String]) { if (args.length != 2) { System.err.println("Usage: PageViewGenerator ") System.exit(1) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index d87b86932dd41..aa530a7121bd0 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -26,20 +26,20 @@ import org.slf4j.{Logger, LoggerFactory} private[sink] trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine - @transient private var log_ : Logger = null + @transient private var _log: Logger = null // Method to get or create the logger for this object protected def log: Logger = { - if (log_ == null) { + if (_log == null) { initializeIfNecessary() var className = this.getClass.getName // Ignore trailing $'s in the class names for Scala objects if (className.endsWith("$")) { className = className.substring(0, className.length - 1) } - log_ = LoggerFactory.getLogger(className) + _log = LoggerFactory.getLogger(className) } - log_ + _log } // Log methods that take only a String diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 1bfa35a8b3d1d..74bd0165c6209 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -41,12 +41,12 @@ import org.apache.spark.util.Utils private[streaming] class FlumeInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel, enableDecompression: Boolean -) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel, enableDecompression) @@ -60,7 +60,7 @@ class FlumeInputDStream[T: ClassTag]( * which are not serializable. */ class SparkFlumeEvent() extends Externalizable { - var event : AvroFlumeEvent = new AvroFlumeEvent() + var event: AvroFlumeEvent = new AvroFlumeEvent() /* De-serialize from bytes. */ def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { @@ -75,12 +75,12 @@ class SparkFlumeEvent() extends Externalizable { val keyLength = in.readInt() val keyBuff = new Array[Byte](keyLength) in.readFully(keyBuff) - val key : String = Utils.deserialize(keyBuff) + val key: String = Utils.deserialize(keyBuff) val valLength = in.readInt() val valBuff = new Array[Byte](valLength) in.readFully(valBuff) - val value : String = Utils.deserialize(valBuff) + val value: String = Utils.deserialize(valBuff) headers.put(key, value) } @@ -109,7 +109,7 @@ class SparkFlumeEvent() extends Externalizable { } private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = { + def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { val event = new SparkFlumeEvent event.event = in event @@ -118,13 +118,13 @@ private[streaming] object SparkFlumeEvent { /** A simple server that implements Flume's Avro protocol. */ private[streaming] -class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { - override def append(event : AvroFlumeEvent) : Status = { +class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol { + override def append(event: AvroFlumeEvent): Status = { receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } - override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { + override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = { events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 8a087474d3169..54d8c8b03f206 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -58,11 +58,11 @@ class DirectKafkaInputDStream[ U <: Decoder[K]: ClassTag, T <: Decoder[V]: ClassTag, R: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, val kafkaParams: Map[String, String], val fromOffsets: Map[TopicAndPartition, Long], messageHandler: MessageAndMetadata[K, V] => R - ) extends InputDStream[R](ssc_) with Logging { + ) extends InputDStream[R](_ssc) with Logging { val maxRetries = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRetries", 1) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 21e971bc4d23b..48f3064e25fc1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -384,7 +384,7 @@ object KafkaCluster { val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => val hpa = hp.split(":") if (hpa.size == 1) { - throw new SparkException(s"Broker not the in correct format of : [$brokers]") + throw new SparkException(s"Broker not in the correct format of : [$brokers]") } (hpa(0), hpa(1).toInt) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 2274ccd34d1aa..902cd83c51bf9 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -48,12 +48,12 @@ class KafkaInputDStream[ V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], useReliableReceiver: Boolean, storageLevel: StorageLevel - ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](_ssc) with Logging { def getReceiver(): Receiver[(K, V)] = { if (!useReliableReceiver) { diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index fec638f70f34b..27919c09511fb 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -156,7 +156,7 @@ class KafkaRDD[ var requestOffset = part.fromOffset var iter: Iterator[MessageAndOffset] = null - // The idea is to use the provided preferred host, except on task retry atttempts, + // The idea is to use the provided preferred host, except on task retry attempts, // to minimize number of kafka metadata requests private def connectLeader: SimpleConsumer = { if (context.attemptNumber > 0) { diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 80e2df62de3fe..7b9aee39ffb76 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -50,7 +50,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite private var ssc: StreamingContext = _ private var tempDirectory: File = null - override def beforeAll() : Unit = { + override def beforeAll(): Unit = { kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 116c170489e96..079bd8a9a87ea 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -38,11 +38,11 @@ import org.apache.spark.streaming.receiver.Receiver private[streaming] class MQTTInputDStream( - ssc_ : StreamingContext, + _ssc: StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](ssc_) { + ) extends ReceiverInputDStream[String](_ssc) { private[streaming] override def name: String = s"MQTT stream [$id]" diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index a48eec70b9f78..bdd57fdde3b89 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -39,11 +39,11 @@ import org.apache.spark.streaming.receiver.Receiver */ private[streaming] class TwitterInputDStream( - ssc_ : StreamingContext, + _ssc: StreamingContext, twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel - ) extends ReceiverInputDStream[Status](ssc_) { + ) extends ReceiverInputDStream[Status](_ssc) { private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index de749626ec09c..6a73bc0e30d05 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.util.Random -import com.amazonaws.auth.{DefaultAWSCredentialsProviderChain, BasicAWSCredentials} +import com.amazonaws.auth.{BasicAWSCredentials, DefaultAWSCredentialsProviderChain} import com.amazonaws.regions.RegionUtils import com.amazonaws.services.kinesis.AmazonKinesisClient import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 3321c7527edb4..5223c81a8e0e0 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -24,10 +24,10 @@ import com.amazonaws.services.kinesis.model.Record import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.streaming.{Duration, StreamingContext, Time} import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.scheduler.ReceivedBlockInfo -import org.apache.spark.streaming.{Duration, StreamingContext, Time} private[kinesis] class KinesisInputDStream[T: ClassTag]( _ssc: StreamingContext, diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index abb9b6cd32f1c..48ee2a959786b 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import scala.util.control.NonFatal import com.amazonaws.auth.{AWSCredentials, AWSCredentialsProvider, DefaultAWSCredentialsProviderChain} -import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessorCheckpointer, IRecordProcessor, IRecordProcessorFactory} +import com.amazonaws.services.kinesis.clientlibrary.interfaces.{IRecordProcessor, IRecordProcessorCheckpointer, IRecordProcessorFactory} import com.amazonaws.services.kinesis.clientlibrary.lib.worker.{InitialPositionInStream, KinesisClientLibConfiguration, Worker} import com.amazonaws.services.kinesis.model.Record diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala index 2de6195716e5c..15ac588b82587 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -24,9 +24,9 @@ import com.amazonaws.services.kinesis.model.Record import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, StreamingContext} import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.{Duration, StreamingContext} object KinesisUtils { /** diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index d85b4cda8ce98..e916f1ee0893b 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.kinesis import org.scalatest.BeforeAndAfterAll -import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) extends KinesisFunSuite with BeforeAndAfterAll { @@ -158,9 +158,9 @@ abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) testBlockRemove: Boolean = false ): Unit = { require(shardIds.size > 1, "Need at least 2 shards to test") - require(numPartitionsInBM <= shardIds.size , + require(numPartitionsInBM <= shardIds.size, "Number of partitions in BlockManager cannot be more than the Kinesis test shards available") - require(numPartitionsInKinesis <= shardIds.size , + require(numPartitionsInKinesis <= shardIds.size, "Number of partitions in Kinesis cannot be more than the Kinesis test shards available") require(numPartitionsInBM <= numPartitions, "Number of partitions in BlockManager cannot be more than that in RDD") diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala index 645e64a0bc3a0..e1499a8220991 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kinesis -import java.util.concurrent.{TimeoutException, ExecutorService} +import java.util.concurrent.{ExecutorService, TimeoutException} import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ @@ -28,7 +28,7 @@ import org.mockito.Matchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer -import org.scalatest.{PrivateMethodTester, BeforeAndAfterEach} +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ import org.scalatest.mock.MockitoSugar diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index e5c70db554a27..fd15b6ccdc889 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -27,8 +27,8 @@ import com.amazonaws.services.kinesis.model.Record import org.mockito.Matchers._ import org.mockito.Matchers.{eq => meq} import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.mock.MockitoSugar import org.apache.spark.streaming.{Duration, TestSuiteBase} import org.apache.spark.util.Utils diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 78263f9dca65c..ee6a5f0390d04 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -25,10 +25,11 @@ import scala.util.Random import com.amazonaws.regions.RegionUtils import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream import com.amazonaws.services.kinesis.model.Record +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.{StorageLevel, StreamBlockId} @@ -38,7 +39,6 @@ import org.apache.spark.streaming.kinesis.KinesisTestUtils._ import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult import org.apache.spark.streaming.scheduler.ReceivedBlockInfo import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkContext} abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFunSuite with Eventually with BeforeAndAfter with BeforeAndAfterAll { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index fc36e12dd2aed..d048fb5d561f3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -21,7 +21,6 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.SparkException -import org.apache.spark.SparkContext._ import org.apache.spark.graphx.lib._ import org.apache.spark.rdd.RDD @@ -379,7 +378,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runUntilConvergenceWithOptions]] */ def personalizedPageRank(src: VertexId, tol: Double, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) } @@ -392,7 +391,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @see [[org.apache.spark.graphx.lib.PageRank$#runWithOptions]] */ def staticPersonalizedPageRank(src: VertexId, numIter: Int, - resetProb: Double = 0.15) : Graph[Double, Double] = { + resetProb: Double = 0.15): Graph[Double, Double] = { PageRank.runWithOptions(graph, numIter, resetProb, Some(src)) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala index f79f9c7ec448f..b4bec7cba5207 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala @@ -41,8 +41,8 @@ class ReplicatedVertexView[VD: ClassTag, ED: ClassTag]( * shipping level. */ def withEdges[VD2: ClassTag, ED2: ClassTag]( - edges_ : EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { - new ReplicatedVertexView(edges_, hasSrcId, hasDstId) + _edges: EdgeRDDImpl[ED2, VD2]): ReplicatedVertexView[VD2, ED2] = { + new ReplicatedVertexView(_edges, hasSrcId, hasDstId) } /** 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 3f203c4eca485..96d807f9f9ceb 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 @@ -102,8 +102,8 @@ class ShippableVertexPartition[VD: ClassTag]( extends VertexPartitionBase[VD] { /** Return a new ShippableVertexPartition with the specified routing table. */ - def withRoutingTable(routingTable_ : RoutingTablePartition): ShippableVertexPartition[VD] = { - new ShippableVertexPartition(index, values, mask, routingTable_) + def withRoutingTable(_routingTable: RoutingTablePartition): ShippableVertexPartition[VD] = { + new ShippableVertexPartition(index, values, mask, _routingTable) } /** 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 f508b483a2f1b..7c680dcb99cd2 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 @@ -32,7 +32,7 @@ import org.apache.spark.util.collection.BitSet * example, [[VertexPartition.VertexPartitionOpsConstructor]]). */ private[graphx] abstract class VertexPartitionBaseOps - [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] + [VD: ClassTag, Self[X] <: VertexPartitionBase[X]: VertexPartitionBaseOpsConstructor] (self: Self[VD]) extends Serializable with Logging { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 35b26c998e1d9..46faad2e68c50 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -138,7 +138,7 @@ object PageRank extends Logging { // edge partitions. prevRankGraph = rankGraph val rPrb = if (personalized) { - (src: VertexId , id: VertexId) => resetProb * delta(src, id) + (src: VertexId, id: VertexId) => resetProb * delta(src, id) } else { (src: VertexId, id: VertexId) => resetProb } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index a95f0f17517d1..269c89c310550 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -231,11 +231,9 @@ private List buildPySparkShellCommand(Map env) throws IO // the pyspark command line, then run it using spark-submit. if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".py")) { System.err.println( - "WARNING: Running python applications through 'pyspark' is deprecated as of Spark 1.0.\n" + + "Running python applications through 'pyspark' is not supported as of Spark 2.0.\n" + "Use ./bin/spark-submit "); - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.exit(-1); } checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); @@ -258,9 +256,10 @@ private List buildPySparkShellCommand(Map env) throws IO private List buildSparkRCommand(Map env) throws IOException { if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { - appResource = appArgs.get(0); - appArgs.remove(0); - return buildCommand(env); + System.err.println( + "Running R applications through 'sparkR' is not supported as of Spark 2.0.\n" + + "Use ./bin/spark-submit "); + System.exit(-1); } // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS // env variable. diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala index 7443097492d82..7a651a37ac77e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeParams.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, BoostingStrategy => OldBoostingStrategy, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.{Entropy => OldEntropy, Gini => OldGini, Impurity => OldImpurity, Variance => OldVariance} import org.apache.spark.mllib.tree.loss.{Loss => OldLoss} -import org.apache.spark.sql.types.{DoubleType, DataType, StructType} +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} /** * Parameters for Decision Tree-based algorithms. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala index 6a3b20c88d2d2..a689b09341450 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/GaussianMixtureModelWrapper.scala @@ -40,5 +40,9 @@ private[python] class GaussianMixtureModelWrapper(model: GaussianMixtureModel) { SerDe.dumps(JavaConverters.seqAsJavaListConverter(modelGaussians).asJava) } + def predictSoft(point: Vector): Vector = { + Vectors.dense(model.predictSoft(point)) + } + def save(sc: SparkContext, path: String): Unit = model.save(sc, path) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 16bc45bcb627f..42fe27024f8fc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -75,7 +75,7 @@ class GaussianMixtureModel @Since("1.3.0") ( */ @Since("1.5.0") def predict(point: Vector): Int = { - val r = computeSoftAssignments(point.toBreeze.toDenseVector, gaussians, weights, k) + val r = predictSoft(point) r.indexOf(r.max) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index 2fce3ff641101..b30ecb80209d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -387,6 +387,32 @@ class LocalLDAModel private[spark] ( } } + /** + * Predicts the topic mixture distribution for a document (often called "theta" in the + * literature). Returns a vector of zeros for an empty document. + * + * Note this means to allow quick query for single document. For batch documents, please refer + * to [[topicDistributions()]] to avoid overhead. + * + * @param document document to predict topic mixture distributions for + * @return topic mixture distribution for the document + */ + @Since("2.0.0") + def topicDistribution(document: Vector): Vector = { + val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.toBreeze.toDenseMatrix.t).t) + if (document.numNonzeros == 0) { + Vectors.zeros(this.k) + } else { + val (gamma, _) = OnlineLDAOptimizer.variationalTopicInference( + document, + expElogbeta, + this.docConcentration.toBreeze, + gammaShape, + this.k) + Vectors.dense(normalize(gamma, 1.0).toArray) + } + } + /** * Java-friendly version of [[topicDistributions]] */ 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 a7e1b76df6a7d..dee898827f30f 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 @@ -151,7 +151,7 @@ class Word2Vec extends Serializable with Logging { /** context words from [-window, window] */ private var window = 5 - private var trainWordsCount = 0 + private var trainWordsCount = 0L private var vocabSize = 0 @transient private var vocab: Array[VocabWord] = null @transient private var vocabHash = mutable.HashMap.empty[String, Int] @@ -159,13 +159,13 @@ class Word2Vec extends Serializable with Logging { private def learnVocab(words: RDD[String]): Unit = { vocab = words.map(w => (w, 1)) .reduceByKey(_ + _) + .filter(_._2 >= minCount) .map(x => VocabWord( x._1, x._2, new Array[Int](MAX_CODE_LENGTH), new Array[Int](MAX_CODE_LENGTH), 0)) - .filter(_.cn >= minCount) .collect() .sortWith((a, b) => a.cn > b.cn) @@ -179,7 +179,7 @@ class Word2Vec extends Serializable with Logging { trainWordsCount += vocab(a).cn a += 1 } - logInfo("trainWordsCount = " + trainWordsCount) + logInfo(s"vocabSize = $vocabSize, trainWordsCount = $trainWordsCount") } private def createExpTable(): Array[Float] = { @@ -332,7 +332,7 @@ class Word2Vec extends Serializable with Logging { val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) val syn0Modify = new Array[Int](vocabSize) val syn1Modify = new Array[Int](vocabSize) - val model = iter.foldLeft((bcSyn0Global.value, bcSyn1Global.value, 0, 0)) { + val model = iter.foldLeft((bcSyn0Global.value, bcSyn1Global.value, 0L, 0L)) { case ((syn0, syn1, lastWordCount, wordCount), sentence) => var lwc = lastWordCount var wc = wordCount @@ -543,7 +543,12 @@ class Word2VecModel private[spark] ( val cosVec = cosineVec.map(_.toDouble) var ind = 0 while (ind < numWords) { - cosVec(ind) /= wordVecNorms(ind) + val norm = wordVecNorms(ind) + if (norm == 0.0) { + cosVec(ind) = 0.0 + } else { + cosVec(ind) /= norm + } ind += 1 } wordList.zip(cosVec) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index ffae0e7ed0ca4..1250bc1a07cb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -17,8 +17,8 @@ package org.apache.spark.mllib.fpm -import java.lang.{Iterable => JavaIterable} import java.{util => ju} +import java.lang.{Iterable => JavaIterable} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -29,16 +29,15 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, render} -import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} +import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.fpm.FPGrowth._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index f22f2df320f0d..4a3fb06469818 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -109,7 +109,9 @@ private[stat] object ChiSqTest extends Logging { } i += 1 distinctLabels += label - features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + val brzFeatures = features.toBreeze + (startCol until endCol).map { col => + val feature = brzFeatures(col) allDistinctFeatures(col) += feature (col, feature, label) } @@ -122,7 +124,7 @@ private[stat] object ChiSqTest extends Logging { pairCounts.keys.filter(_._1 == startCol).map(_._3).toArray.distinct.zipWithIndex.toMap } val numLabels = labels.size - pairCounts.keys.groupBy(_._1).map { case (col, keys) => + pairCounts.keys.groupBy(_._1).foreach { case (col, keys) => val features = keys.map(_._2).toArray.distinct.zipWithIndex.toMap val numRows = features.size val contingency = new BDM(numRows, numLabels, new Array[Double](numRows * numLabels)) 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 c73774fcd8c46..07ba0d8ccb2a8 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 @@ -25,10 +25,10 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl._ import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index 729a211574822..1b71256c585bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -22,8 +22,8 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.impl.PeriodicRDDCheckpointer import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.BoostingStrategy import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.BoostingStrategy import org.apache.spark.mllib.tree.impl.TimeTracker import org.apache.spark.mllib.tree.impurity.Variance import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index a684cdd18c2fc..570a76f960796 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -26,9 +26,9 @@ import org.apache.spark.Logging import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, NodeIdCache, TimeTracker, TreePoint} import org.apache.spark.mllib.tree.impurity.Impurities diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index faef60e084cc1..ea23196d2c801 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -366,7 +366,8 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { (0, 0.99504), (1, 0.99504), (1, 0.99504), (1, 0.99504)) - val actualPredictions = ldaModel.topicDistributions(docs).map { case (id, topics) => + val actualPredictions = ldaModel.topicDistributions(docs).cache() + val topTopics = actualPredictions.map { case (id, topics) => // convert results to expectedPredictions format, which only has highest probability topic val topicsBz = topics.toBreeze.toDenseVector (id, (argmax(topicsBz), max(topicsBz))) @@ -374,9 +375,17 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext { .values .collect() - expectedPredictions.zip(actualPredictions).forall { case (expected, actual) => - expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D) + expectedPredictions.zip(topTopics).foreach { case (expected, actual) => + assert(expected._1 === actual._1 && (expected._2 ~== actual._2 relTol 1E-3D)) } + + docs.collect() + .map(doc => ldaModel.topicDistribution(doc._2)) + .zip(actualPredictions.map(_._2).collect()) + .foreach { case (single, batch) => + assert(single ~== batch relTol 1E-3D) + } + actualPredictions.unpersist() } test("OnlineLDAOptimizer with asymmetric prior") { diff --git a/pom.xml b/pom.xml index 0eac212754320..fc5cf970e0601 100644 --- a/pom.xml +++ b/pom.xml @@ -152,9 +152,9 @@ 1.7.7 hadoop2 0.7.1 - 1.4.0 + 1.6.1 - 0.10.1 + 0.10.2 4.3.2 @@ -167,7 +167,7 @@ ${scala.version} org.scala-lang 1.9.13 - 2.4.4 + 2.5.3 1.1.2 1.1.2 1.2.0-incubating @@ -1046,6 +1046,12 @@ zookeeper ${zookeeper.version} ${hadoop.deps.scope} + + + org.jboss.netty + netty + + org.codehaus.jackson @@ -1771,7 +1777,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4 + 1.4.1 enforce-versions @@ -1786,6 +1792,19 @@ ${java.version} + + + + org.jboss.netty + + true + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 9ba9f8286f10c..41856443af49b 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -91,11 +91,16 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" + // The resolvers setting for MQTT Repository is needed for mqttv3(1.0.1) + // because spark-streaming-mqtt(1.6.0) depends on it. + // Remove the setting on updating previousSparkVersion. val previousSparkVersion = "1.6.0" val fullId = "spark-" + projectRef.project + "_2.10" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), - binaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value)) + binaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value), + sbt.Keys.resolvers += + "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases") } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 0d5f938d9ef5c..4206d1fada421 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -135,6 +135,18 @@ object MimaExcludes { ) ++ Seq( // SPARK-12510 Refactor ActorReceiver to support Java ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.streaming.receiver.ActorReceiver") + ) ++ Seq( + // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkSink.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log__="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.Logging.org$apache$spark$streaming$flume$sink$Logging$$_log_="), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log_"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.sink.TransactionProcessor.org$apache$spark$streaming$flume$sink$Logging$$log__=") ) case v if v.startsWith("1.6") => Seq( diff --git a/python/docs/Makefile b/python/docs/Makefile index 4cec74f057fbe..b6d24d8599cf7 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -7,7 +7,7 @@ SPHINXBUILD = sphinx-build PAPER = BUILDDIR = _build -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.9.1-src.zip) # User-friendly check for sphinx-build ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) diff --git a/python/lib/py4j-0.9-src.zip b/python/lib/py4j-0.9-src.zip deleted file mode 100644 index dace2d0fe3b0bd01d24c07e7747aa68980896598..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 44846 zcma&NbC7RMmnB@bZQZiXTej``m2KO$ZR3`0+tw}Hw!i1?nVyb!V!r6kh?8;t$h~q$ zWUP~CuUx4h4GIPW^shER_a^oqFaNs(2f_og_h7YTP*s5j0)-1{R1H+PP<3&K1p)>; z1qA~7k51v=!j<6ljnV!I|JQ@~pW&`1j!qVKw)FNM{|D$_VE-48p5Ff+r>956OGLp) z&qV?AFYtdsLH$Q=H}z?rs_;NSApc~E^`GFD2CfErMt0WLCPvQxOycz4%yOk;W4FPH z@_nt>@6;=yy(-c1v4>=hF=o2e`IyKWz6Fl4B~)U=twtlwrzp2i^W3{Lbt_t|i$@{1 zL8|~NYU60inT^etOk}AWKoqwgZ_|N+T?fW zO(QMj{-f3*IAh)FXXfAyDRL6Yj1s$KTe46Ci4o?7VJuxsV-e4XHESH1Hs#(eowgx| z{F=LJ9qEBGlgR;FRwT9_nP~6g8Qw`C<{Zqu?n837_qq|o%kv9&4rouH;yvgy&1_V= z#E{{%?}0kWQGGGh)V+0op?M3nT6;JN21BZ*Zk;O5V6tfPP{^1EiC1)-Noo}gGF4>k zK5cVaDBCIHYj!mZ!ZYW=B-vWQXy|?~O@f@r?Bg0lnl~j3DcwLU2KlN|T$9evV?F;k zhP3YMuY%B9J7Dh8eT>%=o%_PEeu#R!TG;-F?Am3DOOVE#f<~Lx!{h11Xe+AwU(%%f8XNv?kckCb{U?=}`_9Jv@W! ze0GMI7kRpf_rDj7`Zh6t=ZHz)7F&*kbJE4scr<`?9`949>PJz^rO}X<5rM%=re95~ zg{4bw!^ z7m!N2g3OnuHK}r&a0OFWVaWyZ!9`hD1Pe_0=!0Ah+8&dT391c%sR?HG8%oxS z)MSYgf)bSUO$4OJEw+Uh3Pc*O7=h2u7c+4xe(+|J)=Nx3-YNer?qzE=5gYIgM#d@e zCSD`H*pI6ra~~1I#vNrmAkB>!_XLUJ&+2ai0@h(U_|3#8jr;3>8FQ5}_t%wW;hN|& zw&ybzoRptgBz3XGdLq_S6-po#bV*%Y#jP0!7X(!7D{vNn`ZK3$g27|F7BVh2IrTfM zk_NMsJt@#sG6->Qzn{#e9M~T5BzcnR49I-%`Ab#TgLPZx6eN(deoR0CJaBJ1CU-0z zR*0ds$tiuI3#@Z?)5xGqUVau-x-jNb2TU`l8mfv46e_OUm*((2F?D@Z$k zk))H$E zYWaH~FqbgWGko8x@SvLdEoX{$`c(Y4c-2KJ&9%a>KzNS`R)Q?s%3H+&5QG=S{11OJ#Fa#l$28(zc>Z(6$U!T~;KdO<}Le5Y8 zrVcnr+NTsC=5wLok?oq3`@!^PsspuNRZ`&-&!X6(L5)%Kg!A@{0{~R>Ah(-Adg4c| zXX2OyBJ7ndTCdiZ+^;ELV-jEml8L%X6CgKM5WzC>q5yPIjCHBR`mI(mXucfwLa-*c z%BFb!+pdI6Ps>NkH#X*Sddx6=fvxp${Bm=J;TQ{Air+h9WyJXt6{T@q)wzINc zXjK+}dpE1$vp1RoTFV7pD{G8L*ZOOSJUEtYBJ5Rh*$i}SJmcUUSRlyDTUN+AU>jp) zn;*pm_iZG0-K_8kR)FQBTG#gaH;`9?3HmZTn;`7K%tUH;Z74DBVBy13FZ`jwb>b__ zm~&xdf_SL;(7;njjRH)3en)}Rx>OxAX+qEd(4CU1Iv=s-leofgMkCtKR-)x8C$*37Na**iDBJSOC&1N@nTt6z zza!vgmFq@4sfbcmSKi;}PU-xI6KBX5D%xCFaQVL45{IeMhWBU?Dz^Y_b)P_2*3lgS zouj^dBEWUvM4CpgLam^reZ{9QW8dDiQ~?59)hH4{CoX~|3)R-RL^1_i&{{%!0Cm;F zwZ8oKQGrRO(vE5JVB>ryd8uL9VCCPeG?N_MCDNg4d8$BkDfeGRlaodKg$EpFX5uHk zq9@{JpwFN3@>Xfrdj3#@t-ZFzt=P87ikkw#V8mFBA7+26EABq)Tmw-~&z4+#ZIcQ5 z=EOpxiY60E4lBBl+u4E%(|oA(q3CGmepdS`JiSpVGi-*l^M1scBPdLHz%B+6&p0hP z4iM$#V=e8FJQUDNZZdA~Ha4vF?Zc{uh0Byr(|` z=v#j>)QR|F89^ms`Te1OP=Uk~q@uOfpw;lCc^H;uYrfi2Nb}=`BJj0>szJ7jVvFZ9 z;aod!Gm%yifYwlDC8r}TLuQB5<|Y3HV@^~n5F!qh-Of_DUy-0=l^B`!cquW$AHM&d z-_nMz^BJ2B!(fT_dUg*2*REm)r!=B&kwGmiCfu&lnt_@R-mV`J*f^|$y~^A^N>{EQ zw^Z$KjejWdU}>=gCy+2g=r&JNfCyw5g?hcN)0xRe%RauF7Q8>Q;Km`qji1fWO~lK) zYfrtTe5?Ebts|Xtt~>eG>hQuD{_FZ&63+{qC%^4zt@!#EV@o}RY8u1PlkLy$=nHXY zgh4ViK%AxPO}9qYK)R5{MgHaeg+-MUp*bJxCPpoB0FfLks9E1SJ4`2L9L-=>rH)7) zeIC^%DJZE~OAxp_bLuV;eROZpqUt&_|IfhHV)B`k{GUWK7Dfg|GY@a{vfso8#%*?f zOadJ9FWi|Q6BP1@^j!d1nbnsCuakw1Cv07ybHa-B{zoM7c1}iS{O3&lYbD)uHN_KuRbcWnQh^$jCR(z)%O9Q-NgaV+D9B$xLF|_s+QVmov-VKhs(ye6S`MT z(afw!#cm1_Wg4$Od4>7QbzmC+d zN(6jfKWBIIZY|F@!^>7I;pxdQ4QJ7wcWNlIx4Da#HPMeGQ5+P8d5s|I0~*-*9m*pA zhL#aM5f6M#OumylxE*a66?JJPw=wxosLo!%Z6&SgvZittVD06@i}6V%_7PKx(l(k{ zoDDwER1cG}d~ z%@X?X(t!W)#nUu)J)17^y4CDuO#-H0-QCS7LhX`{lz!{mRs%b6z53X5ZB{6KDLHN3 zI=`{7OB_*;s=J2n0yik}?=-O!yqbu`V1XCIGiFT-5SBcqm|gfqx>ZO}uWK3Qc? zE7;9AWTkc#Sc#F|khV1~T7tCE#SBq{sged2a?(vNH8F9Hssm8OR?D>ZZQT28rw`s* zR(nA|nC(_s`Dt`hc1m6ZTHKjTDI<*@wyopa?M6`+?w7jMtk~(&hOp=EJWUDt8Gg^z zM71AedPtO1`v7aU7xYr^7ysDNpUiF9^jeaaNzeprm)^&QOf_?*D_9&r2JDp;3oWip zS%+6~-Rb&qc<~vHIx#866?63R(w?kSD^@y2!zN=-9`sZ6fM1U(#Zf_k|BxzS@vgF} z>D!1M%Q{^Q*R)ICgkr#Dr_D{{)z6Caik|jnUn^M!YOS@^*7CD@o^C-rifVsq1a^{8 zDhknlfd+=#AIwk+QR9l4dB0IsHeAb&4HZ8dCskgp8@%j|j5b4~t~EIqe9#XvNCcAC zXyiER7&&;WD?u0p?FkO?oDQWWqf>oZHu1%~a@+XX65R@*^fji;J#3H*5mF$`L4GHkWJL>Px=fbTigs;6bb=NbS5?JGUv% zfh^-_zvg1KW#d8gX2#JWh}qWJ_HzC>p=YnS_51fSO*qosL~(MCv)A5?hznN`bOrDE~{K2E|E49N$Ka#x-tiC2>`CbxF=J0()y1 zef7y+(itU!P|xx(9cR|#UPF|`h3UpqE_-(_ixg^bt42U$>OLWDt3_#WEf{#SMsw z$Rz8E$SD=3Ccj2{o_mGY35b80x{#8j*!VChmB;>86-#+U;?)_#Uz(XJ61X(Yj;r)bAe~5Ny)VFz4{*+kf;ym39U0 zqo0W_Rob~(0dO;~A}yfC$_=SZxWa$tPCP@zL2cqq%1Ac3q0Q~YnP&f9$^S9@%<()) zHf}8eYfQ9&_vL%9KzS>Hm2tA+``wL^4Y;&&&`!NqM0{|BAcg^*WUhicgo`Wwc^gB6 zXC?{UCBF`ewvj}slt`2?(zMo>HMbn}yt*vSIsd%cOG4w6h_MtOgFD6_N~tQGgRRKm zg~3fb5Vzi;5`FMFE-Va!0|BU)7g`LTDwV0I-^IUK%=GKac;6PB09X0^COzlqC;WF> z=;d}<$0Gs(but10A^j&U>>cf#?fy|O{ySpeSpB0~+;{qd%YO`Sw+Sskbi(eq0*@*x zyWww`@%ZFxk2T3`SUxd6N}SWZ@7RSFQz|7c+B#sO$%@QGu?%`hw%BM-^8N~A9Y^Jd zmQ1#ej8A~yfMA-Gn0X1L91HStFTM+9sGlkeBic?NF~V)&(`r08q7fac_cx74eifIT z;X9FELEA%1@|JhnG}*A(g`}MsXO|q09i}bWOA<~XNQ~A*o_rY(ePx9UP>rjK%HC{z zln`JJc||((LRoMgmCzBfN;NHRH01;Dr;UG?jYJwMN-Z$EIwm2J9!*>=vZ?d z2b~Vj8`7*ZpbEzy5`t%)Y)4WbXvONmlo_3Rqd!Y9Z2uMbX&zM_ z;l-j6tJ?`NmBc-#A2^B1T_2FuiG^TV!{N=9esHem*S{}66yx;hjke@SG5N-0#}4k$ zb+%07*7hj23YE37V$Yq?H(<_vXZmn9mBX-;!PsL%mrAsV^t^>x)Y|s!miOTP^uE&J zst0#HB8~W+3UwyFp^*b;3(yG@5RPIzNuN9jFj-mdEa|*Wy)pGBoD&du8Of1!ET z9FOrN*jQP1=5~cdZEk`vUF@u^uT*0n^X2Tok6vD2%QfIzpfqU#FLOA#Hr6mGZDC8F zg}hk;f$FqRJ9j|bkH%PbGS_Ym%J3|1VQa@#*D>12gb(#Yu%$+TRCsl-D@TC;+wGFZ zP_}AG212cTv(nw|*v^Z(;mM1y`tg!qTEas}Tb7dR+INPpu*dGLJ>GjXkvo11>DvcC zWIdTPTbBK9#<{m;e@c(>>MY{-9zjb)dgz;Ac9cso3(QFNh&?}B3iNgcS9jiZf6DO9 za@SXM{Armlvu*F`#pA!jQXg)Z8}Ms1lUxcbsOhh0L6{_(Amw<~9+0&{B5lsBxFv(w z^=n6c^Dtc-#4fR$@RB7yk%0g3WarV-)bLhO59&UG`-aAL_?<&2|0oa}Pj4 zzs4h*^{92A96&Km9Nd=sHdGzy<@}9t#@xX$8{jFc{70^f(St3hGdlIl)F|}`jAvSi zA0B`b3w?1|(fFsf4rifpx?;o>Gn0j-Fl|5&pM`A?h$i9rHdQeexzxuk%rZM`x84S9 zK&8;*7qluDik0O;gVc0Y`Zo>7Yy-m4hECBW@|ilangXp$w)MbhH?jU8y(n=jRQ^DP zi}Hw&L+>^Ft-` zx4IyJ@P^bapUK?FRRI@*@V55-6P6Z7O@(7#gc=e~jR-SFyaULzw$^-t`xoeKganmR z9!FLo!@!ENLVUgj0+|P)1l0gY-EZ)p9AbETHjFz+2p(@~@$shMJ7NO<~5Xmf1p?@6Ai z4U$&I`SAD>C<7T2nOA2%Kxhtaz+dK>cp5kacT9yIHOb?+KO`8x(%=bUUh^I1*RsTt z0$(bLWlT^84cqko?jmPd016+FI|gDKfGm4jg84V~4Zn7cAx|H9Jj*;nuasIh*zEuA z^%bIHS)Td_VR)3PprxP>V9 z$00L)v9DAT`{FIfmN{9BF1Tdj!z7wLmmzf{wZeRR9=CHyn*hwfA5IBVz#DEC0eqs~`NeWTl7}>e{ignk1 zH<){OSd>4+sG*yHt#q2uXCQ}YR^kdE=F@wX&5;!-i3su4m@%cKJ3v#f;?zY zi;a)Yg8j2Xh#jkd3RN=obBG1(2pxjt8J2#Ti?#rj>ET%w9rrE(#V9bgFNKdlFjygS zmB>HVZl62L$8$Jz)SKULUGfC7&^Fbx;D1!wG#Go7OKZxjpQ)bhCBhN!CXsl{#ek6# z5Gn~*PqT_ZR}%7{@}LWMOeuSHIByZ9MIh8N>AOX~3>OPM?62V)+gbGvk`$qw;6!`Z55n z!#zWxEs~pXQwp?H`y9OVnDG!9|6BexC9`(*`%k%UT%k>_()8eL=6X#|z+2@0&v41| z#jYz`6Wp8H0Of|F^!O{+p))& z0R+7KLoX>v!*|`(jt3saqdOG2GE_{+#{TuQfnMPcpCxz@#!H%#T~lmgAePd^SL2lH zsUD)vMq+Bntg7(!MrXf_!A>x!DX)_4q@)HWkFu11nM9zjHXu{0HHAGjUX+*CfJZ2a z8p?L*oV+na)-m`46yZ{wQ!uxnM8af@%o&Xs1yc8TY&CZ!25-;FO|(-7z|D@l>mp6im4DMu zVFf(FZaUZheD4RdTp10#LB0qgegC%j5&mk(q~jRNKQ)Ro2}M^XAfPb^;-yt{ zIaQgh>N|aG>^j9 z21~PzGsm}jGZLFBo%+G`C?NAk+WMjSzQ4SP8`F){9p6s=$YS>E?VCjLT6m&Lg}vTDHYemuY@@-vW7?si#gpoZ*l6tKjI5q3;c%)02;$ zm7mq)?gFOi52mHs5R@^lNJHa7gVW8RTZq-^6Mj-bk$jRY?mIkF1X9;O3gg4FXNtcX zFou`4!>`JgMfgiuZ12q4`Kly+J()kog6KOG9PRIyF~->};=M%L$SA~R`p!!Z&NvW| zW(cx=@Jo$flpq+v?;z@iLDZ_M#QAreFh+F3a-?5kkVJlyLXXC@8jY^Lq?)D;J)ojV z9-?sZa0aTZuq4HJBEg{RbF*;5z|(D+?LKS)A7uxl-hFwJzj5M>f*ej2L$m9ofDKVq z#=P`Ncz=>JpM=!9Wi5yC6^tsPs*XX&p;k3nO?A9p$@DzaLfljQOX#k9gl;hz>aLgD zd8;63?a}NB(-&bBfp+`%$;)1_xYMH@w==xd!wE#s*^HsKXX9~2+-u|g+s(b{w1J7{ zghex_2ke&}cN^Y2mfW}Ltiml3qB+)Q%sl=@J38HcG3%gHv~ndu`0B|+E#FHlZgt>i zw;?rPabDnC19+Zs=v;Bpu*CN_ObmF*gx{n*^8snC1BS0hWX0N-5m|J6pjRbCoukP^ zD6YB!3!fBFwFeZq2YDs;XCYH%O@#b`k~F?Ubq9^_gW9oWZK|@G7*u@j76r>3?6%i; zjVb036S>BjEqrxBl7tar$7Tf{Aj~ab`v$t zQh0qwtEJ=;k%pQ-g%aL=bwLXHQD}BE3Gpa>nGqJ_Z$zJ!2X1WG*4@lA`XoByp9QrW z=sOA;1SFZ{-*dshTC0jGNWIia_KSPYj4F2wZco2tHNNAE^M=OJrtR`*G2A`5RQdDT zKr~HnE2(#H%rGYR=*!cY7L%Beu<^dX6T zW zqK2 zP?XVNW`C*m1OD%(hkpzT|EK+*%3e>;!q&oBPw&5}?En9fVEZ}a>x^Ur78xA zTR4wt?|rzXyD4Qz&$E32ir=nsX1crY_tRG2F4ErJnd9;SS-MWq`bTBk%(14dk^;OD z9y56ouatXplB^#>%FxdChj@9hIr)x7S3~*@<)U{2LThz`y z^eYsGj2h+hk^0UNhDo+I8ob>`(wI(CFAKK_!Vq;p=*J5>U_N$0e=nrw=1a~9@Lk`T zFMgi_^1a^ekV6xmuI*MW{e-doIiTmfo`#AeLj~n1s%*BW=r_X;fp(bfWC8YBa5K*c zK6w%!jaW8Ykt545yp~@Wi%DLAJ5!EI$I5(SdkNx#BYcbB(Gxd9u=pA}a-LO7gSa;J zNjZNBBq#R^N)Ax*@N~VQh$o&8;VUATTwBK49kb7{4?RTB?!QSD4FV4<4=rnYoybkx z9WsIv_)FbjPtnZE%IbH=D`SWK%q%^*vBYI}zkVGXb_~Tis@KIQVW(YAD06$2_b*_TqD}3k#T-fN`Wu3nP|@R?{YF$M++qi|<#N8LFeR>)oWe5V_f*-RRAB!gv-kS1%_I zUd-s>Yq*anA;_co)b8jFBBFTFs*Y)?ahQ*8AdM&tiy)9W$6z86$bumQ7(v;5>+|bs zx=jz|k z1?pTV7Scfw9+4pN>^ySQ^NcM*jY*C1F$^w`XZTpD!FEbLx};*Pp_@t=!Hal%ywVrz zVL_kr-LLTv=P6n~T|t!vj?YKQa?sCHfu^xXM-jMaEQJvvw>r+uRC8*AN#8)2{u|F z^%6BygsBQ)!1IjwY$FKp6(UC{@CeDxn2b6QrT~gKP8ur3ike*p6s&xx9SE)(Z{7|Z z80xf(3^vyb4I6F;6M2m+{r8Syjv+1}#9R+9&llv#uu-ewcj}DjRLBBM_&$L49*eGm9zsBA+Fh;UWe`3C zLri4h)RMKfx29umFUFd=|Bh zLG$1{5)VQR$@KwZffYSw73DNVgV4JwVqrtf5-~;l>HEe>#%?&L-ASZx(1VNUrqraQ zB8(fi57XYs1#mRsO<3Xw;JET^J7;H$}L|4*6353>xXPya4+ zG%{l_IpSl;B~G=P=TwE=LZ(D}KFj%5vK9h{0{oFZfr{lU8Tdqpdh<(lKm((hA+lvv ze^u!4s6uJI6edq?v>Ys}Y2gRab`qf*GBUzn0R!9tP$ey#PB6HLAP+;wUPNwVDKq-m z$b7~Dkg?Hz&L$GfhJ($&wBe9ndtWO4<_#~o%jSEHj;Jwb*jzoGy$oIU4wy2UJAZ1A z(o1wuRLmUU5&VpCbWM{e1{Xx4&T1CIXV>B8d%%&;S|ibXr0ga&xlwC2#S>|3eNo%p zh@9-qK!NT3L)8%db|Do>>hp=4M1+Z_;1<|<*36mST8Gwb@~1=k15nTB6-hmtk*)T` zZ7p$-#70bqBAGOp(CTWwOMwTO(V>PL17*#_X)bNnXR9SI$Sh+4Ax937~qYKI_kh9`I6i zWKZjWq=#P7e4{8mu7h2j3`yN(r4mXJV}!j@O-BA4-1iOjpt2KuH=>UkS{*H^>qub3 zGImTX-XD1LHjaP$OJRYPFjn zdlA4jBz0I+>s^JYn|%Sv`0D9kXF9UU4Or{`Q*MAfhed@~h=z|2

encyYo)zHI9u zo5qy0N6^f}S#Ew^R_97A|Km2K(8q2<^}T(u{5~dq*wBa_-mUZ2Q)Ae}9h2Klj4!Bu zzniMEr1I2IdWCQQqO}`YF_PQBk8s;LgtQqG+fp?mh5bye9SiBxOnz z9bKMvkI|;oLEc>5&fbUkr5kM7SR?D=`$1W5_i)aqVarp0Zsoibc=MTy9bf?1MI5|O=3eg>S<%=v|32{mi+1|q?WiQ)dPhh{JB zY7l*exDVsX{IOlACc}nWisdFl5xILGg5RRSnPdFA!4e%WQ)iLJJExS_#VfAh>Fg0! z2V~kdSOF98VW?rk{%JHg$}~4xC8JP6XFs_#4qcI$ByH-e&8L{2FE{T0u08vzuH{zc zo0^Fvl>_O;!!Q>ZC@au(uqNf#PTXuYZm-=q7IJF#&q!7C(PD1laDadkx3{*$03TJq z1hF!ZXi&ZirKgeuQVFTs1nSaV7^Lh1Mu}@#`5vug+!ozicF_3S_ffYXo6$N{Jt`a2 z34s55ebck)!{%Q9@rM^7t5Ojeml6B2rf0)wE_E4Vj{5KA`fe=AS5>}6&wHgC8Nvr| zVJE4maQd8Q{fE*9;^NKGiciGDicR>{Bjn2Ub`%Nm(HFg+-qIZu$I;<*O*tk1JqCSd zBs5R6iHsLtk?%Bx)c09 zmUBxCht?Qq$!DK41ksFSB`Ttu8egL;A>u?Er&}t2<58pE7mMFSZ2t3`41TfV?GWT@ zSh&nPKK@9Je|?q`uL`|D@uU>iOi8_&fW0<&YgS(omSM?H{R=cBr^9C#@b-6GR)6EH zChW%r_t$l1TD00>;QR6m%L_td1#I}bpy|V}nxNg}P3g}Q)wDG)uUq*aa4+kP7=m*b zQTH72h%`?QG?QZoc9;2f+{!xcOjV7ZF|s97(3>UcN$W~BG>^5Co&xJ6s?rHo6gTShP@WtM+~Ku& zB@1M%7X`7%(mB*C5bHb1i;CCpCa%fBkPG0cio^);Yyh;G6{&Pf`tu|QN_wJ05^|X< zZy|tsW1g8a)4Pc@sFP>tu~vq$TPX3Z*todo7$~C376hjgSQQ2&YbRP!`Iih}y2uq= zl;egktNVE0Lys#TQ|x z0Rj}^yQzQXk6Fme<=3XcP;v-nlX}k+4at)1N?V2%MhL9;@Jv#eVl>AjC0+Qkn1s8NBMq)iP-hu(iN4riZDLzT7D)rh9e*0oy;hbtqABsmcA2eLb*xfjB`X zO3`hDRh)XZah}K}9i}lo&;JU*bi`C8@_9xhUpyFZl2fW^Vcvw=0TVR_wrtR^q z$4@@Te3X#Q@AQ7z-*-PhN{Oc=t3lZYMo#dQMbcEte)QHv;ojb}sXoJ)$-~lQq2%u{ z8ZNnBxImYcAYE2|^nCVE8(--X^!5ri;pI!RX3a9b$NbWb0VO(3isC~!=EKl`&%>Dl zrkfdUmwhhK``a5n7>Jz*#Nd&>GuUj9li9x%OFX2iTSXjpHUxVb0XdBYlJ%))5u?E( zyhDIR*Ro=a4a0GuN@Z%)!paBoH6^mJ{%GKU_&)W}JhU$4!W^-A(91Fz)RQ z8CR#*Bt_?{tDOu|QjLBYI%ve+s;AVP92~X(yCaFc6XIpfj$pi%3rd@tpjcglP>#?2 zv1N?9xb?Md46&9ODPHgI zh-gU_*%TATzl{tHu`WJ0%2B11YA7?lXsvCNj8W6ECIQS9>1!XFJDK%>_Kn8ZQH8tmrU(QDl#9-D7=udO5WV7!1W`B{kCq(zPBN zHIOjko7fnVr77mL>Ta>52$cX{8OP~`&d_4^Vex#qo}$0EnPpjB;eHEqa(Cs<#mkmU zekisp@BP$`5?qpnW92Zt?y!chhJV~^{jq?04Yc8xuj6sKb=)QHowL3slimSD6(Kgq zfYMY(bGzz%Y2DqKkGI2KM`+WEZ_A_7^(j=7op7)Q62?%dH8il@)3;CO4yNKb>dT^& ze+D^M1hEaRIA*KQI5eoa)epE*vC~k)SG%uLfI1Du90}n*ZA<$`0tTtahV1V!>d zm1ud7tMh3K!?^+%-v(pHoAb2o?jaYNxCIg8M|NjX|x-`6W%{?720_Tpri?HZ_N)jPTnQN zG&Pm9Ge5klJ3v=SsYwVmvu%8?m+n6F17G=F2R2aRyM-ykBJY3E z40LsIJ*ic)@9WI9V+uFqiImaw zHod*`J&(PM6TEkJ=jpjV*YEWN+(dMZ;4C~Qj(leh2?1r+ZlQ5iDwA_E3|<4>2`~1= zQ|el!&qD634qls5Hnv{$_H#Ad8}ks?_6B& zY!KPp`nSiLRUYUcArdrU$l;!lh`LrXpU0Zaj_}a5BjG`Y(CnfeWX_lLGXpsKi12}x zQAum>;t-d?pg#shj)!?7EociMmSV<1c65fBjbU| zKrFQ(c!cd%!{~ok;thPJU9ZX3FQ6r71-3)CGUeY@}%cKFC#X%R);|l_D6$FI_yeF}|qLQsp zgRLIE z@IT(nU;nNP|Lc~Yo{@pIwV{ELm7a;afsMVj$$z5_rzpru4+MeA2nQ6+Z`!z})_ z&`4A;N31!rR)exKKt5ZNBo;x&ROT;11mUt=|rHUL7ogcj$&xaFJREMkC z2V5M2@GaI!>**{X77C2&Pse2ei&wp(_p zfQ#q>4}KH|t-WWxVwZ8}ugCMk&e)-tf$3@J z9P1-y>0wjs$^@N*>q60-?+Sx0XBs3ZtBV!O?)Ty?`kr@YZ1NH_IbAXNzYY@qOSzZX zM>o9qXLWF(fq-!TeTDu*xwkQ}|DUM#!)iNr|JZ1KXY~vQq^VUEvj(I^$>v4`BqY52 z%Pbo?5f3(#(&49ETGEF4nQCu6{55lcndoyaBD`W(6N15}0QMMN%>t)mPxJRp zl0*y^zNCMztXvgbhjKV2RayWf(hwYPwR-%hrC=6D0KSlN!eH38|5mHqn9)U-Rbd7G z13$`rVni5&;gmyKyh(e3{u`Y?ey;%~{8HHJFvR*=Aj=MnV*M!RZ!K)V+((lE(v({Y zkxHfRDbb?YpS4Se%W)!=Z6?l#ap`An*DEX~FLE=AO>z~_@ihGBb&jn8|JOAGHljx| zCKr&^vU;B1Qe<={^RMZ2i;}xIG2$Bz!Cg(p?h&O&fGe21meTTtS&&sdoBH2TTq&oa zQw;*-+1B3inwfEb-jNE+68}n}U`y(j-WHXR%R8E6oWaT!{MZ%z$R_BVKP%-o5;kN~ zRtF z^En@?0f+CFwO`n=Hf!YfhcWP!yNZlNG}!q&!gH<8)?o95)Uxhd{3)u(V^WLc!PPVm zAwSL@mE;5`I}?_Z z4gY~9+3&%~VLf|%^nkN2c*NG3>|awLDw;vOA=T~h0%iiglmKFqR9a!?P<n z^xSm1PW73MHdEW|@0g9bBg8mQ)@zc6-gIoHP=U>BeO93+qwoWYo-RptTpVoh11;VO zK~-D}8Z?gVn!Nx+`xce+uS`>r6L?RbI!QVi;lc?H7Trbpnw0ur3*a+ zq5g9*SsfYZANKt3%gg`k_{`M8*1+1r)5P(ALZD^!9jlFh2=t;yyos3wj+!-)kEAia z5+=y8)={PyY@MfF){r7~FZ8l{gZ7hr(Ja|XUrmT2>DKXa&9gP`3Ru;Y(#KIEk#d|N z_2BE9A0Xbouki|)VNv@kGoml$E^#dq09L94}3$4p3bk^N1n;u zD)i|w{d9Z%MAwbv15PR}c_tLNRU=6ybaXSVeB0c23WR<$x(6jM&aD{;Zs6GQPH^aX znSl(Y!zN5m5?#SUT-*`Z60Q-kiZXw*=?;%6XDds=VFQr-5>7s8%2?19W!JSn+8AxmR!BC z8k?!}p>V4OuiC7;Lz!L7v}t-w8moWuSMFNKy|c#20)cBfJ>bnv7nftUJip zDn0UCeiu}=cF`vRYbgUKW}DP~{UPr#HS^|aUaX$4BtDPKUWG|&XS=I^FKjZytW)d? z1$%~D-1|&l{j~IL`Lw1mp-PMcMkwiSt%8(VO?#|DP0_J`{;yw&PBGxnKrkR6(0}%> z|JO!kZ1F$h*_DR0-2ppN&xbnFOo6KF}ch5FJ#c^c8GCDwqKWG z5nU?+Im70xQd0|-KHfaNCNyHor3D5T{x&lpn&iuH2n@uXsLzxKV}UAP=jFcbc8eLz?aSbJ{MW#C`7C z7;+zn?Yi9shG2~8HQ*dxzTA8sZa(%A<*R(dGt#t=sZRCU7x3ptzKN}~V4B*N64TBy zlC8Jqzm?8&sSQ11$N1;=;()~KxLnK;H*`EynBPy=3bm+#fl_Nbci5Ri^ z@zMnz4cF3U9t3Fc2c8;8X{mqfz;b#CgNt$P5 zGMn-ZWK^!~x$&bbA*~+VA>mHYLVXV2{(YN>3v*u9E_h_$*VWb8lM&p6i^5 zxqFvOd2TJa*7K0T|Kn`&(q>|CQ)Z1N(1XEI6BcyaRvStpVg>JMF8LX-#|0G9NtlLCvp#UBI4nubkf`j4r@HldDsWNG5$<4HBc;%-mzy=?|t@sm7;`B%0bv_tAeaKU$amXrG#`Diho&|5) zCU1*36f-u#cgtaG!d_@7mK*X`H9IrX!-yho|9UFwgT;sI#OK!#-paMh>f2)dg^61& zvheEEW`p|c{F?p*JpzW9X~$zrY=gx>Lnf|)=g!`WgV$HYXsorIsn)QKr)X_k*X_mV z?Ac14ESE4z<-^yuUW-yH%>_RZC?bi*WuhNob5~IBFk~}!iEMlm;Leado52k>nru9C z#SxTigImgy1Lu0$sL5)Hmoc~lzd;T$e*WVR47sx}koEFQKbjk%MLCn|-U!yo;Bkhr z+5l2^I|x08RRJpYA~F#p*<{0kxdXTZohR@!oa9y;Xa8%2N& zEkt~TmIg$G4gHeY>_$ZUXwdk(*IA|i)VEfQb2U!@t*aK-8^6GVLMj<9jSsv7E=(9v$V^uEejnkN%$L7C$I4l~=vq9j*6>!RLe zfej!HtWi-*`)a;@H0vJ)4+i&-J`P%gtqiFVXU)jiEfa_DW{mOl1OA)w-j~N>^Xsv; zi!VrGNM4aQmVQ}B{sRR_C6=5L=eD#tYT>0;K$fXiGlts#yF3Z?pYi$skURZvyi>Z` zzUT=C0I){>fAnkr!aG*xj!yqczc!_zZM(sW?EO-!w}ke`8ALrsb22UTiReNDS-Z}~ z6B63LzSi(hLz#k-!Ous0l93{Ed7C-uikM#f3-0~xtxiP25xdL{4CthuzgWO_KQ@dm ztlaKL&!b1rK~^TL5{$>+tVl5oGAWbDkH_SYp<)RG14I^OUe4$I~rJ9_BpQ3%1++fc30sgfMm4P zw0#&muyi093VkD){axPI_^(UK53DU5FIsr#jCNa3m+y_08HPHOhng-EM7IZD- z)j*6OL?#K_KLnGw!LT*LJ)jn>br9K(J47Y_=r8}1T$MxK)rXNp6G_XY_fup8cGpmq z^yoe(Ik(6Q+p6nYL#kI{adYiGSgJs^B|F%}z(N~KjW`W;y+Fqt%DYmsKMt_as?`jp ztvugf1~a6YNG{I-wX~(NfT3u#;OSQlFNje#QhJWGsf|2NPE*R2gBhMB47( zSlTq8!u%-z5QlRYj{Gn}V)p_8*L}q8uT`Ab?G67)WyqY%B=B_Sg*UO@yV8n<@uf3e zu<9vHu`T{hSvrC{M%J%$gFZSPRlkm(Dl~``xANuyx!6(2?p*6i=qH3Eybf|;uj`~? z5{*7&zEIpDht?Hwr;~EA12_0Fl7jxeih6{2W*?8Q-ZniuXD1@*KlUncael2)jOdO= zxVVXqsM`MlOk6`!u>-XGrx85N5=YK1zZaM}5G>o|>u076keqycBXH4PE`e(i8wwV4 zch~J!z43x+?FIJ-`78WHcaCM!J_pdCl3ef;j_40id@;IB(x%ZfESp=$iip-u?WAn`T@wO{S@hMGC5jX$8_5r{0_d6pn&WX7);*Sefk>D!K59=>qKleqf6_x(!zMw>j%$ zRgRC=RQ7gb%7`=qH#JmevNp;f%AF}8UoVsgh-FiTMd$p!tA4a-QJmv~(s(T&hJ`q{ ze@5M37+M1|Pz(z{FD$&6Qz6$*pNY-&r_)H5uc~U&;V|5B1siqm>oHlMCkf2tsz0`# zvpTm3KUqLs4a1PmM!=FDW;GB|42_81U}nPPB07{dX(97qh8E}E65d7I^2X~L+FlQ~qo*q45<*I%XH~-pnEGLV- zk$vr9@wPg?ZrGqBZ*pkzaN=u3@QzLt@?Xs*wD{t|$D}({6-p=!l<(omy@aMQ3j~9y zayTWdU<{(j3!*=)vTNyIVPM}c4VH9gyvU~u%CqdeMOHK91g*=M8F0Q>2zAm-C9or3 z%n6loHee-pw!YIu^Rb+&^(k?;;aO0k!7)m7PgDK2lB+&%{9yird6g~7}uzwCu6t>7^6O1hwO2kgt5b0vA}R zqj`-IeS)8z9e~YP(eB>Oe3-Ddtd=Y_8=@YlxEiNP?`qkSZQ!dVGvTIEX>Akmvl~n{ zb#?P$1h{eIiC+2KVcChhwHLKBW;?dB&qLM%^cKlI zUH}Rmu|sfEWMdHMC3?C-W}zrP3_u5Fb{a`TZQxZ3UFPK(%r-i~zBlD)&NIUls+`!| zH#^6Fa6#VZ^rCw36+)Y1MBSskzF%i$wcg;a_?aVx#r0H*x}uAfvrM|+vC@KCI^qE~ z!iJf*Lpie!eVuE7#~U1Cj=AkhNzA>Z`0OBG-X;CQV=DV=J0v+W!_A#cqkvxOR;1Hz zwAiKJF0haoIvc3TH(hkmq4(cb-`lvFQLz4t!24U&aQ;~!|5ecR9UT5M;cc{%WNhZ& zP52L0wJbgXa)L2jLSc)JjtY(Pt|lJgcou0=$ascD*39D_*GVcqNE)``9m|luoRfOf zXuZ2$rlB6c<2%N~J1btpOC03qtTUD$fb|@C1CIkbI|0yeeO!iEPRT#;BmJnD!Moxa zrt9|tBp;W40;H%cVn`0J=8WoU$S48znC6MdYuUy$n4Hwc4;%N0+Pg4=;q%&i%VoYL z>mwD5{G26cd%;vJ#JTn!QNv(@`c)?n*YO<28LF^K=_3 zF=P1#X3Y!2j$@%jkF{0<4}QZg-Z8dhjn7uu%(84Qi5NdgAF3BvISG8A)YhWZ8G^~! z%1mku0QRV7mnj}G^9_3+k{O5AF#>x3UVMI11)SFn(ki&8D%1mNy97AMyY@srqd|qD zt7^18*a|wW2pQUVt07ubD93{X=;Y4~DoHF2hlqLfMaUT@BJYf7kP(@B^QiZ%LQ;J& zcUNr83U&SJux=Du;$do+Xrd~3{{fqDa+f6H?9@z=yhp#F^%D!=6kg(d!p`XStYWR( z3Ca7QL+AVs+?Yaf(CFTy<6)dF7`VXYuODWoaIWOdp znd1AZT60Fay4Q>0sUnOj4F#Yti9+c z#VW&d?TG2){^EPRcIKW*GgS%Az^H^`a_tbF1Q=x2$%Xu)tl%TVbt(BQj4spKoXVudod6T2l#k!8tqNqEvM|hZP?YZ%5~Hxn zBIXLm1i51Z3rvI-@PB{@fkA}Or3KlAtA>@;5Nw6AzdoMW}g+P>Jc zWe0`Ri@J%{6`NmPg0R*Hi-KoGknMBo+E#twm-%Ax7Uf1B{579cAh6~?Sf0H?`|>{R z;w3H{INks!wly z+f2nR+b&nYc}$ZPJR=8?6Ab-~V*~p&CgKG$j<2)Rni>nN>KP||>i&Tr^NKs%UHgqz z@*BNyQg`ihZeqiW@Dom8Nb*P5j=KcGkOU3IP(IH3Jqu5`PxhabEip`e!uemw75f)* z5&!eY{BMU+jQ@|Rl(bxj&0p}f`GgWg=6SbR?pF#e%w}HoMPL%k<-61a)nsrQcr(lsPvbW z!~JI*4cwiKbse1?%xz5n6EGF5G!c_YkG%DO(mDoK0ckNyT_LvCBu(W!r(#i(DR0bS z2PrgM@=SunTfQ@gcfkT8Fh7Ei`uxN(!aEYnL9IiJ&!S%bLglvFt=svWjNe)UL~tL1 z87aW0zBG(Dn$1)}fb9JnV){uWRp`|0R8d`jHgfUmi-32u5ZP1wm3V{yf@p>Ju^6(U zB}B?%(a#~diJ0+_WFO?1%u%(WKNU;`9l$q+8%?&M0_B(bO2Cq$(=}|?qT^-8Dw&oeVhI2p0@opUM`E369%#e-9^)O4-505i?^e%^0 zCg92wsoe^tXL%6q@Nv zq~LzT+gy!W!qfFO6`{M{nAuGn`9RodeoHv{L^e}@3)Pzf{3So80L5_8>fA}vJ#PL3 z_RmL2zvW>=2Mz#WM*skT|IcLlR~z%cCdL0jUAa>GYq-h^|6hiy?Yc&bC^3^f%>0+#(AM~~h*$LSLeT{oX2L%oF~}r|3H!|I9aS4a1@)r`TLULCBV5iMThnPs z)M9>WQ6i9{vL9w`46CP3K7HxzhMyek!&+JKaY7vHeUgZV87}R0c75tW+_v%IQAjg_ z%o6BmNW;@G%2TvA5jD`lMm1Vonjs@~FQu?2Z$ei|-n=e6*sR9xXGbnx*O` zvH&-G<8sRZdB*Bxk5_xh$&h3W_u{E~8iB|_#3iA{zYjN`)RXk&Y4_N55DlNPefzf& z(c1^zDGr?AxxwihHKiWs>|u?D0KO@In^0dCGQbK#8cV#A^~W-dAOH%?=j3Y=iqk;l zU^h%*|2!Y|a-nzOxQ`UB>EIQ+@pnE_BLQ|q?XeIOpAr43QP^)}C+*IozINu!_M?`g z><#AwiAjK0p^}oh`NLROu=)hp^uQlIUP_hk{89F?D!VHSGbSNrEkY%`C4Og^L`?8h zFEq!a9cuX6<%dlI?-yVXV+Mv?E!yCYMSN3OSsJMCWU2JQ5Yz_*q&(0LiztIA8%VCt zBXqOVBg!Ru`ZNdK#m%{vJvn;`@zwrwj*@XDJ$pAc=f;>_@RfS*%$h#U=fS9azrc2I zXVg%LOk!5aZ3c_3jaa`lELscivXrIRS?;#bQwzz-GXY5)f=^#;>m(Y_inxdnzVt|4R) z4^)p=Ge)?z5D}(59I$b(cd1KMc#6RJL80D1mOo(ph zURg!!TB;l!Rm0U_8E+=%mC%Q>BxNcByNu$1( zYFncLQ76UI60`^jbRWypN2}*H%+%$0bYBG_2psr`p1iG*-dJw^swjRoGR=OJbCq&} z#ZXn98kj!?v0>72++#kO$S`Z1S(SlKq8#&U-LpAiE^NR5j*`)Nlmd?f%bR)Cue7vK zf@40BTJ<5bOL1Zznvxx)^4q?A)$RZZ`bC}wAqKV zNZ;Ij)m!che;b9q({1Telak*I9e7P0-#ZVAM8FqGr7@QRUdI@=8s2jI8RQbAWh=H6KEBU*0Z9f3DS|8)N8x%d~;N)63*hXb9M5bAKN$>YS>uj$fInjIILk1NBsyW z+Nze5L1_mGway4tUt)16<4Tn)%10vPseVqp4d$A;Y8;>7DX>zRp%Eenv~uCx2^CNg zoNk|Izj9Zx0=Jr&JB8_6bJ8AUQD)} zs53(crv@AYUV`nG$;-$$?aAFSc-Z{;oq^shpv)bnxqQ z+epPO>7XfW=xFzzQSf2H z!|ff=WM;BlBfhD&$EwEvBgH9VoY^`U|;$ur&J zZoP9=nCCvn=G7*M2y2~z-^b(GsHW?9pi(&aD(ilR@=cUmaijSAmcnwe282%EeAKS4 zn0D27m%$2Vm^IW}Tj)p>J0;Bv#n;Bup1&AqyNrMFkin9T7 z)k!2nt76ch!5n`ptKy~JCDgJ{v<rZ%vZ9TX#iN^@NnuDLNElM*c-+#Rg{W>H~mm>Hi^UZ{rBikH(p3+2-*ifBS3G79 zuJY{h!y*L4o@bn)!k2|g4p5AH5CxGG2e%9Y)z!c2!^>bOb`e8>!gL06x8b6(p?TZV z;Og7h<~OR>PNG%5c&F#~%C)xgS72h%G}vEa|CLH#zm$ ze{Vsx>};k{77x2aCES$4M^TZ~k-jOrwPS1Hr~eeG&-*1t{KQx2iKO!vHCm;(fsK&^F*Y zYR`V59%mZ}7Jfx!C04SxjAEQok(A?dD`O7fw}p5a(j;~c@vGze2zB}KhfDSd`T#Be zQ)sK!M>|tVN;Zq03Hxw&>qXg=>X>uuvwg+a#i?1g#ap0Sx^eJ~)-QUxEMXhPTBEJi zO1f!rL8p!ZkzWe$9D8YPX8x~j)IrR`F9Ez3+4;en_B>Q@XLFylM3I!;!keZi5;}> zzv+DzFlEWD|2~f=sQ+Gu;As3GjgPieb|N$R{>pu7(Jvzf;ufg9*&&*yl2ZiE_!mkG zsT0}7k)2yREayL8a9Bt&t<7TRFS2E?*?YQg97CM7SJKS|BpF75c=JO}mp}?CZ^!L* z49(dTc6v1zT?8?XJllp?P}0yxjz*JHI3VgNt;@wB#2iSPgB!$B7)j$vQkYFlfZFI8 zsc6E72kd2XmJ{>^vn8%{OZX!+ku9&yLO)m!L|eLSrI6S=H}i+H+5*l%RL7*~g3F|= zac2O1qKvr%eDrD<5zCU~G-$`{3c*x~Crg^@H&MJM!;)^;rx^teccsmbq);{IU8dE$ zTE6l)r)J}P716@XK|?N8k1)nnzzwz2WBmipYBg^^yB#G5n;gQ3hvM8VGDZyh)L z@S7)6zxtUx1Y5NVVCrFeUiagy_Qw>HrUTy$K-S%~HV&&#NeCVH9q7I5+}oaG9OX70 zSHxD{^(>gygi|Jf^I5s+)n_h?q>@-RJ{k0jFgtyu4_ApgZ!1^0VJMGUoyzXsV!okm za$pb(YbON%#w~y%$Vjr0Y`T%>!u|S2EBvCvZS}3$fmDwQ>hu%P;8T;t01i$0(*Rlf zp>4os*Lp&t+v{wH(|ZE*gIpFUa|nDkpEN-eP0sth*F${eKbwl7=*G%OG;9j+M;ret z<(mu2$mrx}4v6D5geYfEZJLa8F);2H89rxIbmTrOli|V9|I=yY&UKG-EW&mVRQKr> zmNz?qN+?F0AZTN9|L&KfI>zI_(H|3QLwt6;002mkfPZgh{7=Q&zcAMSbZPj1&>dIa zmaZEj38$`Kzo;E&{uG6ZTuL2pYiG;57>COwsl_HA8aeXt;tR2o2od)JD$&uV{F)YMeejF62zn9d}Mv(SuRY#m-5U7Y+x$+~!SZQHYGPYKVf zh|9MWOp{qV=r*y5J3j@cHNkL8-ydFl(#m$zoVz5?Bp!G3mL1Puu)Gjj_}*Mg`zWbZl@Jhkd+KFc}GEY8?ueYuu>ZQ6Zc{ zGGw6653XVi*ht;r?(fK98zSzrr1YRn#YkH}z(Tg<1-k%MXs)zW{%TL+av1CG> z^1Cvmk_NA2Nk~~uz}G^#00Bbwr-HP#;D~eT+(KOyY3?*-x}umobZl*IO-$7-j;4u? zo+ym8;+BL5o%F;OpoG$g?tVjK(nYD3_lF?3T)Urg0_ZCngx6U|CsN-bk5 z`s`qB;6yj}yuBjy@kyxZMcJDwH_Rk~PFN+vQ*910#DQiVx}KJlZ8oCW}C z{E{}eHcxBLD1d))zugP~qrqZ{R#gq_wk-sFphz7c$chr)@G^hk1ME)gnQ7uMka}Zn zjo-*7A#us;8^y_h_Pl#AS?L@1QX#j*ASFzA?irBDL~xz@NvV5a2C?_@G6{gJV=rPX z2iT>q5MbbTyI0k1dEomA%$zs7!hjW|Mj(#!yySOp%5;tX%+}wv_P*)q%GO%l*y+Z| z!XLH(johBs)SpI2c8{)~=PBbBUpF`2_BOwfuGHMGok!5m8)MlX9xk4qAl2NjqZ!}t zM)%jRBh^x}wY9f1UoYF#SFKd`Ro{o>rSCqrwV~5bO*6NjT|Fn4bZuOmSeYGzyPnG| z@1M7owIMc9)3v^{Ot!+niNXb>bKytf*a_X7o*_Wyf~%ejp0E-&DG#_;S;?P8syR=% zfvrB{^av7a@cC??bUFY{J#o$E(E+0+fO>L3)B7_WIE$H~?6jM5NFMAikB=icy28&z zzEmMnUa4mt;3Wiy>_`j*38*~vA7RI6!}qhH8NVu(R6j2n|Cs3$K3YIa)FmA5KtVo& zBUw7%>#wQ5(NcaQv?VgckmCTvc_+Ur314xzu_j8$pRSK$?3 z_U!7P^y>?XU~>mK5z7EDZU1odva|t@LXcsK5#(A!w`k=o4lJW)-5{qM(1ESDMw94n zj>WzLMfu(8ud5qVe^IiyFl#!R-8=AcnNMl@M}&mrJoHf5YH6>ImnZW(?DNIl-S?Qj zOM@@B#J7=-iRg$JNke37Z6_mGyj0*2^7-?(%JXilE8}A|$x<*hw2d9etTL79$4TlV zCtapzLE5$5JQzBb^Wx)??y$&5QJ=^5-hyeEAlxoJNBJ$tfUh+>EPHsf-Lp(1-aTea z+jEJn-D3l7TxMmNpO&s(+qJC_%_|x1kQ_qmb7z5fHs?h65!UHr$4gYh>~K}u0sW~P zKm`Dq(3T?B3g=q}#fAO9%ct1Zpj4%jid2a+9A6N#jPiq=1)YWZtbci2*l-GlovW(5*_4Pr^ zx~G3q!7!y0A>z?r$*XaN@y2DBCtg!Z>jcw-BSGgJ6c2E0ILNQUo0SZvg3l^Ew(V;>InL#DMd#! z{IZwVlsN}llae?V4iC9WLI9+#V+(<^JCHb3BUSd^KlA1szPlQeu0+E7Yewd>GN8Dq z+-T+w%#yejD9p3F>SZ%X$Y**m(UTRXP{G9(&fdVG4pe&5buJWcDWt?|`$TC+o;nzD zM5%n|rhb^Cv~8tK`){;J^4tM{AhsZ=i3rBTr2IDvAY7U%T1{rm$C$i;7Rlt0kw?*L zdE^!2ga<_74f{}TrdOB&w5k|K1YVirXVjntRyx0TMMi*bW!SuP*#<+7@K=GDLZyV% zlV|q$Ii@Zs@W4k$`}zf)%USZi^YSWt59_o|nf#NeF|Zn#MR0)XIqALneu19nEL4RE z-1jhvNi({oy^~3gWUM`#L{Lo8PFv}$w4vi0dz)Qw`=1a`0h1$F-_rG+aZRNsx0kE< z3HM%PuoR}};YK+J*AsvX8XS*QARRdAy-XE2bg=gwNXBst3o_x0%b^_YMT0cozS27P ztYfJm{$Z~m?zb}A$~bK44RYaqTku)1C z=CrGk8p8RNY3}fA_uSK{+g)^Edig0lGY!a91#JTL=!W}EK5<+NNfMjOK`lsHh!4VW z!$99nc^fpEv^_fz?EEegJbLlw-)kzL84phMf4pf{Yz%7iBJ_~|g8456}X3N|claF;T9A}&a{k;;E9>_#o zO*33Oa9fbh&kh(gYm%xcXA|%>w)g@CJ~uRI`myql8gyc%a%C~&=P2k1Cc9ukH?RT( z;W6+zlwE$fr`!qu7B6+MGjdYpQ1Yy4vpzX|Oo2m0PYPlW;h9L-9(b>*XD3hV8{}hN znx+v?x=KK(Xq1)(@QWHZq{lAQutxpz4Gw(uKszzJ83o`h9J|ld6_r@KqAj%aaUUa+ zF>}Tyo#5n)(g|k0(X)-xp~fo0Bh=7PMcmy`|Z*a1F>8-v!;VK zZCPR0i^Z^2dll7q3k%ZECkL`8?8uR__EyCGTZ39e&VsZuGKiExIC#vw0cDNKiuaiF zK0!ek4+D)w(X$p+xWY_x5x<3CuX>3x-gOB-wFqa+MIv3A*Qvn{JWbl{N6tJTU|vCf zVtZDYPaT5#57gs$_Lj49SQQN67XA=x>-u2(8UH*eulTjUkJ8bP4GC6g2R9uas#|Pc z{6=OpPX@Fl>fP%)edK+Rqv;&oEK4@^INfRfdvYZNR7|M~z`DK9KZ zvMDhxW|0hon>#-mS6|<}(!14ckwrvv<=9neFQv}IoSnqvkAKzHO~$3Z!IQw4tLL6Q zo0$3NS6@b#Kaag?OC=J56C;j3@=gtrDK}C(Xb^+cwtJvg4+^&?>CEeJ#Gr!`6JRQ* zX!C@24udDX)Tw;PI}FihPGk%1F1PJrId$BtV7NIW>(EGer!C^-yhNBG&10VurMP$m zP2*Bl+LBRL?Rtb~BilfsqQpwXxpHS|_?2;84(Vdk$Vn9;qePvO7dwmo3iH=V#H)1n zB)8l?fcxk4*kvWDE?)FD_ApA~=lKM}841@?f(f)am}TgNAr-l_Y;IYs6XvP}M>%US zP=1JqLk3Q4jBfO_vL&_TElj$S>fhyLx1Nwdaw?84CtEqnW?&tWGNh?B6_dw-m~_lR zS~B8oa7tzHmigM6^t>98P@HU~70{^?X;!H+L6chTFqZ*V;7cb~bc55hJf~|G)JA`v zp{XEb?YyjB8m(|~SkL%z0AD4oRH$NaQBn?Ds{MK5AKo#^F+o*3y~#?ii2eiHHnjW; z7aUABG}A8*;UeX6M(BBu_@s6kTyw z_J5uTqFy!(6HWnxwtKnv*f<&u2H6N>5DJb~QEWI!VCZ_Y$=VPco66)r)Sjc)C^>e1 z*Z3KyUy}+?u#F~&W;?WQY9wYYDa#hHLl4(jNZW`x8dYjT`w1Z%NyLCkJc0>R*TGU+ z4M(zw6~lu*hhyrgOuaYARpB7IRb!nwNMziAk)bxr)*1XpPl;eYU$&S~_rvj2l`75| z@pDe)b1_i0Bulo@&D|s~hlCNiCdjfDwIhq)B7W{8F<4jSOwXMc2b& zS&O}w!Yy|`t~Jdgj#OS)kKCK|pzq^RMUF>L{I+Wq6W;G2yp^TIy?#R_ z|6x~7MCK@N5vC(X8UoeN8{)1-Ip7q5H>9C8FJHh5wh98g1KFKtc_;pLEtfRpkjrXH zC)g7C0&S<;#h*92Qn0RC$Yz?IV#Exqc=@=rQW)4AfaQ}1UcTY;+eAMkmybRff@@W6 zc2`x-CAtFQcPyJOQOIDf-mVJoauaUnT2I@mg-ZUMZ(Dj2VN{tX@cPZRQ8b8Np&b;=}-?j<6~CdX&<^P1s| z!X*ynDb*J%<&lcd4>uQ8*q*^R07P+2vyULDEF#7^Bq2FFJ`99PUA-kps&&<^)^rc% zz7?3NIq!4E1XsZFhVFpebSL$ACvrW*C>@r;=AH-^6*_xeJ|{XI4hSd0uUiZ(CjAk1Sc3k29bSi57AQLR4d{GKRL za~2Fvoy%tK!Q35%(oSHik4S54gqwFJ^t+B{dxDFspJIOzHYpsi&a@S69oIY+xe zhgsv+rse{wIFNx`MOy~DL8!h>B&1FjKJZB}a z1EBvvi!6sI<7_>Muxgli^vBk%)xNReC{Q8U_@Tg@?MAisC3drJ9vS>+)$HoAFMCrq z;U+124n#D3qf$8)4kpuvb-f8sqXmLsY#p}=Pp)1=D;33ak9yvYB72+9Hhbr|(-pmc zdi-X)R#=9I>dae*90}!<{ZR|;o`#M~5>TNntm>)WZ&F`$%U?*j3)A-q(zS$#?mf90 z6+G&rW*DdD^KOy99}4WG=&GrY?rk4Xx=&npsMJb%I>NGZdpxn;`HDd&aHY=y8fgsv z#yzG%u%T|f5yvPr+*y1C69F8zt5{rLxX`PYc6?l2B=0hAZ>#IzsGz;7Thci85_@NH z6mQf{3{Q|TgiMuJ0dpy6uiAJb3w_yUBoyae%*xLKu#t5rpJ9aLWslixbP{u8exYrI z9R&#m!{3cxN9znU;5eWJ!P;LvxeL`8$bqq}Vq+p4HNOvCmz^Q!ewl=#MmXfDkY43abD0tF|PJ^B`#L+Qe%y|KQ7(9 zrea9D0gR&zEKnoHZ36kmplJd<^{oWG<${miqe17p;>!O0_78UBBKp3>R8R z$hEkcX|j0d5$+n)!VFIx=HP4A{fd?204;m}Dw-D5S|x-di(*+;%lyncH;T~t3yApT z^LT_KI}IbQy z|77E-+${0UJ{X7F+BH8;8mr(;j)JKa79|u-aTYu#8Xe)RPuSU?m1yB;eJr|M9~t1` z`o5(>K@(5fN+*45YLIcPHd>*YPpC_t79tF203e$^-PlEtTWoUkqQpQ+zzA>H8}_gq z)}yOp!tFs);YjTOJVe18G+MgvO0XB0*S5=wtC@Ans=c=pauYo14dmz;#!bMJ!q$pb zku{#%9Gu3_Jj8W3&1WN4I$1?Jx1ln3slCd^hQjnt)4+K(Gm?+R*gn6(5x#tCj5Mu( zT8{w2twD_h|0Q%d-^m9Ly}wze@D@}xQq&{{1xvQFJu&@jKGTP1ks8TO_fPpiH(C*?t*oasVUSb5Gdx zOs3axR2U4DyjMY?2~X~?dl|;rQJNX{Q(m@^-jFe45nI5fs@_OTOD4oDP!9}U&+m&> zxN>UvMWQhi_)={94HB_a40zd$McaRjLwSq zSxXp|e9T@QH>q=t?8iqPKn`h}hpzqfTTVAT6LY&#oCp<60v=1uTMH)D^Kv;1^tf5I z4Kxns&11a75t{&(Eh^a!-=`&lX6O*{`8GRjE~HGUKePsiH+-*mC%&qhbDO{Fo45y( z&Fu=STQ4!Z(F;-k5<&~IpgEnPKdXa=O7`X zP*=)467vyQJG}3k@mEK#Zt$!+g?eb|AP#J>P89@8ejm4Q&^?5`fvPQr3wTt{MPup< z692_W*$!)eK7QCn6Qx&~5)G79h7YX9StA@~g%pcx)hxfD*9}fEznaoD4HZkAkaS^~ zAgj5ovux;JMp3D;0Ph$eBP?Q6H_>7*mnrH&QRq4-C`7AzdaS>;vV(gO2A9GFQ7;i8 z+2L>-oz08b-D&V0XD|eCoR@qW4;?t5k7cw+1wLg37H{n|obh4V|DgPw<{rVYfgE#X z&rFCOD|aK^5me2@K0KDdH@qV+FFBxMjte~8uP+w;3$BdfnLu^2**w`c*TkK{Rm4O% zf0ihL)Placp!YUvJy11PJY|<(ZQjj@eRxOgc1luSk@rxzM^UH;3 z6|o&b$WX;2HOJxM_0d6&bWh_7X>UKSZDJKU$vw<5_tK1d8EHN~-BN<3eyChi8U z5o1WiW{&-99UfvaY=Hv}*FYul1G{t*A*)uk&e7k7qJm_G(y+pjTagT3JWSNJlp(_Bod4BDp#1sFvJ%r*J6&nkcs0D?XAa)prU_ zb%Fa&ZD@D&UJeT$ZMIv~QrT2TLLE9n=&kz!(RMNcsaR&QX@u5feG3iF0b)#{8$25R z(YW~Pmrtk~_bN3F`!8f*-An|HYMOgZfBD7tuJ>PRv1nWSl zoLQv_@_zc`827b_@gdKjWg7IN)O62U+k)w=~NZNvi z3Z59i9V9vWJ9T4eT^a7WhM$LMr`t_E$wqjy%;!sq#GXXc8@)M%YnuC=Fc#HDu{ntH zQ`vZRt1rItoLo!bji?ffOb_k&F>Ri;wu6{mrxCNjUTMd)BY=;#JltQ`#2*en4v$;i z7Xoqp>A}mHUCLaZ-!nY1E>?qGPp_pCRjO9r3D^qHs8I?pvL$jvz_f@{y=j)5j}`Au z?pSV0q!m+cQ9XdXnzeneo@O*kO_6+-`futSo~JK`Rj(c7h+0-0cHpfW2$1rv_1AnV zAF^W|maR$8XO2;&hA!)0eH~jTbUSd%yPw@$=GWv#@LB$?Yu$t}P6Ly6HfwqisICk~ zxYFzprJcZc38*r+Lk-@ydSiWB;pxoGDp@VY@xdVTTdkZ~a4~HF;qU|Rj~@9#E9HPg zvV4I@M)3QyeY41l4s7{3o)Qq{db>W{A;pkwFshD}%6Q2d$VpvaC0y6ue;ICu#=i%A zsLZ6y0P2XhbS40uM^CzSiwKqT{lKn`o9@=|8V9`8w=D0)JRpprC0cL1E}}x8Em;|l z{BE&#h6_}XKfra$>3PHp4oS}ztfTcCF@B5o&*A2!xaQvgoNZ($E}i27+Uc6{3?E@<#}Fq34O%4N1lsB)GN9OL&doJR&Ai3a0joGFyJwOc^tEy9C@laE zAlp%$+X*P%+tSO+{7l84x;^DdnSI?u$~(L%2=B%toZg*s8iZUeG=>V=zm%JiiRCHN zLqPo!{w2p?80WzNnh+nd7!04%;ND>e&p*0UQ~%fN!WF+?E`yWuN53WU1tp7LaT|w; zGnJer{-SgqNV#x@)cKbW{fE>ma?i}ifdtHCJ@}# zF%nMOw$tj2p)opHiRA?b8I=cx;pH)l)>chfbn*j!Y zEKCU78|tR=Gx;;6qZc!)y@)n1cDlxL*I{XHmn!MG36Ucg>?a~RK82>r#1i+-VxFsm z!s{OtIQh3(xHO9bJ+~kuLum|nA_3pYjI{4E?;qx^PooC3ivf5!*BMKPz#5$pjC85G zYdt;PqY!^#J%_3?Li%D#nTBQplD%8E#Tu7`!rPEtEXWLaX&ow9;8W>$JpXK+W%Tl% zm!;FGOhw7wucbeir$C>GtXQd#g)r<1C8m?&h{W{2BG}H~FejNf6b}Q2b6g>CAbBeg zPD!x?uDAV2f^xrH@_-p(W#eM>r zyJz^~^a`j-LVF7lphr3=EFo(L#O3Wm)Vh4rgrpE}UZ{ld_VwkBG^t)T@;Bt_JxZ%s zj1Gidsu>3ZrY{!48jbJqqXJ=wVmfGkV3q_~Q{C1HH${;vJnLUYR4Pxvsk zfCI_x{N^RR7ws{7-I~>>*2+m1Y-SQY0Pb}`j)Hw7WjBh8arP^u2&3n`1_AQ-tETo2 z3F(fGyrXEVJqJUBG-*l?W2}uFx0fN&L&8rjLpgL)Ua%0Z8S6bzO!_sVsupr2MHCOd zNZC1H-xYcm3CgE}xBPKC{TOycFM&|y=s!lJ KIT9YYK$giZ6&Z#Gw;~_PO{a#W zfn+%A&&voYgGG@gBIg$OsoJ(JlLQav74_;H)%+EFVWHXau_;?O_vww=h~scnGEn%? zY>@b#HjfPGYuNeQl|m3@vt6aVnEJ`SxccgURCbn8ab$}Y1_Cq$hv1gr8eD_ByAvch zjXMMg?(W*SLlfLxf&~o(mteu&3GkTA+)*t5Rcb>Jdkc2K9`{5Gy|Kx(xZqz%D9It?K40QW6x2ja-Us_3K2h;zF^w_)3jW-s{lXdEjdDs5 z*=DW6KGN_YN{i&>$SfxO)gUX)jKoz2JT$Dy{%(hf3NN}CX9i9hEIH$XeK#A2Je3P$ zZQwUVWNv#5dp2hWja3LtIJwq>2u5M&+PbEfOcP+WFyp)>%AKn5O$wC*n>Qc39Od;w zW^=0Y$JX9LxAECz3e$`a*(m|$cq3bY)R%aUt{FvJ-zJ3J`p&{Xaw-GL>PoYQt9#EmBN!dwQ8BeBs<@$tdi)FHM6g- zJubQ%QjYnJM6=6?yaxg4M`lLKz1`3!+;nvdTN-qu`-1HqE({yDwGs{J56Nk$1KjR8 zBQiZPn^eWfKYp+!wGa{jkl$G-x3~5!q&30d9H%}ss_d5?k5BR{x*fQXNjJiwfrj)x zY3jW_&?6$W9My2?6XCFZ@TOynDMB`xTaL$10RoPVT3wnCGqX&fb#Ha#iE4OvUJ_^U zW)5qW{y1S;_c*&5-dZ`?Ix|8D#6rN}r)}i#h#$sIvKU)GCfH{XByblyrqDVm9>5-u zPKiYHdNvQlSq!TO0CK~^1T%?WcVGsy}SNvMPc&S!P7vg%361B206I8J@B1Jir} z7~QI5^eg^?@dN3?Nz=^T)%M-l>i4t4B!2hp(QSiN8_wF!{N*w)?6W@R&4K&2%{LZI zRIrC$doFcV%aC&4&yHBRu&W7-;f!FIvW$=kh5&DG(poOS{UMft14pbMt0JfsE~0=4 z@w~%0TW9O5CqMQ(H@4r8(9`7p0E!7lt$oh0#(aTlnX)D`KDs=Dvt>oP^HJbtMz0oA zj4Cs%=p*A@ueR(A<&cMc7;^LQ?)>i2w-*zkjpc`9A$J{2Riy(t=Anxg1vuD%?aJc zRl09Sys69KP_~br98%c*Jj)vi9Lb-%Ti@K0TspG=m(~P&YY&&`!QK1t#mSA&bU56M zk9TbVwuuoeQ=M0t9Um3nspwRt5p6@qyLQ+VcwT1Xlv+hfuhghUU3;Tcb*>>B&a+`w4fJ5{9%*CuDYvAyjyH6MWp%k5iSTAW-MI`28J=f-;!kYgPd^buQN`?Mu$F&gi;?l_lTk-V=-0!WMm6nclWHf(QFYe?#yNHWCkd zLuhNe2#Z&jhn4x7Q|T6~K+I?QOa5^-g}QcafaOQh zbqO4Df^2$Kp?PsN{1)EpP3SvWj%_5B`qdIhfZYd{eV^JF_fE-^=mPkFa2*o^nX~ngzYdpm|=iYgjSE}Q9**Yc9EMYo!z4!&4R<1V=Ze zxaieNa0Z-3qL4nYiE*6l=mxadbwRsktEJ-%%Wr?FC0Pliq#wvC>bR|-ZV!v}QlyRk z)Ur$EOCPwzLWqf*7(@^*0SV(iF93g%${o!o`8sc(tao5)ucXaqSDoftX`&{8JG~qb z%6i0|Be0_fVHOBN*TUL}C_-?CK3bWdef=6E~C9jttIM5vX}@f*e?HO+UEr>i4e~M0&=l7b}~LqHeSB@YVF)HP=73Y zWp`aZVgX0bTFiG}zuLT>;GeWC6AFerCAYE(=hFZL7k2OolditshG3Vf{&a;cyrAlk zSvgBH8p(S5(hpf1=2g&CzOO>d%R*L&oj7bY6F0;k+MNtC$?qAccmgHofR!GN8|;^K z@B-Q?av)BD#*s82rU5@Myk-!1tV5BP1J=K2ysEOp1B!R>2R=;&lLi|~kjKzl zSG|c_Pz!mnVV*VpnZUP!Re$RjWNELjld_G9jm6oSzA`zdmJmtT^IT-a&=8(fy+d@m z?fI&4nf#z;3}*fEO{Tce6yJhvxTeX!;Kq;X8v+45OnOebhi~>)EEZ>}8W0_Z(h4Ns zdkSR#(l5$EV(in0SSr8-5}g{HL`=3cpV~Tu(k@i9-wu93{&ZW2Q!4az-wv(CC>JQsR4zN6kEU#V$=QpGJBY243jzS;XK#`awV zo^pcIipD^}`S?p^{2ZW~d9%)gUQ%$7v>_m55`{3a1ZYT$#~h_N)5$F^XHl)X=Qr4C zaw~3@{q9xcHTO~AQUsza!_xIcW+)ryM&&wX0%(0-ck*i}2j&qb`n8RhhdI^}06AG{ zu;yM-ie=Tu30n7_1PyPK@SUF`Rnt$Dt;bkVy$33rZBCA&f<^>rwzPc?^Rfe@Y^nIK zMlDGkAag*H0WZFku;-jZ*SFm|pmv>}-t_vt0%(6iLC-F;A*r4aCBvz3nI(vuvvdier;$zhlZ9Y1M!fiVRY{U+3q zWPRhTgCs76zLqb0RVo*Mu@R(oi;^2z(lr4TQZ7f1Fo=KG#-PSdT-&bboD`YnelHcXoC`cF`r{q_e0fiD16P)o*#<6BtpWBz-v0;4dUwO_Jw3&Q-rJ}^mQ zBHq3zn~tKb$E)4ES2~iP7ib-wnA)!K2%~QNPRH*zsFc;WEi7P? zHXKVH64k#MrIpBH_@*5_<`rj;JICjrs4U=f(lsGF?q~!BKpECPQkOwlNFyBAJuSz6Q62LA0wrz+40<leo0+cT-rjAUmdjn}z^L8gs1FP{E7tJs}|5!rz{0c#j z6xlv}IXc?vh<_YGHFV7Gb%xBbMFOH!kyMH*v0iO7psyPeq52-virQ3;fqW|O-d+G> zlElA)1>Ga(Mq1AZwk^;^Dpqptg(w=gjg9^~=M*_2Yid*VT|U9Fkp~d5kv2(x(#sj! z^<{tSbtoK7CvGlvFOCZQ^M0UD937{$6qiD(8EiaX%@2W1^HabkC8xRY7duy{qu_f3 zXUb|FhU09}IF3z$*?!CPySz7sVd6JVSP~ z=i@~;{0%t$wAnL9D3Shnjlgj*wx;xTUuAf@w_ONXO<8~p*Dj4*7EXZ%tju8?KkEJqhFsRv zy6}i79g(`wbDqy$+hCIUpeAkD#K1&P9W!ga(p~azF>?uKNlp1zP9o!>y*xF>a4xEm zr`jpOQOTBW?83%|ZSp`mb6IQJ%>b(o;kV$p5NU3c!7>TwGwK1|)JpN#K$>}K>bDL( zPPEh0s7}D?I2E-D7f6Jamr=~mF*kIaO1Z00!%8-3y$uu<-mtZsm(oCf*cObO+hyrhJ+1z81@ySMifb-dLEOG+1}*@)kp@+*iKhrHY`no8cs zOO4x(^`RWu1}~Sjf3$aYMuh9Z4AAgHPj`_$4@PILi^Jbi{}3^2qv;Y*MOhF)Q$9p6 zhf)6M2(l{WF+Wve+hWvKR0q4$Q^*%s(nJ%12On+lx^H!{fhI5-f2#qw6rd8xl~-{eWKJHxu%)!UfF2JGknNZ*zcImFF{w6 zf`eRHt13)2#Tat?RYIyg0OFY>C`Tgv5g|y}^$P41QF7DrlaF-CIb7qF5;+Wf z-rK2Wz*+BH3am!&i9|NJBxqcmJ~{%jx`&GNSGb-;A}uun1ecCNq%2_r9vDhbxQHsh zv}E{BzKob*{Iny(gg(i@kHf$7(bN&IexSk4OzAV9t_RL(0{(_KHH_YC1rAUFuk7t4 z`@XVEInw3FG(-Dn%Q#k;y-k<2g{dGfI^$mQzt6B>I+6fDN!HVB8QfiB9q zRak5ml+eMu;+d%4*L|tcM{i%Lu+c$t%avjEFt4(;nusm<$qA-R3kD_&z{1XsqS#4_kO3V3O1Ox~ll*@t-h&1m1-W&U|>R zMAgQaZGzU+uz+Jj6(*Y*P1ucb0$kdt0pWQ%4Rr2C(t7OcxJX_3OKUf{Sw8Nyj+42~ zmUzfDZtLLvq75{YV}U?Oyg9Ou?vl}m#~)Ch)%EFhbGG4HQ+XJ%7v@yEN6M@B@RPA_ zQ65B@n4&I`+irtEAbE6|zDwZ4O4|DBV=Tnk&6a>$j9VRE8!4`oNQQmFmUuNZ9gUGI zaa>Ij5>SHF@J#eglePdsYTa?HPP-~@Yebvm=1l&xnWUX_8R^M#uK5s8P2=Jc{k0W& z^K9~v9#YTn8P4ACJH%^Td2|V?$9LV`r{lf7436iWAE-Z6roIXDUB&H}t8XBSEs%UJ zy(hl4TPrtd0oF;o9p-sgU&_1tPfM>YS}_*5NOzqh5r(K{ zt-2p#T2P5L2y!n%suq%#m?ails1v_;QqWrWnFTLe;hteR8B4b7tOf6mc#F2CU*%_# z+0b+_VP+BJu5(}(>_I@*XF)*Yz&ta@3>d30!-nsO%*GLe7Hm!j()80vXcBE%5Ttx> zqd12jfa!1<1-;ZG_^~2LYejMW-0jhdthoQXr9(oKU)mbnMXUosh*!QMYl@QV?s`;e zSrxg?I`gNQtoLxh(inO)iJeleh{8a!dQx;m#6|CMF*Rea=)g;smL`w+Knr<2rTZ^Q z?H78c;*_~V+3(%$-XF<2+qv=0@n3x8wX)s9@>%wta4igiI;9X+KwjVTIJ+?L|K7pq z);J(AZgo&XRe0AH@maW<5_VDDyJ-33;1CPxQ~v(`kcCWJQ<>9By&7PC!lbp+3G_^U z2?pr6C;SX8ce5iFeFzC-`ujeoR|7f?6(Dtl&GHQ(4B7G$m{OMbuu~pN*G!3Fbiw5U zi!TGI%Q=RE#7{7wPBRO=Ia~J;!cM9S0 zP6p&5)pjlTL`}wmFPG3tn%L#yJn`d4k7vs-PO4zu6w>94pF&~kypSPRK<1GJR^ZoW z4_wr@FnoXGEZWsIv@7@A1iQ+n;k-DxzcE|YuQ&n}deF8Ju>W8@2G7Ie$B?X#o9;dJ z@VJ4gm6fW**B&rnyD0aT1uS&#;Kg|4ko>C2#0V20uAy<;`leol)bTZs1(xCX$SHM` zI0^=0NcVE^y22_=Q53fF4@zs1EcZac^s9%^6tx^#XQoGI4#$jrVuJ9I*VRM0YvbGV z(l-8Aiq01n?%~A9euQZ|{z9Kh6rIT6a=C_H5TV4$D5ASax`w$^EH{1Jus24gL!#4J zuZK!NzSH%D;aLcCwF)!AVc!$r-;+};a09)%)W?aW=)!j4_O&L-I$pAwYk41OqHXYQ zhIFV$@Y733RY7Fm}^z%9PX>JsIKby?l@QV^DX?|TJBQ|R<={q1njMj%?;XZ_RWp1YeJ&lygpNjukrqK zxw^HVNAX3EzJT6AX8YWOJ#?;{pB-sEk@gG49P&bgrD@hZ!l*0HGIt%1T!Z8p_wQD2 z+Og!dg4Betiq|QYL$DtVdTnXZdKT<8&T|sWv30DlI+ zxs(wb^;=pFIXSfAoUR^oYcp}NCkap%1~hx<(jc0wpZlR}G%&P=jEu^UR-cY<%%8^X zxjNcGHkcA411Xa>XHH`~uZ+<7W8u%uW)qFVCE61%Fbceh&)={c;h^C7(fM=o#oytb zWioZhDVQX_M*+=lKRcvvCKhJV%RsDhezrmxFaCNvBPIuO=Jm_)#Z|xLvUqU=bQ1fc z?EzY@(51CmCB2tkSOr>+g}p;H6xc!IqvCH`lfQzXH&Nb`SxZDPs{NoGLK^|`v@L}- zbWV^c?H8@;qY6FldcLzq;K{zV$HT2L>AQO=6~kDf8^e^EO`7See3Xr<<|^k|IromB zO`ZS$=0r<@TmY? z$+}e_NA0Cib$?FxVIU7i))4Tx$vaRqu(_%)0^}(m^7w13Rdtil5Qe!j4}CwLp3npw zJB1LGh!Ky>Cs*H+0I96pDnGv>mZAX`lUvcOBY8+Ov#r1-iIENz@AB7dw>gQmN87N&rcV@tO7vA*nG6Vq&VaFJrsPLC=;=M zz5U_G?%rxzE(^ID0Vv~bKbel1H}zn14uyl6fz#(Wm+{oYcS;B2xiUUkN=e^s@-1Tr z9L*WQt!A!}D~$L2Pwh#RZ*u*=Jm#%Tph7^P{cHF7zmrz}udVCqi(&I(s9w{0OmB&S z*xGsIVQpv|uKE8s1l>iG&z>wcP6r9)P`a~bwqFk; z-oPd<)4x)ONlHV!Z)w%MUnd*r?G-%{!vqUdU{)*W50K;O+jf&<{2&GP!?3`zriw{+ zPv>dT$Ubl^XiTo7CeukEq~!5S&f-!GFyN*1LdE-Y$aF^`Tz%`kWQZj*1cT#8R#MHi zvpe1c+N+SHK1IkEPnHBul~`Jp+SNkoa!6^Lkr$%dl5@hCbkTW(BhbOUs$&wq`CzIF z+FBNmTdj@DW8-S~>(|4oiwcd$Y>lO|v1Hj3+BSCpTDz!D8SwHu>+hOSX9|Jpwj9~c zfCa&NG~lpE510+wTh+ZqKWy*Oc1R;fECnM2^z0c+hJlomTdmJv;(TZx7ly&`$*o(e zD{(!hD56h8lxiJR9i~)QdU$e;$&2*!N`Zt)PPha&Q$Dc~r={lww=)z@c1QjjGr+gW zKB+b|2?&;W*FQ5-@K3)qa}V0TNpG8egScb2jP6~l235&jO7$dq=eep4o7H06^^j#i?mnwyK zMeX4d_cU_bAS@2luxN}eUTeTc;`V~i|N><*^tMw+c&%=j=x85KQyJG7!I`6XYwMcWnNXReQTIn{mWAHQVp7p9mX6}pHc zqD5}qi;-Pvy>}1|d{#&*Znqs-i^Li#y;~kU4~Nd*uxeH=Og#K!ul&ef;dX0SWVD4M z;z0oh{@zenoN8KUFnt2C&yxLuoxd&=K4#qVATed_Q_WWqdw_pv&}pZG+kH$rPxL^R zo?g`Y!968+Qf(8kiz*wFD& z+~NPBE*$P5HTvjIGXMtxf&4^W_|eYkXYCDbtZd&o{y#MJGFmL7Q6M0i86hB0pJ-}5 zW?uc&w6(W!v@x`?{6AIyzMuZ3`q%)Ejm!Gz0Qpn(Nxs8BweH`mBK%d>KT3Tk_~Bow ze`a(2^BDL~cQuVwoVQ~_K#bTxKoCEveD_nqUkz0LV|n^drT!1(X-eXsWA@L1HqY@_ zTl`t-Kbqz5GZTLg^fc`6SCA*y6VT7vJc<7MJ;2j6v|jVYMuKl14iu$<7c1z zw~_AeQJ(q&{z{3|`ESZom%!g6JhfK)l>pZNZ^9F!wci6gb-wx)U}N|f;Aa#4>Vx%r n%1@t4epRN=`AhlVGu}VmBw=9xm<;~-Vle!83zshVy{ZZ%05BNuz_4O_7EM4^V=^Q-&57IZ!|A$0Opy0_OMWA44Y;9<63iU7G ze{BHqKj&JRz>4by1pt7<`hS3p{tR^rH^CbqnG2fIrM`svp_Ea6?A1sPwsL2=eu4E*OLd%w-%q zQ2Z-s7k)9pTM%g>NL6lW4@_2o=-#Y`!7yzFek}tL-_1CH?IGa&u;u}KoZBRii_n6u ztD9gg0Gh7k2Ak}{t}U?KN< z8>nxgMAR;J`Pm4@n!teZ4g$m+^FCf$-vElC7rzrMaoR(ff@#&73t@P?g&Ylqp4p9v zQ;L8_Mz8Zz&+DY6Dz_{m6L>??HevR{3xXI3JCE)3?RupGfxN+mK4yz6I0#rb;9ter z7$-!T36_Hy$*3Jx^GLt(gZ5Nl8mlTL%DRn3UnsFgksilJ#0A~ROXP_^H*IDNZxCZ6 zo1;;;649O>x&>FrSo*rT5qa@* zeOPiV!<@VrlPCsP_sif?D5H?X>j>WDoNGu#qa#Vw{-671_X?ZxwKSh)n3)5`k!u5> zEfBG!WH>KJa5&jtnmd5l0x$Xu$}^Y2rY0$`9Vh{CoYdO$il@)L#oXIsLU3%i-aEJO zj{JMOj?u&F>FB{&m840<%UYH2*XB}o*Z*zL>C^Ov^i7fnf8mYB>r(|=T6-M zv6x5buAa!Kh*U)vQr!69o1P{CeI(Q*6k{u3+AC>l5Z0oy=qqcTfz_< zWhA%KSJ41ss;t6yh(q?TlgTCrGjwnQh5|F@=APJPeC%{E?cvV|ZODYmS#PGToUu)) z#V0Xj_Q`DAE2RHE*`J?^ZHc*~lq)2iL>pM^VDi@ZAli%>@)rBFzI^>NG;Gv~yA8%M z6ZG)o);|_n+wfTvqE?9E5zH-A*`t|!tj@7eZuJcpeX^FG)5M6PtV}YE_PcRGC!~{K zWP32k903QLBg$zFm~9He#| zm#cFsse&tz0bc-W*(@WtkR7_NtEn6C6w;+=Qv9oI+hYRefdl|krj&Vk|52enh9# z_&zqWS9sSY{I0cm?BM*{<7yfjh12 zLVb2x>>h=mA5$OKFY_nP99F#++64%P-6q_BkU`^OV+SP)m79Kp+{wIv@0L84*^J^<>oZM?YMfv0rqBiO^e`aqoRRJgnp74N@`F(Xr2GdXHisFlvC<8TTEB zk@pyo1@}F+;6P8Ew4xsYGRrtRRY^Y^PE^M{N+1Dr(B`5=~%{p>u|$ zyvTbY_L**07QfEyO4!Fu0CA&e$t5u*N@r+|TgWI9;EVyx&wfe0W=E(~xA*LX4B)F1 z+UV(uOiI)9?D|H-AHVG<4D0|Vw9^0p9YdT4&O|+hZ#?VN*B#jAun%mqGP|c2rl}qJ zO;J1pIYwl!1!i2!HLcE!AJ77M$=MT|U1CZ_R-53Z{wb^9(}J3))%!d-9M2H5L2)xx z6b4z*<8I{AWf}dKKsl1t8oS6FL&v@Hxp;ai;bHxgZ9-TK4^Vg4#Tw$d8&N0eA2Xv2 zC6OYE!LF4i|BXWa!IzlwNT19(IJe<-OxuH`PyCYmza|_Of;W2u<$=+*$ z`q45Bfpg%#DA6X4i#z(G)C0Ooc18L{0}KDr0#)+mmNBtZgY2r*7aQKFK9E0%mq{s) zUmn{FU&2KgNxh|zW6&ZPloX~$``K*r9(Z~jHXR0&PD z-U7?YLHoT8?3*q$PZ>Qs(xy$tK#QSf0N#%OMyRhj!g5cptg|ifG0KOYE=q{+%r9P^ zg2d3kGO{EfdO!E74PItGn3NF0u8?lrf!5Im))B{w<^eSuIjeU6YwJD6zA~mQWLC)_ z4r>`E8FCstTuL((8h*jljiy>RCa>PJ4(0nkJ(%L0fH%JiNtNRUqi3mN$#hBGPs4eR zI|5b|2gDC_740%%}M#AD0uN7b6P+E06_e=Ihk478QNHSnL7QKch$A}cGRXA z!k<-rMjd=BbnRl&IUI^l{U6jjx9iX>{8kmq-97Bq#U4TgEam%F-SEd zs&f*7B{dZF)Gf?DpIXPS_ohwW?{w3u+X@}`tQ}Rdi6jLJ+VUu4FV}k91tm<=6qr^t~*erpKD~U zj(5SwYgRhx0h#tx&`7&U%UHf2AmH>Bq8PY7)?m36;odeh)$Vdb-ptiED#r8dBK%+d z7h-#Z?~xHYA^p93*0bZ60Y_GXP^C&W6arMzhV_7G7kI+56I)m)rZDb}ivecR!t_2I zfxFX?QdN*12*aG>z9yc>xD}xq)*XozhwbRFTz??*_`EaPx?r+%rh|57dq-Ip)==~W zf}pA?)V#r)wx)sU(R#B{Q;^rOFhALp`lC4zEJ(-c&@OYU4KtTXCVh52_px@^tHUb8 zeYpe}c%PO`QND18w&$5~$+~}WA;@Gd{LihO;*wqvenz7DAlvCSfX6JWm%VwwZV{Nu zSiz!wDzZOoP97WW0A__|X=iOm9w(r~Y7hd=OjPs*yAp(Wk<*$Lr>7VX*|nhH5ht!4 zwu_8_XauE_ygh}v(d7+9#@%g6Y`_NM;@3YX_f#bpOSHSSJ-~x34X|F9hi-$DcbtNv z6jP)kw=qPtNbN?D3mWXMZsM(g%ZSAt&tqF)St6w`Yr{7okoemVENX@Er@yXh@0=7q z&!$CYyUAjH6+j_>jD^K1R&^(`7hpzh*k0K9TO%P8=520LGY)-0{c-Qeb!)-v&>%l!7zr+`?=Q*ooG@L?gKh@s zX_Bk>gKGE#KXwS&1b*l_e9o`D?aXG$H7H|{kQ>VDaBP|R`#)^%|B|lThBPl#SO5SZ zF#rIJe|rZjLpMWxV|yDLQ)3rPdpqa<<|K4%?Ke4)es1&!ockoSS0y?>_YrK+#>}?6 zo)THYw?Waig-UF>)TpKT6y*-6U;1{Z?nI0AaLDC0X%v7(ZJo?GvN8Fx2}^j4>HRzg z$ktip>fqKk=c~|vhg0R+Hm|cy7gdQg=#H4AO?g6Zn`pofp0tiY8R}QRGKX$S5R-^z zl-MNOlZ6_IjL|QRVrg5Oinu>*SmH>vDfaJZwT;;2*F03~Ne-16O%GYKA~E$zMf(=d zaZUr#=b#?+9+P`~){W_3UtYO#fO`WKAAnz|XQMhKhK;8E4%LZ{8;U8X9&82*En2D6 zI>L$2=uQ%9Zl0}n;L&iLbyrbhxQ>&p6DI;SKXj;-jS6XZl@pVq+Ad?=tuXa{4_$X1o&nst7k8u-W2r1jp23qtSg0l7*K(B4jU9ty_> zz#4Gspa&ka>y{ZBbP|{U47+I_@v@yD_4&sZFwclPXuE)t_3T?SV|6}EzY$Nhito-2 z8_@_1rce}y08}V1`$g-~m}0(|=AvSzM>$sXau2QZ*&Cr>=IOyd@GKhlZ=v(#h)LfS zTaANq(8knyHUf2>98jeiL{Z44QInJrfWS(oUr(!rtl|p2DeNUYke}TUO$Ir&?gK*l z8vvS1{gvG#BZ!NRd%~F05$#%q8;F=H>M+39dB-;>Adz$frUvY67NM?gR^>9~45q5a zkPGC4iL$8(7MSui0J)(@Cc2F2^@0H-^~XGts#sz@5#zZUDUcGf zq&}|V&YYbS3?lXoFpEF^g+n#L@F`vk5gU_?>Vri|gGtJP1Yjx|h%k5HkIa@F$UfmD zS(54u(0t#;YjyXdO?&1PIDm^mOh5rFU|%{qS1b-jh>?xeVNkx2nzLpmDhK*)s<02T)uCp^(?ooHd~{ksNFDh;BliFD@>6TNPF^P;!A3OQnSH7zrV-EhsLu%@G94XrA zQ}I9I)t9N%Hwwgou$~btcv*H;cZ!2RaIcE_pPsjMz%QP^b$0jr#mg+ihZyd^v1|AkFSo&GGzq-3ggqR!>&%tW4!}=wSu| z+w0-D`S&04yxF!hB~%haj=feU}WX3E2N#!O|h~qPvU|Hwi0{p);M@8fbvmoYx@J6 zh%3Q({TW`(VD=#9B6WMV6lnL*uwki}{*a(L@l|C^xll4e+*EuhpeZEA0j9n@QJ^%g z)hCRaVAQ&(&Pml>PZ;w_oMBjhD~kU5JV3^wWh;khBFu#fe^^8M9Wi5YM9HC4b*{6o9+SYBa{{pzSiS7*O8ujB70jvil&@_G%Y6B+eFFt!6 z`|+Wn3=rU~L6QhMbrmdGsIkE&kSW-P)Dqg)RaZS)>(Bof6_{i!?VKhHHpyp{ml}}` zR_0-$o@D1Lkq%YMQw5++dH61xoGjumJY+XF7eDP2Jry?xe)*D@w@$m!_lFp2>$59v z!?a6Q+!6=|A;f6SR?H%Oqe5dm$Xn7qqp#rrZiAbw&j ztrWZ3chs!~hu`V66wXJqfyeS+0y8md0dN?~K?p@rk9T^tXbS|{OeJfP#h(V?+Iztj zy@Q>lD5R)1BZRC#-yf)K6B|R}b^QV9YGqb@pgYzzu=%2|7xBX|hDgHrQ%Uux0*)g{ zNn@iyqv1vUI3mm1a=opP_JwZ!gSqgaB0alApf^V^Ttif@P+S{uMr)98nv?SA)0UkSS+43i;s#hJU`^=f4er3;x|O9TJkY&W7HA{;mI(9S`2Km!*pWCQ4D8Q>Iu|Q=aF5Lf|6Ra1Oa<8r|uI`NB0*k zt8XInD+jL^lh38(D-+F`8R!+wJ$+EicnA$m+U@@^3b4<=a%Fx_kjul$(knFj=o+e6OiaLRED#U$Zb$}go|#M zboP~C17!e62Wm^>YS0IeUs_EC%AZQ|IbEgGs|gmy%eRh#M9&)0>7ahSt%40bim z#mAC`0VAFT7Lu$P6c&E8yY(#^3~kcYMOrGZ!z!q}W3Q3Muj4zOBsJrC&4Hk*NM5ti zChgsq8x*^*?bq0r+l;@rT(RaTh1GePEJ)eu{L;obStX*&qY|GGgqv(>{Oxs3Pkpvf zKhOp4DGqqiK4!nh&I(~ywbD-OdRs3%S~kI+(7SG)E}{2prT%mOc!!kUj$Z)tSH3C0 zl5BT!|F6^AkN8U&`OrLC31h-K@aOI6^X}{={%QA*HBIC+e6TO%5c?9++d5s>s61_5 zC}?r8U77*^Sb|$~+1c<$5PA8z?raW1Oq!NlnVNvw6d|uKjtP&Sv!4&k1}N72K)Hf& zAH4`NziII#d&YVkr)Y&m_l9*6#DVH~h@ox#O{8{pqVCttOLhp=AWmgoED`lOJS`?#G+PMcrD- z?Tr2tsvG)<Tajq2JC@$;^!v- zx3uTaoy~LN*(c-L%TSl|h{)xq;q-Fdn{?o3&(_(~3idOOS*e``)?y^LB<;$9JJF*&5RtQ>bgi`t7TdTwjTX<(}(Y^t9`(qO!liR{M33WyCtsztsabK z6p_Y{J2vqh_M=D(4@=!@)@-zC!;64U`Zr4O+oQ!Sk73YLf90sCdeLW?U?HsRHr_j-TWz4?qsof#G5irM>k zX-?Ov6sw$~p_4Hu4+qG5L2t&C;wZsDe@T=ucvo4}3~a?tWSuWZYC9xvL(yQe)8?je z8fL|LMbG-OZiAi_&bGmxM76&(0=tMP6oqKMfdj)G4reHYsIbM%ecmam z8gFFBhKpZJlB%xP4PW>FjTgP!d)y=w!Y4;N?J5EQ0E9?@04V=K)65NBOx+DV{~JZ~wRYbWOFVP`hN5wv z)m)G!b|dq?t)8LkY7`-tsveba>h8e9MlT%G-mmUr{&*>PmmofchClT>UioFTV$(r;#$ zbbT&LZ-(WOeK^|uqLc5Xzi>;LO*-^(QcfnySY9R?cb~!A$f|tWE#>CkJ#|gcpB7DO zX7|jZiM-5W%b0pR3Dvf+7kRjT9hF=>X|Z9UPg3PXb!D!wYFI2YlYanx;^Hiy;8tt} zWnryJY_v+!lBpaKN>;w|A&)OTU@O2@*AVQV^mL>?fd@W26r_DnlR@_Pq97a33mr$N z*ZZ2BnX&ta?k#tXQv-MtpBUZ8|KkMAj+5Qmdh-jVWRke%8C zlv4T8KWu7ExvA9keldnr=nhg%0)6Lz@VV;gMHxCJv3qW==))vcbtB4Z1HTc{P^c;F z{Wc;|Q+&Vy=!d%*0Oq2a^3ne&FWJS>J!E*)=w%#uIbBrN7^Eh3@86<{e9xR;tq-N` z#$I$*hCMxtG{36)Qs;%6CDMzkMS5#&AaQxoZotgb)0eFyIhaT=+*#5BkWO6D9?d6iIi}SsUxb*Qb z()Ujaa%leYyUm`i>3eACaef4p=cecD;J285?+?BA$MyR8J8AdFLEi6E8Gl9?zmA`? zS7J_1-p?q!TLispM9??8z1}sxPnvG@OkY^&D?FT-dD{B%{QcPzE!yvCP7{9b7hBic z%^3|hhwr1Ai-V_&mrzf5@0sKKf%E;IHL*o)@VXzCk;8JBdk3!NQ~@f=EcAtiNZ&dgkOmI$Wl$nKrOY`3M?S|3{O|k(Z6n!LaeQNg6ie$_9We-gfgk#CYFd2k6<9i=-S~r4HfKrw(cZa=I$+cQA4Wq7Z(5 z@6xw*a*vcU7ew~N#+VA6fVdwd*QF7wPPeoK!?XGTBzU@=uCv*iCw5F768rLbWc^r z2sieT<+^)NEbjZ05@do#P%d76v|je&+_7G3NS8xkVC)<~=2WT8KTl<(9vVB!lqYcI zHUQKsVqbf|Cf6+TZZu}O?>=E1&j4}A$WfhO-|ApT0m&QC-f}EojP-!E=Xai9WA#=* z8y8Sf<1eANZoXs6L4QSFz^M*aa@Cvbbd{oxvCp19;iDVp{#K_SGMq_(RRWYbZ7XB1 z8nJ_w`eo7wcxS(;UXN8(O5VGF7^^0Kj5e1$T0cEEmZLjhOk0*?4g~!Lb5JNfKmEdg zx#BdsIBkFET;8o+KOw6FW$kJZU7Q@RtToKZTkQ`8j=Nmhnp55D2_QvnIv{q*?{Wnpdgx)1Xc3^VVZlo?;1JNv{P&4S0+t=ACszI!>Mh79wgeF_W znJGhR|B>n0?E7A;_SW-PsOLG-x>>98^2XjBgu36jC)1vT^J72-r^C;2nS zR5H6(%FYg}06uUhKaC!KVjWHMt>8qD0tXKFkZ>pffWTe0@ZgufzNI!e(V3OTvvPVV zOcDBTEu2^ho!^5vu!Yf4(LJKP3D=Gk!Xc65%BD6StL$kGI46*aHyssd2?E37g?w-b zI$~+6*3i#2g#=o)boK@?V{*(AF4KX|E(E`8dw?|g%u)k0eIa3L3&0+sZ4F`0`=nV? z%n=3H&W#d1vzD|D)Yg{#|f+Xq)NoV zK7*&gK*_*$xw0Wkj3qV{NEt8fB9s9Sp;u28+;C1?Q51VSf>lt0!7Kloycc-4_GB0d zHAOLJt+LXALA2s;a>?&;S}FmcNHx8s?LFn3Oi694Q1=rVxK3xz%P7GA?HpW-4`^$rcca1Iw((sXew~EPTT|{-^f6X zOTD3T%V#2)E?=()v1KJGmehxe17@c0rY|5(yXbFVLUpIuEgPrwuewcmJ;8n<>GmPSVmaHnD?h+rzT)m3R$)Alv`!-R*CWsU#_3LJ2 z;6OB}L<4%$>010@@wXU6=1!Jocv7d`1D;B(FU9Br>j~2X*aw5+$WWKy?EqHltG7}U z@WDgkTC6)UVw9;x2%65#JmSa`piqDev*wvS2Kwa1E944&-LuKnH%2tlSW;AMgO@hU zK{~c_BzJNI+CL%!3p0+aL}JIRXmQ-ECEs zFp@poCFlmbeVJ@U7Cw>gPSXhc2h42%-CgI^=)V-4ATzmp4b0BsTR@7^_+(_iKS zZ2^p-qaOA#cStqsp?>kl>@w7KP9n5ph+86ayC}X)2h~kovZ0PlVqnBA}dsTO(PG? z>M|hK>wCsZi*O`LRB$=jHCEZ*4kABgaC-P;4}QEp=erNC@m296_Atj7lD-cw5vXxF z-(pPR%z=&lkF`w6#YJ)|q8zZ+1*uBe0)g^^+-$SaqC@o}=VdHuL?2wr=j5PQdcJBy zgb67*dR>fUseS)8go}Qs))dU|dgD^C25rcQ*8sZZfEtu4YqIjHc|&pXYwPxwESs@c zgPr5e4O_6aa~p=?IYzwWfbf`qdgG1ZI^TH9URm%1rTwA6fG*u?atRV(ruSYi+21>YXj`p z_p}$+1kXt!kuQY0YRf+Lk1R@#_t=S0=69V^;DGNfC`Y&P{x&3-dBHlt#X!Uk|@zS*M2Q6e6$TaZ;&S0mI{gRle#e9elFbK6`b@x-1 z7p{S}JOi;A#kt8^;!JnbYZ4#77ojv#&udu#)2Z-WC$^oeWu4Sicq0xNFuE#PI|!!} zYTOtBkQC!7IngNUu~8cOxytJ?=~oC+gizMVt^74vFDHc>+$3^aY6~YR&%3d+HI}#s zBEKy(s1{1Jt7y#u-L7;QQ?T%WS9F2aBXn!hRJ3eCZLhryLzLIUY9E%)=yLfz-i{zQ zy-0mUn!=9CRQb$psjrg?SZH|qrD{DW_oZ>QijM>8^cqN%7LhmirW!m3;chi`W1)uF zaJFY#3>Iu&lsiyq@MgRW{9Tt!_$Qb>i&rxGasv`+)MTycLzyoJKwfneMT}%V4!4O& zZK2{BE!)!d_-ID7n>i^mi8-ehHFt62gKn*?0Z0d$aF#uNztC+(V1Ir=cd15FDoxdp)o@DW#aJn2sNR!|LE@v?cst#S`2ITD@Qn| z`9R9(_G_nh=WOkT+_=V;i){-adowUPXv_gRk315|G1N-II6=2xkL4oH>YsafS<(~ z4W49vbhEs&JL?Kg%}j+=Z>M<`m)h~}sPn7S8|e&qyo6HwCb`K0PC`U3L0};8M7YK` z)TOG%eD6gs?NGjVn4LjPrFvRIH#EfjGwRn=v3_!~Fd(+^l0X-s`ad-!sr6LeMLv-m znGHX7J?*n~C1869Y@16e7r?6&cHY*uAVu(>q1>Iuw9~UnCl{xR@RxbQ8@gXT?ug|d zuV1GpyI=5Qs0Kj))!RZr^WsDq{Vm<|DSL6-FqdE%lR}Gy`?EI`u`{Hp>0@|tdm%3D z8nbhaqT2v4Xo4Z&Q&V>PTE0t9mx%U_g18cVv0c5T%4N$h|w4(wEmJ3$|+-3K96%F;F(#-C4U!BH?Io5#8|ZkOu>zynM{{rw(~aP-0~tl z*jUvRV20?3F%j;@%(_;>&ig_vZQ2sN{k`DJUw6T#NFC`0RH&1xv|c8%{)+DIxQ}(E zAE@(%=52n434i9jbP!AdS@#SaJ_A~3S@jJY&8N$hJ;Tni*$DxX0wOPhbqBiieqg?v zreU{JYjsnsoUqYjxjy;6c>YPiR?|}7$H%V8;cHL&)O?W3h?R9nZtu}@-CwFNuq(pl z%r}ifNE{ppdLgD-jJj+zN~;pZqAOX%KC5}uMtAQxn3=fcuNIZ#r9Su7qi7=e;&{?Q zf4H{cmQG;Jj2RQd*AwHqfFr5gS$(6HZ)d(XU#z>TLg;UIXT5Pa_NjnF(qrHtw?)VG z8&8w9c$1gn<}V``-PM07wn0*w%PgLE`y$;}$sgBZAgXX+D zqm_lS);FP$A~Hfis27i}{_&L$T5c(V zy{u+hCz=qVBUO3f)^ap?7m6c@*9?}hkLpqud@)@GZY@VbT?%^(R(z(6hl`XO3euy0 z)SsP*%GGL#xlUe1N(9yN6u*Unq8jvE7o%jWrT!=ohyGqeDXiNN%1Q1XzYgOeiPCkz zEQx{kmY?2YS?rAR&XhIM{P(u0?856eAQm#NY$e-!4qnWrjl7%bCVMYLZ%g#&Nbj z-4iruI>8eorsKAeZ20QJz!2r4b)rqZHd%@9&*yi8mG?uqGh#qPoe0-sFlx>5bKVuUNpgcLO#V@ZlUN7<4Ei1hS>VjA__p& z1%851be1<-=_r(B@P;$OJgtQqNMr{7N4(Ra(O0c`%I1^3v(9^iBF}R%ld@V_VuN<; znZ(gpV=bgF$h!);+JiTa);QZ~YHqUbr$@UvJAH5EAyGsl>k>(x+KOtEq+1UWEu^-Y zC!h`nOJ|h_2ns7LUY=BFu?bi4)l0)pCgU3PW%TH6u2%^eyJKbAExG#acJ9zw>YBr_tk->O;2l~e^}MXL4c572Mav{NRWV1r3xQmlprPT zLg3JN%WGp(AWTj@o>eMjck}j=hghtxmf8rkiRBP1e{G;ut&NjFF;V{WX`$vI50bP$ z6QTn=Cm5(xJGHAFHjAT5mXu%yc9y-!>8UKAvDk2E2nFB#{dulDC}%dN#Zj>3Rzn{4 zq5aSiQ&Ag$mfGHRPYWjG3{Z`JZI1=$8hix}!V>Yg8A2J3;{urk22g;vbmyrFP@+?d zgxfmN9O6JSn(FASu{QL>)=Y_Us_9O-07n(Gnqt`6@Bn?0x#QHNJLnNZ$M|iqVdwe7 zpVXp+{pHG*S%`DVWf%FZ7AaxRzx^H#y%VitB_(v;$<$83)$3ZW6CXuca0P}0BB5Ts zyZt#fzCT^z34Z|gNArGlZKYhOhZiD&^QQ)|ZF$XS&xTCwKzrg__`WfX^}hrGvzy85 zw|~-UFM*!NxZ5ah#_*`OKU#s(XmO`PlB5qyik#=3>7;?UgN^&laW_*XK%y)tZG%4L z7BNcz&BK07Tbh?_A69N4B-X{PY?0%m5$Ex$0?2O1zHLR3QC1#L#w@7`!V2c{9n38h zOwa1R8G2`eB#L5#j!Fk~(A|lO(!+KBo=jssgz@sb^FfCrFIe%_%%66r5QC#}aCgx4r2CLD{S5S`K#<8O$I3ONhAXIZCKxj#sJJV(9=MZtik1eMo` zF`vSNT*d~~;WF1@h_I<*SYKCzPphsNA5o0is&O_CEkqdb3U;#GAhpdMqLWAiqFajx zfx>DQc)+4VhT^eM@f4J=glp5S7pZpnrXP%MqNku}3dKpTpA57_0~$D5R$K9sXa<7n zi|@xAs#AVhvTDo}A!Z%CcU?CSxW;hEKdm8NKd#000Seuus4S>&s>2<5pHfTM;f@;o zu)T^-bUPfo_i6LC>1$MCv}J8??l~J0=86z^0`-o1z%5k*bfqMy*E|#lu zm|3M>>kw!|Sxz*^ZIg;FE+S&)F)OMKE9NdlQLbN90Jp@ze81CjDIe0bW=5ZyW(Ygb zD>o}AExJ4VMi;K!%Ps%|N>XHYRY&z`M8dYpB*uB44mRN&c-W9$>U*%`8;aBI-xY z0HoWet`B&5U#P7kicb_56>qrpDh+3yrh?3sU6!cmZ#aSZP?wYt`Xc0gm)i|}{`+;1 zww{W2JT0uP`FA#KWf^aZ5Fo&Y-BvX*< zeF0Swx#!0d`VQF+t3FDVLXx6|lDzO;%54|UHgL|^L?miRU94b1RZl`|Fu9*fAtKl( zAzBF}3cF=(jeKA*bq_jmp_e4Sph3~3*jaY^Y01;9-^58p4z+hCj@D8*rf??(>L`Uo zYAA{BVGWZ7`@<=h=$77+&FK3_aaSnkO8XVU|5FIqShVmoLRvIhc{kURfe!JUhBaY} zMJ?${+!r`2y25`mGPcFcYbMUevLUl-Vskhh`tNkbZYCT?slT=&cvfa2c`&5o=WXoO zXcf4r5O>CGc;aLl6<^YnjKBCQBC}0i9xqO0*Q_YFAW)5wxrPEoePAR-@KO|Ie6(;$ zZESt=^kzs9#6g=&Ov@%=7cgSz0)5!in^AHE4uJ$a4 zWZ1nt?}>{eZ4%oKYRqXqGXu z`O@)qLWSwJwBOH&x2aJh8`JbfKYhS`aieLAj$>_Hd2LTp(%@{3NBBgTABQe7@h~w7 zw)KPpA>xPe%J7r;^=FyoJ$X`(27ai2r?*}bcJl`l)uc(FD>(4y0y|6(P?GV8ZUCjK zVsi~FC1luWky{Hkeh53rZuXo=^MjTnhJgLTZ&=Pv=DGfm{Gpct0FZd`(-pU*3P`K~ zqad%oswY`laQDI*CVT<0skkqoZtY}edTrp;;UioOj{l-1O}_mG4@Lk=%@S%NUXSF) zCzWRH1*x9Mo39tA0(8z(0waN|+@n3P#=KPoVF3k|nW9`}CXn2m6*|t@s$Boyxx`@a z8YId?gSm(e>X#9Mb$JE|YHRwV3gOb4#zFt;E5&uj<8VUg030INjv8N8rNI+4k4mB+ zQ5Le#dbg3~%40P5Kv!ekO6fsh$+Fu#WJJLteU>k~GY14Z=Ly8m4A*-=18yj>FaqRk zGKqSw{&^G;4lH+|0Hprh6gv^&-K&vy%q?}-MJJ<5Gop!ZI;v6q6n|NNy26jRLl4<1 zR8MMPX88bWoL56kO4b+* zfE{z^z?hTUrgY8(Y64umoVt{_rP1V45eV!{$DlF}Mp!kht{gkYYX~> z4)MB;GEGKOZx`BFWeiP7vM5q@n6~|w(XuX>xPoi9-8`v7*~x-n(Nuc6!|7A!R!m%h&%eIK2pJGy=gKXUd|68%5z?k6MbuNScjow&Sv1Q`@l*5R5O9h}I1$G_h$6zOLWowL? zy?kPJLUm*IFj6=L<-H~A4G3lvb?YkVMD%T|77eir-%>kNTRWj+zqUndnb)W7G+Jcs zOl~>Jed?PR&IzO*9CFwEzlY(?sc3bA_-7H3m2^3&?lQA)bj+^(R=`l5%fY0TNJ+o( zAOcEkXCU)pbuX#1ETM25o~%~O%?kqPE;;u+!;?b54(4C!#J#e4{NG&G}QZZuBX^p7jcylG0D(l&{y_ zbG1oV9xlMu>&RH#6c+mMpEtdyKr6KU5$oRbHWz2G@?^G=Q%-*b8k3_C<~zPzv!6$f ze~#w|UZP#yt37~UZ{thIkd_BzKC_&Nm9GhMt6%+Ihme<|`>p$K$AenMagTlv*YyoFmZM8a&m13=8Oyv%t>Ft(;MGIonSM0a*MuM!;NPt^f;yYSl_8UDVrAd z?G3bHi@fh&_6%Gc5h_&df1-ZvgBknKRyte-~=t zj4rSwt#e6%8?(DUZ+`a~LVX_d9*<9(&N*YxA zR9`l|g1c}0L$vkMmjU>D^Yy=ye!d>j=_h%21)fyPuG0)yuk}B^d^K-P{I$7rTQN|A z|MG`%X=7dilskKQWaUI_#xa331yM^0w^8sNE$yGQ+SAFIFwaC*c&6Ig_%;1jiSchP z*Z9fF)6vcE-VgN_%_Xuw_gZPbq-kN>cRO-Z|+Hrl_hZxaX$DG-lcMn;rf_g==;8?DjDwK zdn8E#mjFf~l8jBWRMYr8Y3o*ptyZ8_Ri!PfXM8{A?1=(s+N(lv1eyfrIH;08gtfcO zc$c{I{M~ceQ9@#9QHO`xSh#v)?(*xNj_!n7k8P;jo+K4Foh(=W;@3?-cV3%`ID2Bw zvnhaHo??;4!k+q)jO0-zj|T0WUSj^Kv(}jZP%EdjULi`gPF1R=C)JK$bs1^r^{!~j z8DiWN&m03*G%R9_OclaW5s1fk$fD9R^zO6{nxqj%v{UOZlwlEK&1c9#`{sIs^xix~ zOkTHQtfA|k%DorkC(!&-4}Dd&J^WPX6^AA_FOr5Q*XPdSkRp}yqHx2OGH)PfB0RlSOZwa_~O!yqbW>9U@8JcNW8glC@-e* z->isFWcl>X;VTbFkx#4@jJgKHSS=cl4i_WK7qi+QTA^^T?_#lAJyEW1OSshw!GSCe zA#N#^viKaa3T`Lg&n+;%BW53#0D!3SM+JDEWp|G zB1|1OXKro7&LMBD#!Gm{>!;EpZjp8ks0Hx&PDfS&&(pv2+aPhXMaxxG+7psWTQ)L{ zPqYz#-Z)SfQ!gO+_37kY zU{pso&NIs;oHgrj#b7o6jx4$SiYO;^`v4#A$)f;qQ7L0>-}Ze@Id2#Z5vD(;^e{V-JYW> zN}dXjG+T$c{f>s$r2!Z4an7KC?#^?1EjI<}X9&OODd4~)kFOfNJ5BT7X(9!* zCFOGg`a9)Htx}D^c_KB%$rlC|d)dK2C>EO<2U3E0m`O}G=yoHqRZ^*VfdqMs0rb|n zxDKhcn#O>{-^N@?p1O#Up>B2ISJ*DzcGV7ff{?>%_AShYp`erCv!>Ma-i(of@sF)3 zuNq&1Pw4R|wTkTV^(gP~HL7m#mYu0Wvd%VfNMB!mj%n$#)Y27J{us!3K#MB_mUEtu z8c=kVfX-E=X5oxUYM(1V^kmJlBrVU%XdXC@S82?&D7G?LgPa9wv2r~Pu4(4$c#(-_ zy=+3Hx2!K{T@%L{ex4|ZG%$-6kT_N8C~TLoxC~l0Gv<;@yX{*K(l_i0vxjtL-=xba zA+{3B*wr^C$>+jl?dX(u^r45gF!ki66R8SguF|p@gQC$x-J{nvTOPT44pf)abb;FI{86wfe;`S@?yJ&P*#nxI( zjEEV6U*jBN&8RiP%YbL!;un@igr1yT4`6$-GaY>UoJ;#ttsSJ0W96v0{d9UOYE$p2 z42(O}Ib&eb_oZV}r)|6KB5)<2fi8=JkP`?dK9{%oZw|%=)Xh?;-y=zIho`4mV&`aH zIe99*jI``GBeP^aR>=Eh_6_g@?mg>xlao)MzRpfy#am2BnYO7imwgdqMJBz>y#|50 zav_9{is$y9Bw_?qjq7fpFyr5dNwd#qIw~a&zYfvZu=d;G@{SJ*taunH*XxBLi5FO= zteVO-CAduztmA?xgxkg557ue5;2ygq8JoJZl}Ep=%(El3OEf1%w-sJg$`)!GaR|F` z0biEbi*6-@5%Ucd;vDlgN6vsoWROWK%XDu?tRWM8-pva_%*%6P#4qF6VDeLiHUUM= zNTDYh44m2eBM59;J&g#_kF|OF151#nnKL9Zx~;+AP!0=u4=PN!fnwTwms9=qtutcb zlFs?<5>So>pr*X7RdCg?0A-Azp(wfL`oej8hdbbBlFr&A%#5bjZW|jLTf)^xns!ccij$*jURK|agQIIM9**@(hyg8( z>1*WtHvnkl0_Xg7fp)(DB)NMa+UB_~x~^xz8gM0&K7hU~(O5h@NW9Vz2c1ibyNy~a zp$xK2B)g|8!E->JK%CQEKETZWNpaw-hT8GUhs(_%P*FcvoD2M|rc= z_~0A%&G=ZJo0SEt+=_BB9WQ0aV+1JziI_5;43nc~hvL+4e5b~+LIoq?Bj>z{P+n#f zSN%2aS-kw9bU|ZD626W{LMz~jUaVb}JpW*ID!6_-{btC4)K#y;fH^k}B5P^Ns1A7m zXWo!UQ*U03o4#5;IfD=6cc%V+T8VkU92H#VRSXP0|g6qaXgy)Wc(z=ryoE_%2QDM}zRy-Sy79skb`phzeJ?**))WaHA=NR1Jq@~5VJe* zHOv8uqh@baWbPPdM?@}EX>}_Ob*Mp2v>P^&0X)Nl?+_a~GMdCSwWu4|2;P6ZwU808 z%WJuTjDuWXfeUxNQ`nzcp0G7oyA)lowA&=r%)#kI3EbT!;VMvSzZpE)K<_F>xI@dL zDHg~IKSDk&Jo(uO#!6_|6~+e1m`vR-*CygD@7`~*txY6la|4x@ojyJ9KC5V#;YGY* z-a}All-s*}$B(bp_xo}D9t>$}!vT8Bsv$*TF_cAH_yg7#Yt7>A^2u)XV_kb-)FsrL zy9U&a%WYZFb9q2M9yF@M(x2$rcBF`P%k$WC0D~^YpbMOG2w~pK0JNsa{j=}$^Svu2 zkFXWfj%|M~aP1aEKk_S*N;Xw@N1R8sY|h$>nf zW6Ds<6iJ;4hT-KT4rwS)qb-XzjXwVMvT44};%DR^fS}8cAFd7(e?)KQ5_$AqM2H#g zWdQS1Haos1-|-X-)G2u_ReW&F09?t$lBc;`6mTbHalu#nYJkDLB$d%E%`*11W7 zIIhF(y&>A()ECkd?nH183&LrPhofcr6iH60l zl*iL(@Q{Hq+NSru&iv4vKhR#!_>gidUaJ?^7U;`hnJ+XFVekIL#iq6 zRMWUAq%G>GcFNhG%83{mo-W3x!Ul(4N|N;U|C|!qD3cvw;g(!1xvr5>FdYX^-l@j) zCRwg-qDpS!2JdfQs)JQDb?22d1@%U6@PR*~D^^h4Q*EyzhHk8|>cgQ#18r4)%5JMd zG8o4@k%*lddL_SMaBX&&%CbN+B?1=TvXOW2SA$N_-~|W%*Xd!V8T+2M5&?)(9=ZNY z2ma%j*c%)B<~9GbjpUDeWCsfOBt=% z3Px0WW(dF;~2AW{0h?~iBQG20z}LC@CAyma$rbcG9$Q{%4I9`jtCw3`kFxl z*WmgL6#3OM1DZ6_hurQP6+E*7fpt97pQr>Gu2CURy7%KGTTu@>9kZ!fDZ=d|ZY@KN zJyc8Dx{t>8_TJ`~vNV;$eamJo$JRudnI~1w&PAmdYVvvP%$L95GSkLuq zeTj%F)9sty-vcJ~Wg!z)NEw5|Sk6>c+qD72wxqJMJ(o9z$|ZUQj~8EqB8N9n7QR3jJR;kQKQ3i-tR7$7h9-jK)3>jx*?9DY4}! zKCXhhE?KY7Z_3+ein|3>y+gxm{4a?sjZlf3$F^=X$P-9 zsg#y`PEr=qMC#9{MkE(N$HShxu~9kK8n0G3a6?Em?|%a7%jPJb)w`1>cvgtgW{$%Q zE>DQ6mLS>>3C1UF1EUdVGQ@4%5FKbY0yFkAzNA!!cx?>96>LiSN)YyPaaIeyFxtum zlvpoSc2XS%O2>9GbFkAWZa5hje7P6$&6Lokq0AG9IVmnE>LQs2>sdU*pM);G8`(3| z(ir<_d!z~WWMilBS8H++3TpFxgaFR=*F!?ap$e-G2^NY?S_$G1V*yy`kMf+Mrc&h4 zsD{koj^cRuQ*blFhXR9{j#!+)rPd~;)8=|CEtB}&g8Qc>8o#Ymc z-7M#f@0xOy+G3@tYF1t1^kzkaK6Q&u;+09nD4>U3%M0#)m;MoFB%x;GYdzzntc{S4 z(8)fL=St-*?LXZ$?PiGuwsy@XR~E~^ej{n)O)t+L#l>8+uuvYpTNR3%_~eDbAoBM- zuCRD3`IETW9tCpG`Qw3@pS1uP^d^**E9en9SnBTP#wX2SepLM(e4iBEyg0acT?~8* zc!3%DCKo_K?Wv_pL_3SrWgvgXT z+t*WXuGOW!o>Mv2N7|gC(NO-UyoE9O`8d%}(+2hU=`;SJO=)en>h0(7@crWGNWDwk zupfn5b2e90a1genN?Qj>@{Z(zbdKaXeyc#~0p@1?j8O!pL5YveU&Fo1uDM&l^^IeB zb$?UXqEk8uK{ct#XW=go?6tD>QhmY+K+gM4w%|%el+PTS zekJ!9XPf~&=Xm0qJDzAVWo(DsN@XRr_M@nsbqs7xczIjiCQ?#~P_}ZNm4uTz(eA(n zHU@*4>MOXVN8oVuOuKIc&@<^Nzwf*CudAC2Qx{HEJ9;_zzA1Y8dGWXL@75|Qy_mh zuw7ML$GaVPokwrQv9>cw#5dbLzr%td$cN7QLpIMWq{O{@f!&g2e5c_(P#GL0olOjF-#HioT{ zj!fi@h3Q)@s*Vpn4J&nGG!HD|*;cGCpR6Wi%b^gcks-v;rR^o+mic{VUnD!pE9d$c&*U*+p1w%IRK>zxgeV{ zfm&b+$jENCvLxdZ+Ik8J-0JHc{v{kjpodM4P>W;M&31uL<#1daVLecz^As)sZm$dbc8WmW{vkFew{@;<>qn)}p7+R;bM<^lT=nlbyw?ft7JirgXEX;pZ<7 zX@KK#+#YCt8W_33OW?xmQb7e#m*CnBEBN@pd|<>#I0AZcTzD&ghqE+!*j%aQtPzd2 z5Z`D6*f5B&)dM?M0L26D%DA=i4>G&sBJVJP!)w7*}tF8b4_Qx4Kt8eJ3i zpjOEry&q5~5$F$FXtNAO1A%@Uxkn05+roE)3^=kPY}YOadoDO*-u;$OSDab4k)ek1 zG{87BIyQdp=V}A-l;|8&Z~1p6&{t2(RRNK!Xl86SSx+yzhg&vs&1F|&)~&P6=Lv#F zLnM$L^OEM%d)==)9xRTvuGxnG?P0beINOiw{3_L_$8#$t_#sW5?xc6u<0(ZQ*I%NO z2*8L<)lG>xDdTaq(8;{59Wk!su}wGJArD$l-FEdNaGa0fP>U0p66rFqM5D2oJ6%nR zm-8=EG$P$I(PnEG8>6{<6Y)&lCSNd^2!o3bMamzh6Aw zzDv5n>>F}y&xoOCSG_U2C3|b{rJX1nFlbPJr3kD0FyxcokMI}ilEZ*tW|Rr(Nm^m2 z<_@glyr3jo_o>gn+w}nBfQ}`Ig>$%ZfFhpOUtOuCJjgzWK;q`+=jYk;=3Z~!%-0O0 zka-k@n(3*P(z_B3vu>qQW%i=dN;XIzKxhd=2z(2;UcMw^U#Ou)lO zV~S%t7u|^vh0g$2R0)I(!e<#NBHyJp!|#aVN()Q<)8zav4(~@|e4h)ijR)(Jqve@l z2=%7eA3PBXxW)J?Ex{hvq#yjl47LF|HKTM-n1zYXA4p%Z8()}7FJGq}{rm=L+9S<` zmoTMv*hwE5Z`7ppGb_q~J9*6g8KKNeOSBqiWduful3SYS*9n@p;L8~cqZQ5J$I>aGV_iRi z(JPnmcbM``1AGzdJ5b0=LK@_znXlFF*GPK*kdB7 z!b4J6Y^MN|#;I8GxnystK}>A6v8S-0u2P=s$&LNy2l8Y#7LfdZZircEnwHaf+g!d; z)VMN4&0A)#pkSv0QzR(KS@Ll87Gs* zSW@TmRzrUmZb)@8Ln>p%d*v1vFkk1_8 zw(*Y&Y$J`5;3?6Bn6a`+C)dSk2H|RTA;pCbB+du8j@P3Pz;*@lP?v@3GF~jyPti|Y zZLMYDS3|>Imy$BKgA2jsltx62%j);vlGY2ecU(gBEEt?3@f(RRL9xSPC9nWhz{Lf- zRpZf6?zfY9CBKk}0U`i8 zJ8VGB=CToT*tZE(Hs>D=0B+I!Mr9ubsW}eNNTi&B`$KqDqt)8rD^sO&rfe*koS`xE zG`h+n|CW@?!L{Eq3-!IO33XqyPb4G4Gq3;bUR1#XW`&t?CZ;P8z98@=1X z$==2OA4u;1flknhW&IBz_t@-3DuvmY6df zL@^QU?NRa&M&CG79!{{ANMwxN#HZDKd`c}k(imWtfcPdZImdS{zlQRFlI$bz9MW{l z_6Hd4+$6iiWZWoi>Ca@L1ia)}O~mP^iLeh=*g%!|nyB2JrWXl8=GGg+i8s=c%eaJ& zh;^D-NwXOr=paqPvxLPI3uYI=wkm>hK~c#igZcUM$Wzyb+a&O8ME;0ogCS)E?uZa9 z%XBA#8p1hEtfj;n&Ff!8S^h$*Cf~;awnrOIa-=31eiH=k2F59gCQ{8m?8sLzL}v*~ z`q@AwvK;-iFV2FoNeq*;J1Bv45^3cF^vC4J`QN;;?gQ<_tEkHnFA?}k2DCqY<(2~L zgw1~#uMyKe;jRobn6ifFNXV28Tpc%w<>%?=XshL+2~uAtQDp^_5=6zFWCwXzCTOsE z%(BT4A9`#P(J7yJkVL{sW+f59{hAE*FitEa3`Ey8k0UlC={{UVOwIxKV^zr=NCZT84?ewUwLajfB!(sH z;3-7L?&yp@G&u7bP7mg+`*}g9`4QRS5T6$pq;+q~>3=d)Mi|f5>vam3ws(o`=(L3` zPllw4DMPkr%b(kMf6R?M-bRZ!>BH=KLc9BFUiDo$GP9F!JuBmpHc0Z)upXI=58#LTOCy?z# z*;w=7@+{07Z*yhMjoTd*p{)(ve7>`~sY-=y!jH29H*R%_HP47=k;=3kwA|_B##G~0 z-o~0aclhH37^2%D{n7#MC>DLi*;2bbD8s9?owXB7UDs$k3pUITPEQ2~&M@cEPyq*< z*5R5?U%qZd3Ph!Rzt-F9)Om}%?Zu0$`o~*gc_|k@ZCOg5``{U_!oj=0_GI75bl#)_ zB>xc1sO@y_Tv_g$Ip>cZhYLEi4_DFsw+I>npHu%tGt$|lGk{#Auh?@lrHCIFa4lCo z&&RAk8J@>Vjy`e9B(|KrJ@_AQGBm{*W`_@rWsykZ1T_T|uM82yklYz8xB)QI$)$In z5;tb2@pae*G(iW`0pYu_id8P50Ep;x_v!1)>_~OJI4-p?jUC>r1O&nZTVqf1>pv)m z4hfe%lB_}-((XLg0vtWt$70EpdqC?r$a7Pc`1$$$Bezoe1WKaA!v@x9Rq9QUgGzER`xzTp?I)vfuI~+yXCP zt&N$URal1v%`hH+9_{!7rJFP&P>7P^WB!;PSs17q<{d{i&}Fcw1fI@(K_xFbaE&o| z138YzJK&atwJ~lvf?KyPw!uUY$+sVgkjB-d$Z=8@r8stip6B7ac(v#3K0rgW`%gv( z(Dz8Gj*8~H2w~N-r}*3@8t2O-Ww?rxR;P=jZt=oGk14+E%R%(YZCxgvYaEw=HlNz5 z+O*n^GW87OcO&W2H*dL3eC+L%tb5pI7j&?)tEd`bcU{o)s2;78)JP4t&2BRkvf63o zZnPCgcs|YaWbg`?>gP{54eC&rLVM?UNQjBwb{6JZd*v7-6SmGYq<;% zMhF*h&JeG59^Nc=$BeQz5wSWo9aNeC2etpG^t@Q2FVg)DJp(< zjUON{B9-{j6NuW2*47*pBqHlfvHK|?g@_h$O8Iu5B4~|iJe+-&*P!5PK#mH2YoNV6 zd#Z6{qQyPAgb?D`KhND|7HD;AD`V~~up z5UKu=?3LCeX>C#fiz|sV#5tRFd+7^+;%ETGYnO$ifraBvY&1LVkA%8>al<_6kIM!2;pf0LX_z85sm}%{Ps{p`lS2 z4yFaSN0F)^Epn50rz%C!AKnUV`JKg>;xlF*YoeJ8c@lSWOSI4TF?%=FhIuVPf-s#N z#TZQ8L^8OBmFzb{TvUS7jd&Syg=xz0 z&$0Hx&FP8Mp$6-ik;R&4J(+&4Yx;gC>xZp#z`Ei|H@;M0&ssG$GmwyuG7(@{z63CQdb}gU2{E)0c48XM&nNMXbrf`QQIK2D4oC4;0{f%ub}0 zDD#~_Wpy+v)f)PY<4slODbxf>gjj;;R6?%lB=in)j~`o*tffU5+hl*x!FZs-CDcHMAjz@wlUksu+2b{E)gr$>T`F@oRfr~ z5`gi<=P-}~`x2>!SZ>OF3COCbls z2m1*JFvPQcwg%M{EvtyEmDY6>b1Y+c;(mxIy@pGdS~CgZIG?J5Qg~cJX+j{Xt6{j< zW^k6hxU-u;JYN10_X32`2VGjHW6zTDeZ@Q(N=8JJfX4YC?}(=_QG76y6-~*W8CEeM zD`~>pNs7%hPf-_RF*QUMRoF)3OJWm{b2KW7+Z13@(!HNk% zTa)1D@jb2*k%3B)$119+9IL+$W^!`z6qy>-X#Ud_O){2Lp$wVHy1_ zD&m3KHX27CEZrj38q@mEL~N$K`xDc*fFKlo`j6`8`F|DpK1Au0O$4#=-|Tjn^b!d<^ln{ba)EhKsO1^k}wmAY{8 z$i4CSGV=EI{8c)P-GC%Jk<^4cB9i7MnJ7GXkh)29I|0h0B2SvAk}ndK%fkzG0#*Cv z0Rc=0)})(JQ+QchoZ37oh#iU&7iWg9?=RT_6UFm%5Uw}@@s=JLqbxlVuEYdQoMv3M zZ>$7BjN?H`mcYwr^rCzsc=3mQ_$^PlWlT$I{E5S45JR)}&77|TG6ii4yS_Sk1e=LNXRT&H!@g00cj%YYbfUouTE=&W3$gGk~=H8qW)&zZ&1K z?p_UN4UIHsEgHFf;jb)sx-kCm6fil+Nv{)6^6M;8OFa z37FTQ_x%3qO%TTz2=#7lA*%~g^{v>N9&lA;(n(QC{UK)oPRg5(JPxgAWnU~_f{pmL zv8IunRw}C1h{mpVD~S~X4fOyrCEUZ>!c>%#@SIj+f^qsXBMe4f_`hM~+?Y^phgp~O z$+QIj-s-<0Z^@|O5M)w#7DEQw>xwE4{;Fhz3x_TYDi8I6FT}DMKM5uIBaFOdK{C*KR_sb`Qb|5%E zat^SVBjaarwj8vGyfBrcsH!k(N!L;Hfd^3DKy&e}N}Eyr@+VPI4X!(kFYg=?=l(+# zl)$t%a6(@qos@>;&_n?bJPrX63G@#1^wr$Dw7I!X)fC$=5tMy*Di>rIoCl=2Y%0je za=;P+(7)LT&e#SS(5y_SP3Pj+Upb}U+Fi|OCuLsQx@Rd-;m4Ikx_*yA_w3gA;#Xck z*OIK3Sd7IueUMgo^AB>W^hKP0dc{X32)a|~Fu7$=SM+T?O~&7Mg-H?Es+f9n8hOMlqFt3;+MoiWOs1$oxlTrUC*0fcIb5F!FRU)pvGrva~bT|IerY@5dDX z&03)<({b4hD7#On?UP_tkX8#cRT7&mved4NYF1_0il&?nkRoGcucXL)mHUhM*Q_8y zOXCD+uP>bAeB<$)G4(B+$|P@y!}gI1MY?%;A}6g!FvXLv3tAc9mR&ktm!-%@G98%)b2mjHiF09ubr0-+`^Q75Ec zBi7*Elrje4oQES;EMh4ika%lL(2MindEKBI;nEjhlEN<5msdO~i?M|)_Z?CEF|y89 z;P*0O;mcOmmE!0)_n_tum(Op_M)AC_#j4RrZZZbsc!8&I{)AXAXWAV~ONTBdj1+_A z`3xcJKOAa|?sjsJ-x={uW%uz)?$Bm(Buiy(mZo_$PEta157g&3L(6MXYtN4=_niBd ze`Tw{hS_-rOa25&+94e~ho!OcV^s=;Zbk0|+CMjo%nv40^1S2kY{abM>-(CE(cf*) zA0&@|BJQ`oC!K$xSZKb78_WViDNZUuGhVm5_K@~XS^k3k*Hf+xp;rLpe=-$yM1X&x z^uOGB|98~?f0Zi#lTcb!-?!doK=AvkPq>4g4T_vSRDhr{xfU+SzR^{#7-EyJUEY)` z^(gdr{T}5v<+@d}o30igNz%RR^NxFW(oJ_=Q_28KiAc(6j>MCnc#$WEdk$Rf(wb0S zVy2`I3~n^{xX6!8RFGnpN>{QVZ6pz*h1Sy0s({QwR=f0y4!-m*kRFjKY0SyleA>k#E1?WGJ$=4pS27wL zS5H+a65p3?Mm>PYJL*-(J|0gG2F;=fvQC*YNi9P_0xNFb7|pFz1rXS^g0~$uyj#1#1s&WIoxGq(MSuC+rgfLYV^3BvRK0Gy~!%&?E&Y$?ha!A*xUj6B0`t zEU_i86+r|SZn~rzZ8^sL7~PmjIC6*UyXa@VYi68oV|zpb7L0(jugF>a@&6) z`q>@oeRB>!5qZ!>JIJHzc65C*kB4qSHpY0-?lc*^$659f?^OEaIsLCG>+NIC12>1q0>>z5|5yhrwVU->hnb;b{-4Lm9o(&`*gyaPSYQAExc@cLto~ysF?4b=^!%S> zTC9q6d^W>>%#a#60zwpolXyg;R^8oIT9v&myrPM$vSg5nj4N!}XZ!9m)B=#S>|^`Z zVS{<+jpnfi4+G4j{Q+kWOeYUEe8xApD6ct}tiJ$Td5T6}$Mgni?pmL5*0J$tat-rnFdGG^S76k4U-)up^O6 zx`%5O{$*R^RV#vAWtWGc)T|`=j$SdPVjw?ZEa?;k^P+XlyjaK#W3Nathxe^v+wMjm zvUwA|yupJ8x=}cPCSbSlJM9(lu8k_?>J{hM<8A-(D7xo}i4mRs(Q1SGB$gk_wrLWz z{i6%?o2s$m1%?+aiz3eAp~cU1HiD1&;Z`1)+H$5AYU~zRcUHwrUt~`-i)>tkK2huI zQ5%fGduE~>&tx8LiCH!;Q@LCvqQ>~%OfG;p8XN?#)v6;qMGEyX5YOU zy=#!wpDaCRxiMe{T)1)6UuIU0Kg1AIixn6Lx z`n+q{8ur5qKIt*IeuB4WQJu8<4(WNBme`h7B(8by$})oLf5Pm|x!c!hIG0pw30%=% z!d~SgP>@JN)P2KVyPQ=&nt%Qe6OI!#7}hm70DuV@006;%6LH3dHa7pUk^WDD?pEuc z5r+-oKSrD`eG?hvuckTyc3@|mcG-0nu^U_5t$+g3EkmM6EK$~%u%ExUmFUSyM=Y9M zcGn>VZ4<{2htA{2xVv`k+nG>OB(MnaizJ8_iP2%-g8j?yf&YM`(c~8a5T0_vShkCV zn+q8YEz6;S)`P?$4_wKa1sKF=q&gBQqci7#qlC#S2@I}?nx#B5B%_@oA`KcwAA9#V1Co-5P-?Plc%Jn8Jp=V+ z{PGUoP2P7zDK(XI^30hI)ERriQpo6(j;Tob;*GDQ$By;Nm){{2KmAH%3V0^U8=phtZ;SFUHe~=-zo<8A}>?XrEMIRuyC9Si|1w4|KsHNh0Y4P#E%x z)a&<)g_B>bP9pL0a{97q7o9?WJY1YgxA^c0KP@L;+B6D?%fDPZwP^%B+Z3Ln^UIbu zZx*Vh6gYPJ;%2xApLWE?>(Sj@@{#*5&uRMfj|oTX2@>v%P;^?l4$rst9PT&wEuL%b zU+k@&hm*)IDsvEDkU(Uh9aTdk(5IC_KzQ^Ws+da+w?=0~)9JWy`i&ZEK~#m;B945c zRG#Tt&n0Z(VpAc2e_bc&^EP}yOB9a}9~S={FB&MkCjFG%rhpt8W_=zLJKT(oN1v@&0LyA~B@{s3^< zrm9XIv{WhXxr)2dBEd$EI|TY4c{H4FZ7^jHagip(zpu)2RC6TuWiYiD{nXSm(c+1u0HDlae`g5 zmbftp^yLU0lR7qddk+UOvw#(zu~?%xYhGCnDa=E0^t5m0N-3e9QJtAmsuuS*=+Ppw_HeNGyv_`rw#4T2MT7Zzc!jE4 z3LA|&t@2;xYMRZdS|;jP(n?&7T?KRuFF?;7X~(VT0LQ9GoL0#dM(V#(1)+QSTiIm^ z!bs&L+{4!Ho=ctBenF!fE`&-XO99Sl`E+i~I+uXP8#JATe(S~WeshGHu>mr*X^95W+1LYS%s z^JspxiXM}^Jc34ecE%#6P*$f26Z(#tbRH=rL8qvP=7q1V_{Ys4 z>Jyv3#QA07)Y1YpHWb`Eq|u-ib{H%G7emv|0T8+r#Cwa2#mm3 z%9*y($@~*!KZHo%{=swgV&$kEp4U~6(fNdKj~rcRzRaZ#pO|+X6-s#I*g^XmzJ*8` zPvQ0O)cBXbk)5lXbNNr(T+a4khK^E-SrCz@9EnzQM#znOsy>L0c%ACRw{-NtVJ|bs zPJ6?mE}EI7eoypsX*RkL(Ftu^Wl3WD18#IgdP)y45{5_ql#-I3PU`B8!%CEjITd~+ zl5c!iE3DH*v#P{xa&@06mS#KWs>8lvWnRCBf!b3oBy??UM?F|Ia>-9+?N^3S7PgEh zOr`o+b=7lNMHge0wML7o&3y?6?#e%&Zcs0Kz}TN~u3N!GFmo!^Qw0E5mK&tTS}03o zW#Pzv?d(EJ4{wj#<`qu0D$P9S3z-$E6FJHfi_+e5!r+8=dbxjp_Pyhmz1+3z$xj6< z;X()whCCoMY;`st=&`iA+1f6S(X~<^(uzs|XMMxjtBchH;lc>bz5&ZUR#A&E0*$}X zUR@@jZp;*9nf@c7%?_}_fXCp6rG#WINxpGC?05Fcvb<~3$QDg+Wb)Ci?39C{#!pH^ zN;y0J>dmR%eS_x@eI)~N|1B~p!ZifNQ)Gz-NXm&9YAY!`KfXrv_3r6?UV<6Jo?E?* zj_M^ev}^u$gb$g{X?$z-C0MHMdtKp=?5|DyO?%tTdT=vZI`!-ANZ)nmH>}h<^xjB^ zD5r%UfgniEN+cbf2iaV1z5UcAe9UCE7^*9uC=Av&xsie~K21r$@ROQn$LX3>yt6@3 z-?luO+0kypC7~?H%j)mf+sxYR`1%Z&UCEbzN~&Jcemk_U_YeJ^YphmM5-bcXC*;`2 z!0q=F%+6JeE!ZbV{M+@%J?4UM2Zb`f?3jUR%&NVuZVd5|QA~Gd)0D9)HOw3Gf2|K6 z=L&So|J8@7|LQ~R|7H_mV(Iih+COMW+aLeanP00TOpOZtGYTZ}p-m7%G+kJ419f{- z1O*tBB#zS1R*{&F|Mva6jU^!$Jh1VmfNwMB_a_i`hn|j@G2Z;?yGBK15lGuMYX^?)3!v)k z7*~G5``CDYsj#;88*eAllO}KMNRuXS{K#F8N6-g2$WcjXGyj*5?UxYE-^c95c&4Yv zybtIFJGF{@YlA<`wqgMdKeb@?N(rLr+S1=nrGyH?n5!8BpYm}qlS58xGyWal{jWj= zZ&$3ZhB~sQ{-qQI)7S2$6GYNcoIwJ1okeSOsq|Qok#q$X`G~l*ZBCFJeq$t)^%8S* zUWx*tGHOQ&kn{B(jko!2yUMEa3*y;dwmuy*Od_OOJaS!M%eLxJxV8I+m%f>%36W+n z-S|DySz|DG%`iH;1rbIWWjDVmb!PIeMUI$g`J&)r?jzO2R!rs9A6P8*TwwB0)7f^U zddXtt*xLq4GYZB^7iu=M-`nJydKII;b||L>tR#HE`9(?DB$zD#DTVSdsHD#K zF0s4N{c^Fm9N|8V$F(H6+;-4sMBFx8@KBWq7A>NUkr)cv(pVZcLA2pEok?s5;2l-8 z{aoUv&n2)_f28Eo=$odRO%@Vmn{^jsZ6~Z%ma2dNV}(HNQ-0#O7a01}`39-?AXU3z z__248hgm%}gsEGdr013aYcH=f;HHUV*3%|^LQ=fY?eqo6f3;lZ#$R@lj-WLM6_s6X zN~w>}>4VS@WixoY7UOi%?Ml&Jtszx9-ui z7r(8|*Q$0IyJC;Dm|o7_Lcp9lj3Jh|^HJBbX;)0;HIjc{#8C>3Fg}I?`yXX2Z^0$f zJ&MR6Qz>1z#zDq$MT7;%D{(_2Me^tAq^Wj8ss7SH{v`U*A)b~>GCe@+d$nUGq=LYKRdQ<6EwLPAo{TEI3 zps}w$43oSLGrAU4 zh@B5d@f6A#GcE$rK`L&jvHVLLKhCC?z}b!oqy)mlgNwi2$%X$OeK}277Q!HNy$sas zG!h(n9W&~d(7Bnl$f&*q+Lj<{^wmX2ufA>z;)pMJeMLM{>(5(wVUOmIA~SeC*Y(h2 zf?}cNA!Wt%g1QQOLFZuv-Ap(9srbJ-`^tbgwrp!0g1fuBJHg#8xVyVsa0mo?WkIt{p?H<*WL84J&F{=Giccl~STliaLszty99o15(G5PE z++Azu9V0LHESJ^Uo1dffoiz(jw1b}d3_Zk$2G1FKu;kFRQ5LzB1?dQ70W;UA^S30q$F0yre2<)Q`qI!S8Ng-4a2DwvXKgj^ z`%IXMXwcgTV|0#;I&ZNb*7mhhWK&kA!`Bo>3avWrwT53Asmo)*j?vK7B9fJu${xIF zkX@v3x(~H|bog*TUu<2`d`8qCg=|PhNJwZKw7=J;Sl^qxCqn0o$I9mQ>B#r!aTQ0& zNT)spUQxrknw_odCJY>%ca@&fv;J1e{DfDx%C1dY#VnfXs~_%uMVz}!0#D?+DfZHs zZ%a~|R*a+a9S3F;K37$6v}|S|M?C?wl2R_h4xpS!uxi5pql|D_%?OgmOK3AivJf+W zWQQ-|3gH%-oDikAw_i@K8*k7!y#S{0_B-XqAQ!+a7Ku#rn>Rjk;inO< z12Gr*34LXOXHEl%hf}rMw5lVE#7yF3`B}M<^Y${6Vs$4SJzVCOT+1ZmmEtDb$qmnHU zx@k1EU>LR8B+x+^bTNsd@o_ZR`G)z(f-FuLD8{A}n2b9f@`0$JZ(GW>!_FMhgSW$O z^2)W3i(RFqVb{CbS4+1_x+);G!PoEraGvi|cLVAx=co*4-%vxM+*AniKXG1fz7pkm z)PVRfqW+*(Ys~EYqlsm5ypEC49&Kjf{6qSjce_6Eam77k+wrktj0gM z9|F5u6?VtK!Dwb?Nio*=Y`VNHmt$ZMmt2EkATdze4fAwu>2v6Jvef|nYq1f84)A7? ze?p^Ftj$XKH$#h;6jgsZduC%<7uA1UC#}BPlLV$e*Ade5oAVOx?NIEG$_w3M7k1N4 zLl8vjN}j&0;edl5FOILECElB66W;kTSPBLOuDiJ8h3#3F`q=H9Grauv(we73jH&{{^Nf1z z2$I`vwP^LnFm=cu!>0Kk(Ao)06p&p_XZgl5Ft9h1t zLokf^l)BEt3c#uzWvtH1$x6T1E5nz)jj~h^b<&gqJI4?Q1;QRIptkl3BlggmBkyZc zgdNc?4rfu(Y2lq0+5oBKz};a~m#V?Km;< zo69($)5|7!Io|LpcB)g4Rv<`S#I%#W74zQGxMnP+gK&r~p(b0fj_I%{GKwIt{EHxW zWNGFFWTTvGcazQ2hdwWpZxD0FTB(Omx5{#7SBI>Y6$xi~ep z7RQ`I^0#_kQJeS>V$30Af+BLGq7)^PhNLV~yl4%gEP8{R3e=aogAicGg>ky-qm;q8 z8HJx;LBSoFzO)uw(G2-yJI_)?yCGM~EUk5NSkpa4q6-fd)>f0q;Kq~D-OkKnt*;U& zB^IQnsrv*+2F~^Q9h#~(lP)K<>Y1q1hEpjd@IO}w43jA3p~2ufVh0YOVrOj@&fcbS z_N|vAUMe{aqNR`l_>^3=3h+&R61ahZ%dM{dmRCE_4l%HHyQ#OpPNhKjO-9BHP!Csl{MNXF7yBbaTt^{24S#_+9Pf zacFB=WF~8BEH26xTbpJhvOYBwYDL>T?Yb9j!nrKgWJha@3f@)0fF5zFc~+t&&t=Sim1Y zG+A-YE&iA^DtluButnd&d11TPZ?fEQkrfu~VDi8u%JaCudsTQ$UOQ%Th#w{(`|S>> zYOwcopI!zuYzFx-+a8cmOS#*YHar=y3kPM@3@B3nt@%Rmj7ZmZ>)s9K+m#-dzey$& z+@o9kXO~lwa+GG>$HS}9wlT{cKz(QYwI`n=e0GvM$XB=TByW2vJ{fcvM&cA!2fJ(O zX`{(+6~3s1GCFW!xh*zZlvZ(Swql^=TXj0m>vmZ8p0epPJu~atuo>j)V^z6eRx^1? zJ~owBNV=BaXm(r9UnJ!0Aez_uH6G?0waj4)<2o0+#7Q%D%O#6!Nx;pVix^ntqGKsg z;`%zn8RgjqW}HJ!_KDIcUOby2p$0=ONF@=GAVRShB92dqB{(wFjIe1W#n{Cnipp^3 zC{nJ-GqI<5*5UiNa`j$ULesYiL_HJ<5o;U)gp^qWwOs!AL;->QnL$!Ioc^*7*NKwc z-R-Agc_L|nv0Ndhka*VM_u&2q*i-P=?>b4ipJP`koyUs3dKplFi}3eWV>OE{()ac? ze$Y>L;VtZ|ov2NDj9ahi^~MFF-BYdqL_s(tp;{efiRh+i*I$4RkPkz{qQLDvY|ZoI z_f|=g5=rw0y|kp$q_)xN9Khp=L3g1hW|rbUN!uoMh5;)msNY#l=*oh)HQ&@G@O&XQ zjvGtaD90$WYv@&o2uXAd60Kc2uwB&%A+%g>%WOGD2Rt8Y%Nxd^+vQ@USsFBpHu38*XT$c--T=Ot})?8x1<+N}*M|%*L4hG2mS@@MB zGsQDaZrQp}D^}=Rz$8n5Zd)Iok=mhU2o_$%WZ$LLk3hweFt(0yO5P`+@p0vfQP|Yo zE{{m{GiXCZA>Ty^zBDP(2V`=xiPx;9_(`tWFfys2hJS9VA*9eH|r%HWR1@`bbRm*YmDzS zfXHW}!`%#r*5~#$UqOM~VnE`PfO#~aa@1}*B-{WVxYge^@Igj~sO1U#VQ?ji0A9=^TA!pLA+3WoH9iFI2Gkt8f8Tp>&a^ig>~o1!lxywG;W$QO9_Bh zz22ge`6EObNdnW-$95nMlp1F`S=~wm8gNs@u}!8h{_QU++W-(hcl3Syo8dsw9}WCD zJGX(-Rt)iur(cAzQNtcUT#JJrSQIVhi!r25&~si2C@;P9S@fl08fMtkEz5@pk*)hn zEwNWJ)vQ-wI;x990R5Op&#u!|=yY5-NRF=WN4+|{FRdrth;J@%J_>kGG1f1-S;<4j zH7Hk&&_yn-el9M~$Ja#Oo8mI_TG+sSydipE1=o`t@RvYIfQX$lS?QfB)hDC4TMh*_ zzfmVAh5!*UV+fjrzqUo#ERXnkQX|Z;p4onPU?txawYIgLNM{vcD6))*To@BoQL-W{ z52@jiO#+I|zAWDdTUfbj+NQWVL*|)>e#tkDlu8+0oP?yr9eyKa+;-*=rJSRvPaQuQ zVo<-QUK=l9Z(qX|1q)9D#Kx<$p zn~BKg5chs3F|zbWt*&SdO;V0qWM4ZqLXa@)$9+L{MR2Z4wWxS9e45X*Jho6v3+^mJ zjCB{wK=&F=Ms?6-R|h3*cH#w7%cE&D27T)b=r{3Z6F5}`k9`IX&ala@T1H5I<}WH1 zv#Vc7D1iM4^>`Mxm_6cn(DQ&P#YgJt8xcWKaMGsWIjGE?$bS^|oqrySqJha$w|~n$ z+>r~3&&dW+wN$}wxJV$phl4-iT=w8zj38fwRaaC5VE9V#Ca#%EBYBwn*TARFVZ}8E zr5GbEiw%&B+z!{Ydk0OH%VGn?K&(DRGS!7&#D|tODb=z|KYVHie2Pu*hovQQ18jB2 z_!MT#OMcb_w<+X%zHZ|=H*mVleSgrsp>C!9wzEi_JVK=2$U2S}vJ&aEH+pm_$C_Aq z9leK&d4?s<=9oc;IYpv+(V$44?N$#a;qw5#zD43b72^EIvxSe1U-u-4lZOYMx3=~5 zql9uNj&##d>Ak>+1pUq%kZdp_Y)n_T3Q}%ql~_jPwD?s-00tMxkRoZ?f6& zAhcNtqHoKxt0Jt1_xT|yB|LsO3j0=TFO~3{WS%4rZM4~i(f39uTFjy215oc?-xcuA zw$r?5@_J<1G)i3sp|(vvC77wWMP581m{6}@Y8yXoLzxDHOFD9m;Dl9vF%-4fth1w4 zFihk%SgAf%^Qb39^V`m{1x?G*Vrp@07gK?u7ZdHAMIRz~A*jS8((~RawIuEvcx8zh zV2BZ}YpRV#wv{;NZkO@BUh&DUv5*cNV@9Fwb8H1M-Chqr+(;0)K$|UlFqvKN?(LA4 zZ=wK|_mRbl)6_L41c2dZVg~J@Ww4$*)deHc0|5t&1!SR3T8c>R%05F`c}RwZiQrX32TL0mqEHES#5GJC>R8n12GodVcv>%;Z-P6{X?qZxt3V zJLt05ch5SRzu`SE+W2L0e`AXUtwbOFFqm0yo76MX`^vH#H<^W!dabxyr0C0}nQzo1 zo1k8)Hw(<*7>fs+;a(C z)+_YTL1%7|gl*))x>>@$K%n;ML8xt(t;e+VtlQ-AH2g8KO|m-T&I(-84cHC+bJEkY zp-)m0+UQCEdtZl@g4n_3O^vWW<%=naXhMpD<);IUm&e2tt&xl5P7x*@{iDxuBVtpg`x?`o!7*QkEvgi6@h}VZznEpPN^h!dNZU3;$`ioiA+AZ zy-T@;W(BN^>aYpo*GS^e*xC6!{iE8hpOjm?V7ll70sXQJ0g|$0nTafq+o8pj`lVWU zW*h3K55?D1lRuwv$#Ou#51@X`0+=&ZZC7nia2{6TH-1_g!$?_aD-fKJrGt!Y_z=l| z-Zgp2&-)<;M=4mP7+Y3eA}u(8Cozf1X}zGDOD<4PPSwwbfgpQW)F6(7ordIO!=;?U zI>=`~_X;PbK!8v6{Go^Tb#Q>Q$FmrZHBV&k%qM0Yx|FPN5{wlKZDD)U(X{Xf&}O95 zg1G1a5;EQ^UeY5McE)CD4Q6Xty!6)v{Sdb~CFEs_6y99fz3-Xc@)K%QA%rHA6fyFIpOu;WR%sbp;1ltA^=ZM{w@wK4MlCBJ3 zVvlXwHh_|jz+t6v3@$R4Zx`ko8c5A?5UqGYms}oo#85(Hk}q33U=n8-Hio#{)Pj}v zB5P4QsN|2`#m(_$8RTtU7;!XTX_<6i^Q^%xYRclS-KHd`W_e&6bzCY)bxX|8PQ7{y z6{x<;_SP}5&_+@tjzgW!&@l(|EEH@F1I#pP)`Mv&O?MZ;^l8MB%X2`@eOI5sP&AzJ z@F|BE#3&jlJVaX6LLMfkDdft)3`rA)U6S43-yjeytm{!`zLvj=!np`Rz8WC0y@P;j z-~H&TnIGHk1^+~4z?{h>aC6~_H?rKh(1?ZctUaEy=pjt8CjLTMIDk7u)~$VpJ~$p$ zy^NnI)Qc3g@L>O8wl$aCsnUheM+ixH8D!f|$5Gul9KFwMCcjk^u z0sZG9>Mr7eT{OO0)A;0+t%#)W(7nLX;i-B)q8l3F>?%50IdT)Vv9Ps_Lg@yIq6}Q4i8q8cD7^6h|~ht)s&~Q*2*Bt z9ehGQohkMcOQ-x2p22%jer?{MIK>5}eqY=V3vp<7i@G(_w*;iG_$A~pH}_~tnOrM* zA~MsLPCZ_}tgKF(!(iP7Y|yQ%!+3d`#6Oj*`iIq&<)L}V-VEyE7Yyli1T5)(*0)XN zgG0ktn3*uSi1uX1Pwb58orI6|iiKT*HI)bMyYntGz9WiOA9-p=f7H%GX$K`X*UkP& z)8hr z!7Dsg$agW5(EOPjACqoJMJOiMU%rDo^B9`O)E^9{%zmG+gfW03D}a8d%(kI>fq{Li zFi_Hs@hF=vAj{(0IkKt&Cumj5M32+aOt7OyB7rUWY(}t*lRhiClhuU=+GmTg%FhLE z*4#6nXmAW;-I7#ZEal1%Yaf}LF;CJ(nY_3!bVqud9GB)xCi7Lti@99~z5#A<$F4@WOH%ysRQAYd z(?rjSwd`W6FEdb0j%eO}-NREmm3tRv82VVop~o_ zXf`YQ2AN^5 zxlj^+oKw8DmoILUzT-BLg7=^ zX7xu4skHvKgEH9i>-dtmM(YWywAe->=icMRcb7efZT1LSW(Z)8SeRBnP!BG{CDV_g zId?fM7R7~@1a)eHpf_H~u`d!MDR%AV(xtu*%Ci#YJ%VdYHFnzmHS58{L( zP%!yjn}tP6pqk`;@d=WAK%x4e*bC)DyA9bBkbE#X)?Mf7MTYcF+RSqEFpoH2 zU5EPxQR(*CBt`2qrfFYLc%wFJ5WgG?TI>W`p7N#IfRHWirY~1Q`A$C8>A{b>#N&S| z)!xUSF)3L*wmTle|Fp)yel;X{%jtZAuHZpxLbgi!$$cmh=XQzhd$-U1q8TeVcS@^iqMgA8Q-5(B#|&lpRYd55L$y!`OO)4{-N0mDjWmi#l3bfMQ_ zZv`k!o3!opx}`}iuMec2yUa;wD4gE1i;J z=;B>&^lX+BKX$J$m-u&CTH|~wvW10{305T9?CwB}U}mF%jA9FlP3+2d1*l6rOoR`O z$EuWWS*TMqOqK_&OzpvjxKi(9RdvQ9QUtQC?yFOC)aV7T5OuT((;_3G0LIl1FgFmzlChwjQ-)E-6c(|7ZXG45lo()!X@V z1NHakkAu-~<%8dq*T1Iny(z0I(T~3hM9olnu|w32#V5RVq|Ur;vk=QJj{H{Hx0rsq z!(k!Cv@(sHK1!E8W$$P|a|m+MT1Yk%kYpGH;>ix$p99G$Js-ByHZWsTSnt%Bbr!_f z^=Rs2K}kX%*d2^d;DD&6v?>;d5VI$#53CVOU?hzuNnkcM25O>bq@oGw@3E7~n2*sD zOqV#(Dc}#)K(;tF4SrQxla(Jygl?Bk4r1{>05yHcnc)6U~+Z4J*^oMV$w-il~n z%s_*VRdz9kmB9_Pk|TZn4k|UypV}QHd+SDgU9O8WP>VPb8FA!xs^b)>Q`ZU|v|=Wz zmbGcJ+l@N?om(xL@g}lBilsuWCs6dX>FTR-mcdY_L&IPv@xTA5|IBX|OMUNSd=+Tf zD1fPp?QzZU8|Ro0Xsa9;#KKxJfxNgm zPy{J)){=E+@?5yjFKD^XbhwS)73&|=!UDee2&nU^N@4&9Cp6c5sC?AYXR~eGBhl$} zvc>7#gLy?R@|W2GKA4Uhp$RAFdD-kBKJlGQ#87l)WhCl10ch63KS_Auf-*GRdz}K} zxDO)A*i;=SoR8+&bmVJK*7+T3UF?)+($2XPie`uDhoGcy998K+w^bCJPXgZgm^`;I)4m!J` zV%AxLm%0MP3hWeNgEaE3Hg3fbB)_%Gm1+neW*h5Rk47PrX(5IhQQi9rFS*ApdPa0W zERP(G1-!~Hqa=LBKl5^gi=%l{3E?(k!KU&fWxUDGAxf@7EWoe3D27pawt}@29VK76 zf2svBdDvHcv(whhBW|KBrXCoTP)x3c19epRW)iY+3}C6i4%QN@TqRdHrC7mXp&v_| zo6p3#3y;Ht4^Fb(+~Cyqm=)TI&`Y}&Y2rzaD)d9&0B7CEMo6MM(6&00)h94i7_|>s zNCIM{&Gn9<%dM5MpRuut7s7dhfes+*x>POsQMp@a8(`06AWsK`>U z3o88dQ>)~HASp}u$Z(wto^r#BG*`z`X>muugem2t*DA~W8OO{Gvg{S*yM@FkEYpa& zLNGxtn85sFp#^-e;6Y#zA#`X#HsH!(Wz~gi_8x=)Q+g+$IKJ)AN4z}DXEeKvmpKnP z#w%Pr53x}YS)jMt}?h|KNuClhwc6_&m z!s$d^MQe-9E-pY=>VieVvm{9WyzkOfzU`BGXZ{f8N*?$;9hW1pZ5FnTn^_CnzX3w0L!JyK$I8ssHl{Gi_>BXQ zN+UoWa#8YfjSEJap}8A)pC5_Uqe@tJHo6S%fRVeDKPizq2czW?P1oXQPZFs0FIFJV zcau)!$BG1#QrYbpTY!DGr^aWKWr^7r|Eh%G8T+^O5g}% z7G(>%o7_44Lak8|vt!(@PK#Pr80_<%3$5|fi3Q&Yb>2DZHOWTdz8n^u8w}`K*n?b} z#VgHTQ~|tgMZjztxc|ben!B3Hy@tl7<-ASug^v!`9V>l{d@`E5zGH-!h`F0po-@OT zbSx*b`YdNi3>jm8erav=2v+Yq&o?*x14E;`@R70Nkyp_7R~TZL>X^eft$gI$Q6m10 zBgFW>93pACR_h*m=+zsP05Xq@*)a!W8)lwu6w+ONn2s{A z7otz1Y+d83XtxMrhppv@4=3^s&6cVcAaFdD?o#g~88KH}Dye~Q(pXAt4Eh1#y)Fd; zD*Pd*ruGp^3GXv=S44_(W5{m}PojV7gi)g{+A;rT9)B;u zts;*-D3@u1}fR+YCgbn?e+4M|AYq!_vrPE2dd*8cKjB_zd0IjVO*9$+# zokA)eE{gEmB^k8}-1#CiRepw9kr$=K+d2fdw;ehPOI@X}5gGk>8E&U+OW2 zlehWLhCA=B^UY3&nvR}75P#qid0^?5b>QDt`0$A(qrj;tX^L8St`U%BtkIOA@{c(C zx4DBORnOvEC-a*p!<(Vwf6b4xk-dYNjrFe~_53kb*~t|O01)I4hXs&7Y#elS{`{Vf z4n8+N89glrEdv>(oFpI+F7Uqz-oE&+7bpPwpP%6YPWb=$_{Y`e%|!wf0GNaz`acQ& z6Do-_AmYt;@@<@d7*GBa>TNIpdA_{=1@+EL;(tQ@;}1Xp=&@6X%@1Qnwu1ryz+(O< zYSMq9{vignwXy%FYsBxNUOYy)g1^Z_^xvKlVc(54>#Y>}kH2kTV`Z!Nf5BAznHw`=1>$$3_Pz7TufzQM9`OGocHcXm{EB_{9{X?oPyVL~`Ssgf_pcOaPJgBNSNij>4F306 z-fO~sWzqEdE6Y2D*zXa%7oPk|pz8Zqg1^X6evjn+y5FxPJV}2gdA|zydkpVagnnfR zO8G0p`}LvUV|ZVp`;|c@`>zb|3U$MY z^Y4v(LYn$-(z~8?)}OHT>e+4Uz5N83IE@B nhxh5yulPh2f8zg|JpFf|pl>+<0073@o9OM%?f3Rm0D%7o(m9di literal 0 HcmV?d00001 diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index d22a7f4c3b167..580cb512d8025 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -202,16 +202,25 @@ class GaussianMixtureModel(JavaModelWrapper, JavaSaveable, JavaLoader): >>> clusterdata_1 = sc.parallelize(array([-0.1,-0.05,-0.01,-0.1, ... 0.9,0.8,0.75,0.935, - ... -0.83,-0.68,-0.91,-0.76 ]).reshape(6, 2)) + ... -0.83,-0.68,-0.91,-0.76 ]).reshape(6, 2), 2) >>> model = GaussianMixture.train(clusterdata_1, 3, convergenceTol=0.0001, ... maxIterations=50, seed=10) >>> labels = model.predict(clusterdata_1).collect() >>> labels[0]==labels[1] False >>> labels[1]==labels[2] - True + False >>> labels[4]==labels[5] True + >>> model.predict([-0.1,-0.05]) + 0 + >>> softPredicted = model.predictSoft([-0.1,-0.05]) + >>> abs(softPredicted[0] - 1.0) < 0.001 + True + >>> abs(softPredicted[1] - 0.0) < 0.001 + True + >>> abs(softPredicted[2] - 0.0) < 0.001 + True >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -277,26 +286,27 @@ def k(self): @since('1.3.0') def predict(self, x): """ - Find the cluster to which the points in 'x' has maximum membership - in this model. + Find the cluster to which the point 'x' or each point in RDD 'x' + has maximum membership in this model. - :param x: RDD of data points. - :return: cluster_labels. RDD of cluster labels. + :param x: vector or RDD of vector represents data points. + :return: cluster label or RDD of cluster labels. """ if isinstance(x, RDD): cluster_labels = self.predictSoft(x).map(lambda z: z.index(max(z))) return cluster_labels else: - raise TypeError("x should be represented by an RDD, " - "but got %s." % type(x)) + z = self.predictSoft(x) + return z.argmax() @since('1.3.0') def predictSoft(self, x): """ - Find the membership of each point in 'x' to all mixture components. + Find the membership of point 'x' or each point in RDD 'x' to all mixture components. - :param x: RDD of data points. - :return: membership_matrix. RDD of array of double values. + :param x: vector or RDD of vector represents data points. + :return: the membership value to all mixture components for vector 'x' + or each vector in RDD 'x'. """ if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) @@ -304,8 +314,7 @@ def predictSoft(self, x): _convert_to_vector(self.weights), means, sigmas) return membership_matrix.map(lambda x: pyarray.array('d', x)) else: - raise TypeError("x should be represented by an RDD, " - "but got %s." % type(x)) + return self.call("predictSoft", _convert_to_vector(x)).toArray() @classmethod @since('1.5.0') diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 99331297c19f0..26cafca8b8381 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -76,4 +76,6 @@ # which allows us to execute the user's PYTHONSTARTUP file: _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP') if _pythonstartup and os.path.isfile(_pythonstartup): - execfile(_pythonstartup) + with open(_pythonstartup) as f: + code = compile(f.read(), _pythonstartup, 'exec') + exec(code) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 900def59d23a5..320451c52c706 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -368,12 +368,12 @@ def when(self, condition, value): >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 4, 1).when(df.age < 3, -1).otherwise(0)).show() - +-----+--------------------------------------------------------+ - | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0| - +-----+--------------------------------------------------------+ - |Alice| -1| - | Bob| 1| - +-----+--------------------------------------------------------+ + +-----+------------------------------------------------------------+ + | name|CASE WHEN (age > 4) THEN 1 WHEN (age < 3) THEN -1 ELSE 0 END| + +-----+------------------------------------------------------------+ + |Alice| -1| + | Bob| 1| + +-----+------------------------------------------------------------+ """ if not isinstance(condition, Column): raise TypeError("condition should be a Column") @@ -393,12 +393,12 @@ def otherwise(self, value): >>> from pyspark.sql import functions as F >>> df.select(df.name, F.when(df.age > 3, 1).otherwise(0)).show() - +-----+---------------------------------+ - | name|CASE WHEN (age > 3) THEN 1 ELSE 0| - +-----+---------------------------------+ - |Alice| 0| - | Bob| 1| - +-----+---------------------------------+ + +-----+-------------------------------------+ + | name|CASE WHEN (age > 3) THEN 1 ELSE 0 END| + +-----+-------------------------------------+ + |Alice| 0| + | Bob| 1| + +-----+-------------------------------------+ """ v = value._jc if isinstance(value, Column) else value jc = self._jc.otherwise(v) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index a7bc288e38861..90a6b5d9c0dda 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -403,10 +403,10 @@ def repartition(self, numPartitions, *cols): +---+-----+ |age| name| +---+-----+ - | 2|Alice| - | 2|Alice| | 5| Bob| | 5| Bob| + | 2|Alice| + | 2|Alice| +---+-----+ >>> data = data.repartition(7, "age") >>> data.show() @@ -552,7 +552,7 @@ def alias(self, alias): >>> df_as2 = df.alias("df_as2") >>> joined_df = df_as1.join(df_as2, col("df_as1.name") == col("df_as2.name"), 'inner') >>> joined_df.select(col("df_as1.name"), col("df_as2.name"), col("df_as2.age")).collect() - [Row(name=u'Alice', name=u'Alice', age=2), Row(name=u'Bob', name=u'Bob', age=5)] + [Row(name=u'Bob', name=u'Bob', age=5), Row(name=u'Alice', name=u'Alice', age=2)] """ assert isinstance(alias, basestring), "alias should be a string" return DataFrame(getattr(self._jdf, "as")(alias), self.sql_ctx) @@ -573,14 +573,14 @@ def join(self, other, on=None, how=None): One of `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] >>> df.join(df2, 'name', 'outer').select('name', 'height').collect() - [Row(name=u'Tom', height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] + [Row(name=u'Tom', height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] >>> cond = [df.name == df3.name, df.age == df3.age] >>> df.join(df3, cond, 'outer').select(df.name, df3.age).collect() - [Row(name=u'Bob', age=5), Row(name=u'Alice', age=2)] + [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df.join(df2, 'name').select(df.name, df2.height).collect() [Row(name=u'Bob', height=85)] @@ -880,9 +880,9 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(avg(age)=3.5)] - >>> df.groupBy('name').agg({'age': 'mean'}).collect() + >>> sorted(df.groupBy('name').agg({'age': 'mean'}).collect()) [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] - >>> df.groupBy(df.name).avg().collect() + >>> sorted(df.groupBy(df.name).avg().collect()) [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)] >>> df.groupBy(['name', df.age]).count().collect() [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)] @@ -901,11 +901,11 @@ def rollup(self, *cols): +-----+----+-----+ | name| age|count| +-----+----+-----+ - |Alice|null| 1| + |Alice| 2| 1| | Bob| 5| 1| | Bob|null| 1| | null|null| 2| - |Alice| 2| 1| + |Alice|null| 1| +-----+----+-----+ """ jgd = self._jdf.rollup(self._jcols(*cols)) @@ -923,12 +923,12 @@ def cube(self, *cols): | name| age|count| +-----+----+-----+ | null| 2| 1| - |Alice|null| 1| + |Alice| 2| 1| | Bob| 5| 1| - | Bob|null| 1| | null| 5| 1| + | Bob|null| 1| | null|null| 2| - |Alice| 2| 1| + |Alice|null| 1| +-----+----+-----+ """ jgd = self._jdf.cube(self._jcols(*cols)) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index b0390cb9942e6..719eca8f5559e 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1023,7 +1023,7 @@ def hash(*cols): """Calculates the hash code of given columns, and returns the result as a int column. >>> sqlContext.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect() - [Row(hash=1358996357)] + [Row(hash=-757602832)] """ sc = SparkContext._active_spark_context jc = sc._jvm.functions.hash(_to_seq(sc, cols, _to_java_column)) diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py index 9ca303a974cd4..ee734cb439287 100644 --- a/python/pyspark/sql/group.py +++ b/python/pyspark/sql/group.py @@ -74,11 +74,11 @@ def agg(self, *exprs): or a list of :class:`Column`. >>> gdf = df.groupBy(df.name) - >>> gdf.agg({"*": "count"}).collect() + >>> sorted(gdf.agg({"*": "count"}).collect()) [Row(name=u'Alice', count(1)=1), Row(name=u'Bob', count(1)=1)] >>> from pyspark.sql import functions as F - >>> gdf.agg(F.min(df.age)).collect() + >>> sorted(gdf.agg(F.min(df.age)).collect()) [Row(name=u'Alice', min(age)=2), Row(name=u'Bob', min(age)=5)] """ assert exprs, "exprs should not be empty" @@ -96,7 +96,7 @@ def agg(self, *exprs): def count(self): """Counts the number of records for each group. - >>> df.groupBy(df.age).count().collect() + >>> sorted(df.groupBy(df.age).count().collect()) [Row(age=2, count=1), Row(age=5, count=1)] """ diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 0f1f005ce3edf..ec3ad9933cf60 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -19,7 +19,6 @@ import os import sys -from threading import RLock, Timer from py4j.java_gateway import java_import, JavaObject @@ -33,63 +32,6 @@ __all__ = ["StreamingContext"] -class Py4jCallbackConnectionCleaner(object): - - """ - A cleaner to clean up callback connections that are not closed by Py4j. See SPARK-12617. - It will scan all callback connections every 30 seconds and close the dead connections. - """ - - def __init__(self, gateway): - self._gateway = gateway - self._stopped = False - self._timer = None - self._lock = RLock() - - def start(self): - if self._stopped: - return - - def clean_closed_connections(): - from py4j.java_gateway import quiet_close, quiet_shutdown - - callback_server = self._gateway._callback_server - if callback_server: - with callback_server.lock: - try: - closed_connections = [] - for connection in callback_server.connections: - if not connection.isAlive(): - quiet_close(connection.input) - quiet_shutdown(connection.socket) - quiet_close(connection.socket) - closed_connections.append(connection) - - for closed_connection in closed_connections: - callback_server.connections.remove(closed_connection) - except Exception: - import traceback - traceback.print_exc() - - self._start_timer(clean_closed_connections) - - self._start_timer(clean_closed_connections) - - def _start_timer(self, f): - with self._lock: - if not self._stopped: - self._timer = Timer(30.0, f) - self._timer.daemon = True - self._timer.start() - - def stop(self): - with self._lock: - self._stopped = True - if self._timer: - self._timer.cancel() - self._timer = None - - class StreamingContext(object): """ Main entry point for Spark Streaming functionality. A StreamingContext @@ -105,9 +47,6 @@ class StreamingContext(object): # Reference to a currently active StreamingContext _activeContext = None - # A cleaner to clean leak sockets of callback server every 30 seconds - _py4j_cleaner = None - def __init__(self, sparkContext, batchDuration=None, jssc=None): """ Create a new StreamingContext. @@ -155,34 +94,12 @@ def _ensure_initialized(cls): # get the GatewayServer object in JVM by ID jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client) # update the port of CallbackClient with real port - gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port) - _py4j_cleaner = Py4jCallbackConnectionCleaner(gw) - _py4j_cleaner.start() + jgws.resetCallbackClient(jgws.getCallbackClient().getAddress(), gw._python_proxy_port) # register serializer for TransformFunction # it happens before creating SparkContext when loading from checkpointing - if cls._transformerSerializer is None: - transformer_serializer = TransformFunctionSerializer() - transformer_serializer.init( - SparkContext._active_spark_context, CloudPickleSerializer(), gw) - # SPARK-12511 streaming driver with checkpointing unable to finalize leading to OOM - # There is an issue that Py4J's PythonProxyHandler.finalize blocks forever. - # (https://github.com/bartdag/py4j/pull/184) - # - # Py4j will create a PythonProxyHandler in Java for "transformer_serializer" when - # calling "registerSerializer". If we call "registerSerializer" twice, the second - # PythonProxyHandler will override the first one, then the first one will be GCed and - # trigger "PythonProxyHandler.finalize". To avoid that, we should not call - # "registerSerializer" more than once, so that "PythonProxyHandler" in Java side won't - # be GCed. - # - # TODO Once Py4J fixes this issue, we should upgrade Py4j to the latest version. - transformer_serializer.gateway.jvm.PythonDStream.registerSerializer( - transformer_serializer) - cls._transformerSerializer = transformer_serializer - else: - cls._transformerSerializer.init( - SparkContext._active_spark_context, CloudPickleSerializer(), gw) + cls._transformerSerializer = TransformFunctionSerializer( + SparkContext._active_spark_context, CloudPickleSerializer(), gw) @classmethod def getOrCreate(cls, checkpointPath, setupFunc): diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index e617fc9ce9eec..abbbf6eb9394f 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -89,10 +89,11 @@ class TransformFunctionSerializer(object): it uses this class to invoke Python, which returns the serialized function as a byte array. """ - def init(self, ctx, serializer, gateway=None): + def __init__(self, ctx, serializer, gateway=None): self.ctx = ctx self.serializer = serializer self.gateway = gateway or self.ctx._gateway + self.gateway.jvm.PythonDStream.registerSerializer(self) self.failure = None def dumps(self, id): diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala index 44650f25f7a18..bb3081d12938e 100644 --- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -30,11 +30,7 @@ object Main extends Logging { val conf = new SparkConf() val rootDir = conf.getOption("spark.repl.classdir").getOrElse(Utils.getLocalDir(conf)) val outputDir = Utils.createTempDir(root = rootDir, namePrefix = "repl") - val s = new Settings() - s.processArguments(List("-Yrepl-class-based", - "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", - "-classpath", getAddedJars.mkString(File.pathSeparator)), true) - // the creation of SecurityManager has to be lazy so SPARK_YARN_MODE is set if needed + var sparkContext: SparkContext = _ var sqlContext: SQLContext = _ var interp = new SparkILoop // this is a public var because tests reset it. diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index d8d9d00d64ebc..0c37985a670b2 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -27,4 +27,4 @@ fi export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9-src.zip:${PYTHONPATH}" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.9.1-src.zip:${PYTHONPATH}" diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 9714c46fe99a0..967a482ba4f9b 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -187,6 +187,22 @@ This file is divided into 3 sections: scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + java,scala,3rdParty,spark + javax?\..* + scala\..* + (?!org\.apache\.spark\.).* + org\.apache\.spark\..* + + + + + + COMMA + + + @@ -207,24 +223,6 @@ This file is divided into 3 sections: - - - - java,scala,3rdParty,spark - javax?\..* - scala\..* - (?!org\.apache\.spark\.).* - org\.apache\.spark\..* - - - - - - - COLON, COMMA - - - diff --git a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g index 2d2bafb1ee34f..f18b6ec496f8f 100644 --- a/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g +++ b/sql/catalyst/src/main/antlr3/org/apache/spark/sql/catalyst/parser/SelectClauseParser.g @@ -131,6 +131,13 @@ selectItem : (tableAllColumns) => tableAllColumns -> ^(TOK_SELEXPR tableAllColumns) | + namedExpression + ; + +namedExpression +@init { gParent.pushMsg("select named expression", state); } +@after { gParent.popMsg(state); } + : ( expression ((KW_AS? identifier) | (KW_AS LPAREN identifier (COMMA identifier)* RPAREN))? ) -> ^(TOK_SELEXPR expression identifier*) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index b8d3c49100476..1a351933a366c 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -566,10 +566,6 @@ public int hashCode() { return Murmur3_x86_32.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, 42); } - public int hashCode(int seed) { - return Murmur3_x86_32.hashUnsafeWords(baseObject, baseOffset, sizeInBytes, seed); - } - @Override public boolean equals(Object other) { if (other instanceof UnsafeRow) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala index 1eda4a9a97644..d0fbdacf6eafd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystQl.scala @@ -22,14 +22,20 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Count +import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.catalyst.parser._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.random.RandomSampler +private[sql] object CatalystQl { + val parser = new CatalystQl + def parseExpression(sql: String): Expression = parser.parseExpression(sql) + def parseTableIdentifier(sql: String): TableIdentifier = parser.parseTableIdentifier(sql) +} + /** * This class translates a HQL String to a Catalyst [[LogicalPlan]] or [[Expression]]. */ @@ -41,16 +47,13 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) { } } - /** - * Returns the AST for the given SQL string. + * The safeParse method allows a user to focus on the parsing/AST transformation logic. This + * method will take care of possible errors during the parsing process. */ - protected def getAst(sql: String): ASTNode = ParseDriver.parse(sql, conf) - - /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String): LogicalPlan = { + protected def safeParse[T](sql: String, ast: ASTNode)(toResult: ASTNode => T): T = { try { - createPlan(sql, ParseDriver.parse(sql, conf)) + toResult(ast) } catch { case e: MatchError => throw e case e: AnalysisException => throw e @@ -58,26 +61,39 @@ private[sql] class CatalystQl(val conf: ParserConf = SimpleParserConf()) { throw new AnalysisException(e.getMessage) case e: NotImplementedError => throw new AnalysisException( - s""" - |Unsupported language features in query: $sql - |${getAst(sql).treeString} + s"""Unsupported language features in query + |== SQL == + |$sql + |== AST == + |${ast.treeString} + |== Error == |$e + |== Stacktrace == |${e.getStackTrace.head} """.stripMargin) } } - protected def createPlan(sql: String, tree: ASTNode): LogicalPlan = nodeToPlan(tree) - - def parseDdl(ddl: String): Seq[Attribute] = { - val tree = getAst(ddl) - assert(tree.text == "TOK_CREATETABLE", "Only CREATE TABLE supported.") - val tableOps = tree.children - val colList = tableOps - .find(_.text == "TOK_TABCOLLIST") - .getOrElse(sys.error("No columnList!")) - - colList.children.map(nodeToAttribute) + /** Creates LogicalPlan for a given SQL string. */ + def parsePlan(sql: String): LogicalPlan = + safeParse(sql, ParseDriver.parsePlan(sql, conf))(nodeToPlan) + + /** Creates Expression for a given SQL string. */ + def parseExpression(sql: String): Expression = + safeParse(sql, ParseDriver.parseExpression(sql, conf))(selExprNodeToExpr(_).get) + + /** Creates TableIdentifier for a given SQL string. */ + def parseTableIdentifier(sql: String): TableIdentifier = + safeParse(sql, ParseDriver.parseTableName(sql, conf))(extractTableIdent) + + def parseDdl(sql: String): Seq[Attribute] = { + safeParse(sql, ParseDriver.parseExpression(sql, conf)) { ast => + val Token("TOK_CREATETABLE", children) = ast + children + .find(_.text == "TOK_TABCOLLIST") + .getOrElse(sys.error("No columnList!")) + .flatMap(_.children.map(nodeToAttribute)) + } } protected def getClauses( @@ -187,7 +203,6 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C val keyMap = keyASTs.zipWithIndex.toMap val bitmasks: Seq[Int] = setASTs.map { - case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0 case Token("TOK_GROUPING_SETS_EXPRESSION", columns) => columns.foldLeft(0)((bitmap, col) => { val keyIndex = keyMap.find(_._1.treeEquals(col)).map(_._2) @@ -737,7 +752,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C /* Case statements */ case Token("TOK_FUNCTION", Token(WHEN(), Nil) :: branches) => - CaseWhen(branches.map(nodeToExpr)) + CaseWhen.createFromParser(branches.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(CASE(), Nil) :: branches) => val keyExpr = nodeToExpr(branches.head) CaseKeyWhen(keyExpr, branches.drop(1).map(nodeToExpr)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 2a132d8b82bef..85ff4ea0c946b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -203,7 +203,7 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { ) protected lazy val ordering: Parser[Seq[SortOrder]] = - ( rep1sep(expression ~ direction.? , ",") ^^ { + ( rep1sep(expression ~ direction.?, ",") ^^ { case exps => exps.map(pair => SortOrder(pair._1, pair._2.getOrElse(Ascending))) } ) @@ -305,7 +305,8 @@ object SqlParser extends AbstractSparkSQLParser with DataTypeParser { throw new AnalysisException(s"invalid function approximate($s) $udfName") } } - | CASE ~> whenThenElse ^^ CaseWhen + | CASE ~> whenThenElse ^^ + { case branches => CaseWhen.createFromParser(branches) } | CASE ~> expression ~ whenThenElse ^^ { case keyPart ~ branches => CaseKeyWhen(keyPart, branches) } ) 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 70fdd0340aea2..dadea6b54a946 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 @@ -1214,6 +1214,8 @@ object CleanupAliases extends Rule[LogicalPlan] { Window(projectList, cleanedWindowExprs, partitionSpec.map(trimAliases), orderSpec.map(trimAliases(_).asInstanceOf[SortOrder]), child) + // Operators that operate on objects should only have expressions from encoders, which should + // never have extra aliases. case o: ObjectOperator => o case other => @@ -1249,26 +1251,22 @@ object ResolveUpCast extends Rule[LogicalPlan] { } def apply(plan: LogicalPlan): LogicalPlan = { - plan transform { - case o: ObjectOperator => o - case other => - other transformExpressions { - case u@UpCast(child, _, _) if !child.resolved => u - - case UpCast(child, dataType, walkedTypePath) => (child.dataType, dataType) match { - case (from: NumericType, to: DecimalType) if !to.isWiderThan(from) => - fail(child, to, walkedTypePath) - case (from: DecimalType, to: NumericType) if !from.isTighterThan(to) => - fail(child, to, walkedTypePath) - case (from, to) if illegalNumericPrecedence(from, to) => - fail(child, to, walkedTypePath) - case (TimestampType, DateType) => - fail(child, DateType, walkedTypePath) - case (StringType, to: NumericType) => - fail(child, to, walkedTypePath) - case _ => Cast(child, dataType) - } - } + plan transformAllExpressions { + case u @ UpCast(child, _, _) if !child.resolved => u + + case UpCast(child, dataType, walkedTypePath) => (child.dataType, dataType) match { + case (from: NumericType, to: DecimalType) if !to.isWiderThan(from) => + fail(child, to, walkedTypePath) + case (from: DecimalType, to: NumericType) if !from.isTighterThan(to) => + fail(child, to, walkedTypePath) + case (from, to) if illegalNumericPrecedence(from, to) => + fail(child, to, walkedTypePath) + case (TimestampType, DateType) => + fail(child, DateType, walkedTypePath) + case (StringType, to: NumericType) => + fail(child, to, walkedTypePath) + case _ => Cast(child, dataType) + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 5c2aa3c06b3e7..d9009e3848e58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -182,6 +182,8 @@ object FunctionRegistry { expression[Average]("avg"), expression[Corr]("corr"), expression[Count]("count"), + expression[CovPopulation]("covar_pop"), + expression[CovSample]("covar_samp"), expression[First]("first"), expression[First]("first_value"), expression[Last]("last"), 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 dbcbd6854b474..2737fe32cd086 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 @@ -482,27 +482,6 @@ object HiveTypeCoercion { private val trueValues = Seq(1.toByte, 1.toShort, 1, 1L, Decimal.ONE) private val falseValues = Seq(0.toByte, 0.toShort, 0, 0L, Decimal.ZERO) - private def buildCaseKeyWhen(booleanExpr: Expression, numericExpr: Expression) = { - CaseKeyWhen(numericExpr, Seq( - Literal(trueValues.head), booleanExpr, - Literal(falseValues.head), Not(booleanExpr), - Literal(false))) - } - - private def transform(booleanExpr: Expression, numericExpr: Expression) = { - If(Or(IsNull(booleanExpr), IsNull(numericExpr)), - Literal.create(null, BooleanType), - buildCaseKeyWhen(booleanExpr, numericExpr)) - } - - private def transformNullSafe(booleanExpr: Expression, numericExpr: Expression) = { - CaseWhen(Seq( - And(IsNull(booleanExpr), IsNull(numericExpr)), Literal(true), - Or(IsNull(booleanExpr), IsNull(numericExpr)), Literal(false), - buildCaseKeyWhen(booleanExpr, numericExpr) - )) - } - def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e @@ -511,6 +490,7 @@ object HiveTypeCoercion { // all other cases are considered as false. // We may simplify the expression if one side is literal numeric values + // TODO: Maybe these rules should go into the optimizer. case EqualTo(bool @ BooleanType(), Literal(value, _: NumericType)) if trueValues.contains(value) => bool case EqualTo(bool @ BooleanType(), Literal(value, _: NumericType)) @@ -529,13 +509,13 @@ object HiveTypeCoercion { if falseValues.contains(value) => And(IsNotNull(bool), Not(bool)) case EqualTo(left @ BooleanType(), right @ NumericType()) => - transform(left , right) + EqualTo(Cast(left, right.dataType), right) case EqualTo(left @ NumericType(), right @ BooleanType()) => - transform(right, left) + EqualTo(left, Cast(right, left.dataType)) case EqualNullSafe(left @ BooleanType(), right @ NumericType()) => - transformNullSafe(left, right) + EqualNullSafe(Cast(left, right.dataType), right) case EqualNullSafe(left @ NumericType(), right @ BooleanType()) => - transformNullSafe(right, left) + EqualNullSafe(left, Cast(right, left.dataType)) } } @@ -638,33 +618,27 @@ object HiveTypeCoercion { */ object CaseWhenCoercion extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { - case c: CaseWhenLike if c.childrenResolved && !c.valueTypesEqual => - logDebug(s"Input values for null casting ${c.valueTypes.mkString(",")}") + case c: CaseWhen if c.childrenResolved && !c.valueTypesEqual => val maybeCommonType = findWiderCommonType(c.valueTypes) maybeCommonType.map { commonType => - val castedBranches = c.branches.grouped(2).map { - case Seq(when, value) if value.dataType != commonType => - Seq(when, Cast(value, commonType)) - case Seq(elseVal) if elseVal.dataType != commonType => - Seq(Cast(elseVal, commonType)) - case other => other - }.reduce(_ ++ _) - c match { - case _: CaseWhen => CaseWhen(castedBranches) - case CaseKeyWhen(key, _) => CaseKeyWhen(key, castedBranches) + var changed = false + val newBranches = c.branches.map { case (condition, value) => + if (value.dataType.sameType(commonType)) { + (condition, value) + } else { + changed = true + (condition, Cast(value, commonType)) + } } - }.getOrElse(c) - - case c: CaseKeyWhen if c.childrenResolved && !c.resolved => - val maybeCommonType = - findWiderCommonType((c.key +: c.whenList).map(_.dataType)) - maybeCommonType.map { commonType => - val castedBranches = c.branches.grouped(2).map { - case Seq(whenExpr, thenExpr) if whenExpr.dataType != commonType => - Seq(Cast(whenExpr, commonType), thenExpr) - case other => other - }.reduce(_ ++ _) - CaseKeyWhen(Cast(c.key, commonType), castedBranches) + val newElseValue = c.elseValue.map { value => + if (value.dataType.sameType(commonType)) { + value + } else { + changed = true + Cast(value, commonType) + } + } + if (changed) CaseWhen(newBranches, newElseValue) else c }.getOrElse(c) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index fc0e87aa68ed4..79eebbf9b1ec4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -160,6 +160,7 @@ abstract class Star extends LeafExpression with NamedExpression { override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override def qualifiers: Seq[String] = throw new UnresolvedException(this, "qualifiers") override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") override lazy val resolved = false def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] @@ -246,6 +247,8 @@ case class MultiAlias(child: Expression, names: Seq[String]) override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") + override lazy val resolved = false override def toString: String = s"$child AS $names" @@ -259,6 +262,7 @@ case class MultiAlias(child: Expression, names: Seq[String]) * @param expressions Expressions to expand. */ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Unevaluable { + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = expressions override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") } @@ -298,6 +302,7 @@ case class UnresolvedAlias(child: Expression, aliasName: Option[String] = None) override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def name: String = throw new UnresolvedException(this, "name") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") override lazy val resolved = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 05f746e72b498..64832dc114e67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -207,6 +207,16 @@ case class ExpressionEncoder[T]( resolve(attrs, OuterScopes.outerScopes).bind(attrs) } + + /** + * Returns a new set (with unique ids) of [[NamedExpression]] that represent the serialized form + * of this object. + */ + def namedExpressions: Seq[NamedExpression] = schema.map(_.name).zip(toRowExpressions).map { + case (_, ne: NamedExpression) => ne.newInstance() + case (name, e) => Alias(e, name)() + } + /** * Returns an encoded version of `t` as a Spark SQL row. Note that multiple calls to * toRow are allowed to return the same actual [[InternalRow]] object. Thus, the caller should diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 7293d5d4472af..c94b2c0e270b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types._ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) extends LeafExpression with NamedExpression { - override def toString: String = s"input[$ordinal, $dataType]" + override def toString: String = s"input[$ordinal, ${dataType.simpleString}]" // Use special getter for primitive types (for UnsafeRow) override def eval(input: InternalRow): Any = { @@ -66,6 +66,8 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) override def exprId: ExprId = throw new UnsupportedOperationException + override def newInstance(): NamedExpression = this + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { val javaType = ctx.javaType(dataType) val value = ctx.getValue(ctx.INPUT_ROW, dataType, ordinal.toString) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala new file mode 100644 index 0000000000000..f53b01be2a0d5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.types._ + +/** + * Compute the covariance between two expressions. + * When applied on empty data (i.e., count is zero), it returns NULL. + * + */ +abstract class Covariance(left: Expression, right: Expression) extends ImperativeAggregate + with Serializable { + override def children: Seq[Expression] = Seq(left, right) + + override def nullable: Boolean = true + + override def dataType: DataType = DoubleType + + override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType) + + override def checkInputDataTypes(): TypeCheckResult = { + if (left.dataType.isInstanceOf[DoubleType] && right.dataType.isInstanceOf[DoubleType]) { + TypeCheckResult.TypeCheckSuccess + } else { + TypeCheckResult.TypeCheckFailure( + s"covariance requires that both arguments are double type, " + + s"not (${left.dataType}, ${right.dataType}).") + } + } + + override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + + override def inputAggBufferAttributes: Seq[AttributeReference] = { + aggBufferAttributes.map(_.newInstance()) + } + + override val aggBufferAttributes: Seq[AttributeReference] = Seq( + AttributeReference("xAvg", DoubleType)(), + AttributeReference("yAvg", DoubleType)(), + AttributeReference("Ck", DoubleType)(), + AttributeReference("count", LongType)()) + + // Local cache of mutableAggBufferOffset(s) that will be used in update and merge + val xAvgOffset = mutableAggBufferOffset + val yAvgOffset = mutableAggBufferOffset + 1 + val CkOffset = mutableAggBufferOffset + 2 + val countOffset = mutableAggBufferOffset + 3 + + // Local cache of inputAggBufferOffset(s) that will be used in update and merge + val inputXAvgOffset = inputAggBufferOffset + val inputYAvgOffset = inputAggBufferOffset + 1 + val inputCkOffset = inputAggBufferOffset + 2 + val inputCountOffset = inputAggBufferOffset + 3 + + override def initialize(buffer: MutableRow): Unit = { + buffer.setDouble(xAvgOffset, 0.0) + buffer.setDouble(yAvgOffset, 0.0) + buffer.setDouble(CkOffset, 0.0) + buffer.setLong(countOffset, 0L) + } + + override def update(buffer: MutableRow, input: InternalRow): Unit = { + val leftEval = left.eval(input) + val rightEval = right.eval(input) + + if (leftEval != null && rightEval != null) { + val x = leftEval.asInstanceOf[Double] + val y = rightEval.asInstanceOf[Double] + + var xAvg = buffer.getDouble(xAvgOffset) + var yAvg = buffer.getDouble(yAvgOffset) + var Ck = buffer.getDouble(CkOffset) + var count = buffer.getLong(countOffset) + + val deltaX = x - xAvg + val deltaY = y - yAvg + count += 1 + xAvg += deltaX / count + yAvg += deltaY / count + Ck += deltaX * (y - yAvg) + + buffer.setDouble(xAvgOffset, xAvg) + buffer.setDouble(yAvgOffset, yAvg) + buffer.setDouble(CkOffset, Ck) + buffer.setLong(countOffset, count) + } + } + + // Merge counters from other partitions. Formula can be found at: + // http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance + override def merge(buffer1: MutableRow, buffer2: InternalRow): Unit = { + val count2 = buffer2.getLong(inputCountOffset) + + // We only go to merge two buffers if there is at least one record aggregated in buffer2. + // We don't need to check count in buffer1 because if count2 is more than zero, totalCount + // is more than zero too, then we won't get a divide by zero exception. + if (count2 > 0) { + var xAvg = buffer1.getDouble(xAvgOffset) + var yAvg = buffer1.getDouble(yAvgOffset) + var Ck = buffer1.getDouble(CkOffset) + var count = buffer1.getLong(countOffset) + + val xAvg2 = buffer2.getDouble(inputXAvgOffset) + val yAvg2 = buffer2.getDouble(inputYAvgOffset) + val Ck2 = buffer2.getDouble(inputCkOffset) + + val totalCount = count + count2 + val deltaX = xAvg - xAvg2 + val deltaY = yAvg - yAvg2 + Ck += Ck2 + deltaX * deltaY * count / totalCount * count2 + xAvg = (xAvg * count + xAvg2 * count2) / totalCount + yAvg = (yAvg * count + yAvg2 * count2) / totalCount + count = totalCount + + buffer1.setDouble(xAvgOffset, xAvg) + buffer1.setDouble(yAvgOffset, yAvg) + buffer1.setDouble(CkOffset, Ck) + buffer1.setLong(countOffset, count) + } + } +} + +case class CovSample( + left: Expression, + right: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends Covariance(left, right) { + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def eval(buffer: InternalRow): Any = { + val count = buffer.getLong(countOffset) + if (count > 1) { + val Ck = buffer.getDouble(CkOffset) + val cov = Ck / (count - 1) + if (cov.isNaN) { + null + } else { + cov + } + } else { + null + } + } +} + +case class CovPopulation( + left: Expression, + right: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends Covariance(left, right) { + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = + copy(inputAggBufferOffset = newInputAggBufferOffset) + + override def eval(buffer: InternalRow): Any = { + val count = buffer.getLong(countOffset) + if (count > 0) { + val Ck = buffer.getDouble(CkOffset) + if (Ck.isNaN) { + null + } else { + Ck / count + } + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 19da849d2bec9..83abbcdc61175 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{sequenceOption, TypeUtils} +import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @@ -45,7 +45,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def dataType: DataType = trueValue.dataType override def eval(input: InternalRow): Any = { - if (true == predicate.eval(input)) { + if (java.lang.Boolean.TRUE.equals(predicate.eval(input))) { trueValue.eval(input) } else { falseValue.eval(input) @@ -78,274 +78,164 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def sql: String = s"(IF(${predicate.sql}, ${trueValue.sql}, ${falseValue.sql}))" } -trait CaseWhenLike extends Expression { - - // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last - // element is the value for the default catch-all case (if provided). - // Hence, `branches` consists of at least two elements, and can have an odd or even length. - def branches: Seq[Expression] +/** + * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". + * When a = true, returns b; when c = true, returns d; else returns e. + * + * @param branches seq of (branch condition, branch value) + * @param elseValue optional value for the else branch + */ +case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[Expression] = None) + extends Expression { - @transient lazy val whenList = - branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq - @transient lazy val thenList = - branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq - val elseValue = if (branches.length % 2 == 0) None else Option(branches.last) + override def children: Seq[Expression] = branches.flatMap(b => b._1 :: b._2 :: Nil) ++ elseValue // both then and else expressions should be considered. - def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType) + def valueTypes: Seq[DataType] = branches.map(_._2.dataType) ++ elseValue.map(_.dataType) + def valueTypesEqual: Boolean = valueTypes.size <= 1 || valueTypes.sliding(2, 1).forall { case Seq(dt1, dt2) => dt1.sameType(dt2) } - override def checkInputDataTypes(): TypeCheckResult = { - if (valueTypesEqual) { - checkTypesInternal() - } else { - TypeCheckResult.TypeCheckFailure( - "THEN and ELSE expressions should all be same type or coercible to a common type") - } - } - - protected def checkTypesInternal(): TypeCheckResult - - override def dataType: DataType = thenList.head.dataType + override def dataType: DataType = branches.head._2.dataType override def nullable: Boolean = { - // If no value is nullable and no elseValue is provided, the whole statement defaults to null. - thenList.exists(_.nullable) || elseValue.map(_.nullable).getOrElse(true) + // Result is nullable if any of the branch is nullable, or if the else value is nullable + branches.exists(_._2.nullable) || elseValue.map(_.nullable).getOrElse(true) } -} - -// scalastyle:off -/** - * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - */ -// scalastyle:on -case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray - override def children: Seq[Expression] = branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if (whenList.forall(_.dataType == BooleanType)) { - TypeCheckResult.TypeCheckSuccess + override def checkInputDataTypes(): TypeCheckResult = { + // Make sure all branch conditions are boolean types. + if (valueTypesEqual) { + if (branches.forall(_._1.dataType == BooleanType)) { + TypeCheckResult.TypeCheckSuccess + } else { + val index = branches.indexWhere(_._1.dataType != BooleanType) + TypeCheckResult.TypeCheckFailure( + s"WHEN expressions in CaseWhen should all be boolean type, " + + s"but the ${index + 1}th when expression's type is ${branches(index)._1}") + } } else { - val index = whenList.indexWhere(_.dataType != BooleanType) TypeCheckResult.TypeCheckFailure( - s"WHEN expressions in CaseWhen should all be boolean type, " + - s"but the ${index + 1}th when expression's type is ${whenList(index)}") + "THEN and ELSE expressions should all be same type or coercible to a common type") } } - /** Written in imperative fashion for performance considerations. */ override def eval(input: InternalRow): Any = { - val len = branchesArr.length var i = 0 - // If all branches fail and an elseVal is not provided, the whole statement - // defaults to null, according to Hive's semantics. - while (i < len - 1) { - if (branchesArr(i).eval(input) == true) { - return branchesArr(i + 1).eval(input) + while (i < branches.size) { + if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { + return branches(i)._2.eval(input) } - i += 2 + i += 1 } - var res: Any = null - if (i == len - 1) { - res = branchesArr(i).eval(input) + if (elseValue.isDefined) { + return elseValue.get.eval(input) + } else { + return null } - return res } override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val len = branchesArr.length - val got = ctx.freshName("got") - - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) + // Generate code that looks like: + // + // condA = ... + // if (condA) { + // valueA + // } else { + // condB = ... + // if (condB) { + // valueB + // } else { + // condC = ... + // if (condC) { + // valueC + // } else { + // elseValue + // } + // } + // } + val cases = branches.map { case (condExpr, valueExpr) => + val cond = condExpr.gen(ctx) + val res = valueExpr.gen(ctx) s""" - if (!$got) { - ${cond.code} - if (!${cond.isNull} && ${cond.value}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } + ${cond.code} + if (!${cond.isNull} && ${cond.value}) { + ${res.code} + ${ev.isNull} = ${res.isNull}; + ${ev.value} = ${res.value}; } """ - }.mkString("\n") + } - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" - if (!$got) { + var generatedCode = cases.mkString("", "\nelse {\n", "\nelse {\n") + + elseValue.foreach { elseExpr => + val res = elseExpr.gen(ctx) + generatedCode += + s""" ${res.code} ${ev.isNull} = ${res.isNull}; ${ev.value} = ${res.value}; - } - """ - } else { - "" + """ } + generatedCode += "}\n" * cases.size + s""" - boolean $got = false; boolean ${ev.isNull} = true; ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - $cases - $other + $generatedCode """ } override def toString: String = { - "CASE" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString + val cases = branches.map { case (c, v) => s" WHEN $c THEN $v" }.mkString + val elseCase = elseValue.map(" ELSE " + _).getOrElse("") + "CASE" + cases + elseCase + " END" } override def sql: String = { - val branchesSQL = branches.map(_.sql) - val (cases, maybeElse) = if (branches.length % 2 == 0) { - (branchesSQL, None) - } else { - (branchesSQL.init, Some(branchesSQL.last)) - } - - val head = s"CASE " - val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" - val body = cases.grouped(2).map { - case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" - }.mkString(" ") - - head + body + tail + val cases = branches.map { case (c, v) => s" WHEN ${c.sql} THEN ${v.sql}" }.mkString + val elseCase = elseValue.map(" ELSE " + _.sql).getOrElse("") + "CASE" + cases + elseCase + " END" } } -// scalastyle:off -/** - * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". - * Refer to this link for the corresponding semantics: - * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions - */ -// scalastyle:on -case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike { - - // Use private[this] Array to speed up evaluation. - @transient private[this] lazy val branchesArr = branches.toArray +/** Factory methods for CaseWhen. */ +object CaseWhen { - override def children: Seq[Expression] = key +: branches - - override protected def checkTypesInternal(): TypeCheckResult = { - if ((key +: whenList).map(_.dataType).distinct.size > 1) { - TypeCheckResult.TypeCheckFailure( - "key and WHEN expressions should all be same type or coercible to a common type") - } else { - TypeCheckResult.TypeCheckSuccess - } + def apply(branches: Seq[(Expression, Expression)], elseValue: Expression): CaseWhen = { + CaseWhen(branches, Option(elseValue)) } - private def evalElse(input: InternalRow): Any = { - if (branchesArr.length % 2 == 0) { - null - } else { - branchesArr(branchesArr.length - 1).eval(input) - } - } - - /** Written in imperative fashion for performance considerations. */ - override def eval(input: InternalRow): Any = { - val evaluatedKey = key.eval(input) - // If key is null, we can just return the else part or null if there is no else. - // If key is not null but doesn't match any when part, we need to return - // the else part or null if there is no else, according to Hive's semantics. - if (evaluatedKey != null) { - val len = branchesArr.length - var i = 0 - while (i < len - 1) { - if (evaluatedKey == branchesArr(i).eval(input)) { - return branchesArr(i + 1).eval(input) - } - i += 2 - } - } - evalElse(input) + /** + * A factory method to faciliate the creation of this expression when used in parsers. + * @param branches Expressions at even position are the branch conditions, and expressions at odd + * position are branch values. + */ + def createFromParser(branches: Seq[Expression]): CaseWhen = { + val cases = branches.grouped(2).flatMap { + case cond :: value :: Nil => Some((cond, value)) + case value :: Nil => None + }.toArray.toSeq // force materialization to make the seq serializable + val elseValue = if (branches.size % 2 == 1) Some(branches.last) else None + CaseWhen(cases, elseValue) } +} - override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val keyEval = key.gen(ctx) - val len = branchesArr.length - val got = ctx.freshName("got") - - val cases = (0 until len/2).map { i => - val cond = branchesArr(i * 2).gen(ctx) - val res = branchesArr(i * 2 + 1).gen(ctx) - s""" - if (!$got) { - ${cond.code} - if (!${cond.isNull} && ${ctx.genEqual(key.dataType, keyEval.value, cond.value)}) { - $got = true; - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } - } - """ - }.mkString("\n") - - val other = if (len % 2 == 1) { - val res = branchesArr(len - 1).gen(ctx) - s""" - if (!$got) { - ${res.code} - ${ev.isNull} = ${res.isNull}; - ${ev.value} = ${res.value}; - } - """ - } else { - "" - } - - s""" - boolean $got = false; - boolean ${ev.isNull} = true; - ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - ${keyEval.code} - if (!${keyEval.isNull}) { - $cases - } - $other - """ - } - - override def toString: String = { - s"CASE $key" + branches.sliding(2, 2).map { - case Seq(cond, value) => s" WHEN $cond THEN $value" - case Seq(elseValue) => s" ELSE $elseValue" - }.mkString - } - - override def sql: String = { - val keySQL = key.sql - val branchesSQL = branches.map(_.sql) - val (cases, maybeElse) = if (branches.length % 2 == 0) { - (branchesSQL, None) - } else { - (branchesSQL.init, Some(branchesSQL.last)) - } - - val head = s"CASE $keySQL " - val tail = maybeElse.map(e => s" ELSE $e").getOrElse("") + " END" - val body = cases.grouped(2).map { - case Seq(whenExpr, thenExpr) => s"WHEN $whenExpr THEN $thenExpr" - }.mkString(" ") - - head + body + tail +/** + * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END". + * When a = b, returns c; when a = d, returns e; else returns f. + */ +object CaseKeyWhen { + def apply(key: Expression, branches: Seq[Expression]): CaseWhen = { + val cases = branches.grouped(2).flatMap { + case cond :: value :: Nil => Some((EqualTo(key, cond), value)) + case value :: Nil => None + }.toArray.toSeq // force materialization to make the seq serializable + val elseValue = if (branches.size % 2 == 1) Some(branches.last) else None + CaseWhen(cases, elseValue) } } @@ -389,7 +279,7 @@ case class Least(children: Seq[Expression]) extends Expression { val evalChildren = children.map(_.gen(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) - def updateEval(eval: GeneratedExpressionCode): String = + def updateEval(eval: GeneratedExpressionCode): String = { s""" ${eval.code} if (!${eval.isNull} && (${ev.isNull} || @@ -398,6 +288,7 @@ case class Least(children: Seq[Expression]) extends Expression { ${ev.value} = ${eval.value}; } """ + } s""" ${first.code} boolean ${ev.isNull} = ${first.isNull}; @@ -447,7 +338,7 @@ case class Greatest(children: Seq[Expression]) extends Expression { val evalChildren = children.map(_.gen(ctx)) val first = evalChildren(0) val rest = evalChildren.drop(1) - def updateEval(eval: GeneratedExpressionCode): String = + def updateEval(eval: GeneratedExpressionCode): String = { s""" ${eval.code} if (!${eval.isNull} && (${ev.isNull} || @@ -456,6 +347,7 @@ case class Greatest(children: Seq[Expression]) extends Expression { ${ev.value} = ${eval.value}; } """ + } s""" ${first.code} boolean ${ev.isNull} = ${first.isNull}; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 0eb915fdc1691..e0b020330278b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -21,13 +21,17 @@ import java.sql.{Date, Timestamp} import org.json4s.JsonAST._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ object Literal { + val TrueLiteral: Literal = Literal(true, BooleanType) + + val FalseLiteral: Literal = Literal(false, BooleanType) + def apply(v: Any): Literal = v match { case i: Int => Literal(i, IntegerType) case l: Long => Literal(l, LongType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala index cc406a39f0408..4751fbe4146fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala @@ -25,8 +25,11 @@ import org.apache.commons.codec.digest.DigestUtils import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.hash.Murmur3_x86_32 +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} +import org.apache.spark.unsafe.Platform /** * A function that calculates an MD5 128-bit checksum and returns it as a hex string @@ -184,8 +187,31 @@ case class Crc32(child: Expression) extends UnaryExpression with ImplicitCastInp * A function that calculates hash value for a group of expressions. Note that the `seed` argument * is not exposed to users and should only be set inside spark SQL. * - * Internally this function will write arguments into an [[UnsafeRow]], and calculate hash code of - * the unsafe row using murmur3 hasher with a seed. + * The hash value for an expression depends on its type and seed: + * - null: seed + * - boolean: turn boolean into int, 1 for true, 0 for false, and then use murmur3 to + * hash this int with seed. + * - byte, short, int: use murmur3 to hash the input as int with seed. + * - long: use murmur3 to hash the long input with seed. + * - float: turn it into int: java.lang.Float.floatToIntBits(input), and hash it. + * - double: turn it into long: java.lang.Double.doubleToLongBits(input), and hash it. + * - decimal: if it's a small decimal, i.e. precision <= 18, turn it into long and hash + * it. Else, turn it into bytes and hash it. + * - calendar interval: hash `microseconds` first, and use the result as seed to hash `months`. + * - binary: use murmur3 to hash the bytes with seed. + * - string: get the bytes of string and hash it. + * - array: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each element, and assign the element hash value + * to `result`. + * - map: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each key-value, and assign the key-value hash + * value to `result`. + * - struct: The `result` starts with seed, then use `result` as seed, recursively + * calculate hash value for each field, and assign the field hash value to + * `result`. + * + * Finally we aggregate the hash values for each expression by the same way of struct. + * * We should use this hash function for both shuffle and bucket, so that we can guarantee shuffle * and bucketing have same data distribution. */ @@ -206,22 +232,225 @@ case class Murmur3Hash(children: Seq[Expression], seed: Int) extends Expression } } - private lazy val unsafeProjection = UnsafeProjection.create(children) + override def prettyName: String = "hash" + + override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")}, $seed)" override def eval(input: InternalRow): Any = { - unsafeProjection(input).hashCode(seed) + var hash = seed + var i = 0 + val len = children.length + while (i < len) { + hash = computeHash(children(i).eval(input), children(i).dataType, hash) + i += 1 + } + hash } + private def computeHash(value: Any, dataType: DataType, seed: Int): Int = { + def hashInt(i: Int): Int = Murmur3_x86_32.hashInt(i, seed) + def hashLong(l: Long): Int = Murmur3_x86_32.hashLong(l, seed) + + value match { + case null => seed + case b: Boolean => hashInt(if (b) 1 else 0) + case b: Byte => hashInt(b) + case s: Short => hashInt(s) + case i: Int => hashInt(i) + case l: Long => hashLong(l) + case f: Float => hashInt(java.lang.Float.floatToIntBits(f)) + case d: Double => hashLong(java.lang.Double.doubleToLongBits(d)) + case d: Decimal => + val precision = dataType.asInstanceOf[DecimalType].precision + if (precision <= Decimal.MAX_LONG_DIGITS) { + hashLong(d.toUnscaledLong) + } else { + val bytes = d.toJavaBigDecimal.unscaledValue().toByteArray + Murmur3_x86_32.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, bytes.length, seed) + } + case c: CalendarInterval => Murmur3_x86_32.hashInt(c.months, hashLong(c.microseconds)) + case a: Array[Byte] => + Murmur3_x86_32.hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) + case s: UTF8String => + Murmur3_x86_32.hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) + + case array: ArrayData => + val elementType = dataType match { + case udt: UserDefinedType[_] => udt.sqlType.asInstanceOf[ArrayType].elementType + case ArrayType(et, _) => et + } + var result = seed + var i = 0 + while (i < array.numElements()) { + result = computeHash(array.get(i, elementType), elementType, result) + i += 1 + } + result + + case map: MapData => + val (kt, vt) = dataType match { + case udt: UserDefinedType[_] => + val mapType = udt.sqlType.asInstanceOf[MapType] + mapType.keyType -> mapType.valueType + case MapType(kt, vt, _) => kt -> vt + } + val keys = map.keyArray() + val values = map.valueArray() + var result = seed + var i = 0 + while (i < map.numElements()) { + result = computeHash(keys.get(i, kt), kt, result) + result = computeHash(values.get(i, vt), vt, result) + i += 1 + } + result + + case struct: InternalRow => + val types: Array[DataType] = dataType match { + case udt: UserDefinedType[_] => + udt.sqlType.asInstanceOf[StructType].map(_.dataType).toArray + case StructType(fields) => fields.map(_.dataType) + } + var result = seed + var i = 0 + val len = struct.numFields + while (i < len) { + result = computeHash(struct.get(i, types(i)), types(i), result) + i += 1 + } + result + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { - val unsafeRow = GenerateUnsafeProjection.createCode(ctx, children) ev.isNull = "false" + val childrenHash = children.zipWithIndex.map { + case (child, dt) => + val childGen = child.gen(ctx) + val childHash = computeHash(childGen.value, child.dataType, ev.value, ctx) + s""" + ${childGen.code} + if (!${childGen.isNull}) { + ${childHash.code} + ${ev.value} = ${childHash.value}; + } + """ + }.mkString("\n") s""" - ${unsafeRow.code} - final int ${ev.value} = ${unsafeRow.value}.hashCode($seed); + int ${ev.value} = $seed; + $childrenHash """ } - override def prettyName: String = "hash" - - override def sql: String = s"$prettyName(${children.map(_.sql).mkString(", ")}, $seed)" + private def computeHash( + input: String, + dataType: DataType, + seed: String, + ctx: CodeGenContext): GeneratedExpressionCode = { + val hasher = classOf[Murmur3_x86_32].getName + def hashInt(i: String): GeneratedExpressionCode = inlineValue(s"$hasher.hashInt($i, $seed)") + def hashLong(l: String): GeneratedExpressionCode = inlineValue(s"$hasher.hashLong($l, $seed)") + def inlineValue(v: String): GeneratedExpressionCode = + GeneratedExpressionCode(code = "", isNull = "false", value = v) + + dataType match { + case NullType => inlineValue(seed) + case BooleanType => hashInt(s"$input ? 1 : 0") + case ByteType | ShortType | IntegerType | DateType => hashInt(input) + case LongType | TimestampType => hashLong(input) + case FloatType => hashInt(s"Float.floatToIntBits($input)") + case DoubleType => hashLong(s"Double.doubleToLongBits($input)") + case d: DecimalType => + if (d.precision <= Decimal.MAX_LONG_DIGITS) { + hashLong(s"$input.toUnscaledLong()") + } else { + val bytes = ctx.freshName("bytes") + val code = s"byte[] $bytes = $input.toJavaBigDecimal().unscaledValue().toByteArray();" + val offset = "Platform.BYTE_ARRAY_OFFSET" + val result = s"$hasher.hashUnsafeBytes($bytes, $offset, $bytes.length, $seed)" + GeneratedExpressionCode(code, "false", result) + } + case CalendarIntervalType => + val microsecondsHash = s"$hasher.hashLong($input.microseconds, $seed)" + val monthsHash = s"$hasher.hashInt($input.months, $microsecondsHash)" + inlineValue(monthsHash) + case BinaryType => + val offset = "Platform.BYTE_ARRAY_OFFSET" + inlineValue(s"$hasher.hashUnsafeBytes($input, $offset, $input.length, $seed)") + case StringType => + val baseObject = s"$input.getBaseObject()" + val baseOffset = s"$input.getBaseOffset()" + val numBytes = s"$input.numBytes()" + inlineValue(s"$hasher.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $seed)") + + case ArrayType(et, _) => + val result = ctx.freshName("result") + val index = ctx.freshName("index") + val element = ctx.freshName("element") + val elementHash = computeHash(element, et, result, ctx) + val code = + s""" + int $result = $seed; + for (int $index = 0; $index < $input.numElements(); $index++) { + if (!$input.isNullAt($index)) { + final ${ctx.javaType(et)} $element = ${ctx.getValue(input, et, index)}; + ${elementHash.code} + $result = ${elementHash.value}; + } + } + """ + GeneratedExpressionCode(code, "false", result) + + case MapType(kt, vt, _) => + val result = ctx.freshName("result") + val index = ctx.freshName("index") + val keys = ctx.freshName("keys") + val values = ctx.freshName("values") + val key = ctx.freshName("key") + val value = ctx.freshName("value") + val keyHash = computeHash(key, kt, result, ctx) + val valueHash = computeHash(value, vt, result, ctx) + val code = + s""" + int $result = $seed; + final ArrayData $keys = $input.keyArray(); + final ArrayData $values = $input.valueArray(); + for (int $index = 0; $index < $input.numElements(); $index++) { + final ${ctx.javaType(kt)} $key = ${ctx.getValue(keys, kt, index)}; + ${keyHash.code} + $result = ${keyHash.value}; + if (!$values.isNullAt($index)) { + final ${ctx.javaType(vt)} $value = ${ctx.getValue(values, vt, index)}; + ${valueHash.code} + $result = ${valueHash.value}; + } + } + """ + GeneratedExpressionCode(code, "false", result) + + case StructType(fields) => + val result = ctx.freshName("result") + val fieldsHash = fields.map(_.dataType).zipWithIndex.map { + case (dt, index) => + val field = ctx.freshName("field") + val fieldHash = computeHash(field, dt, result, ctx) + s""" + if (!$input.isNullAt($index)) { + final ${ctx.javaType(dt)} $field = ${ctx.getValue(input, dt, index.toString)}; + ${fieldHash.code} + $result = ${fieldHash.value}; + } + """ + }.mkString("\n") + val code = + s""" + int $result = $seed; + $fieldsHash + """ + GeneratedExpressionCode(code, "false", result) + + case udt: UserDefinedType[_] => computeHash(input, udt.sqlType, seed, ctx) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index eee708cb02f9d..b6d7a7f5e8d01 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -79,6 +79,9 @@ trait NamedExpression extends Expression { /** Returns the metadata when an expression is a reference to another expression with metadata. */ def metadata: Metadata = Metadata.empty + /** Returns a copy of this expression with a new `exprId`. */ + def newInstance(): NamedExpression + protected def typeSuffix = if (resolved) { dataType match { @@ -144,6 +147,9 @@ case class Alias(child: Expression, name: String)( } } + def newInstance(): NamedExpression = + Alias(child, name)(qualifiers = qualifiers, explicitMetadata = explicitMetadata) + override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifiers) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index c0c3e6e891669..8385f7e1da591 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -172,6 +172,8 @@ case class Invoke( $objNullCheck """ } + + override def toString: String = s"$targetObject.$functionName" } object NewInstance { @@ -253,6 +255,8 @@ case class NewInstance( """ } } + + override def toString: String = s"newInstance($cls)" } /** 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 fb70a2a634d8e..cc3371c08fac4 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 @@ -21,6 +21,7 @@ import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, EliminateSubQueries} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, LeftSemi, RightOuter} @@ -62,10 +63,12 @@ abstract class Optimizer extends RuleExecutor[LogicalPlan] { ConstantFolding, LikeSimplification, BooleanSimplification, + SimplifyConditionals, RemoveDispensableExpressions, SimplifyFilters, SimplifyCasts, - SimplifyCaseConversionExpressions) :: + SimplifyCaseConversionExpressions, + EliminateSerialization) :: Batch("Decimal Optimizations", FixedPoint(100), DecimalAggregates) :: Batch("LocalRelation", FixedPoint(100), @@ -94,6 +97,19 @@ object SamplePushDown extends Rule[LogicalPlan] { } } +/** + * Removes cases where we are unnecessarily going between the object and serialized (InternalRow) + * representation of data item. For example back to back map operations. + */ +object EliminateSerialization extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case m @ MapPartitions(_, input, _, child: ObjectOperator) + if !input.isInstanceOf[Attribute] && m.input.dataType == child.outputObject.dataType => + val childWithoutSerialization = child.withObjectOutput + m.copy(input = childWithoutSerialization.output.head, child = childWithoutSerialization) + } +} + /** * Pushes certain operations to both sides of a Union, Intersect or Except operator. * Operations that are safe to pushdown are listed as follows. @@ -483,7 +499,6 @@ object NullPropagation extends Rule[LogicalPlan] { */ object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case o: ObjectOperator => o case q: LogicalPlan => q transformExpressionsDown { // Skip redundant folding of literals. This rule is technically not necessary. Placing this // here avoids running the next rule for Literal values, which would create a new Literal @@ -520,112 +535,106 @@ object OptimizeIn 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) => (left, right) match { - // true && r => r - case (Literal(true, BooleanType), r) => r - // l && true => l - case (l, Literal(true, BooleanType)) => l - // false && r => false - case (Literal(false, BooleanType), _) => Literal(false) - // l && false => false - case (_, Literal(false, BooleanType)) => Literal(false) - // a && a => a - case (l, r) if l fastEquals r => l - // a && (not(a) || b) => a && b - case (l, Or(l1, r)) if (Not(l) == l1) => And(l, r) - case (l, Or(r, l1)) if (Not(l) == l1) => And(l, r) - case (Or(l, l1), r) if (l1 == Not(r)) => And(l, r) - case (Or(l1, l), r) if (l1 == Not(r)) => And(l, r) - // (a || b) && (a || c) => a || (b && c) - case _ => - // 1. Split left and right to get the disjunctive predicates, - // i.e. lhs = (a, b), rhs = (a, c) - // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) - // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) - // 4. Apply the formula, get the optimized predicate: common || (ldiff && rdiff) - val lhs = splitDisjunctivePredicates(left) - val rhs = splitDisjunctivePredicates(right) - val common = lhs.filter(e => rhs.exists(e.semanticEquals(_))) - if (common.isEmpty) { - // No common factors, return the original predicate - and + case TrueLiteral And e => e + case e And TrueLiteral => e + case FalseLiteral Or e => e + case e Or FalseLiteral => e + + case FalseLiteral And _ => FalseLiteral + case _ And FalseLiteral => FalseLiteral + case TrueLiteral Or _ => TrueLiteral + case _ Or TrueLiteral => TrueLiteral + + case a And b if a.semanticEquals(b) => a + case a Or b if a.semanticEquals(b) => a + + case a And (b Or c) if Not(a).semanticEquals(b) => And(a, c) + case a And (b Or c) if Not(a).semanticEquals(c) => And(a, b) + case (a Or b) And c if a.semanticEquals(Not(c)) => And(b, c) + case (a Or b) And c if b.semanticEquals(Not(c)) => And(a, c) + + case a Or (b And c) if Not(a).semanticEquals(b) => Or(a, c) + case a Or (b And c) if Not(a).semanticEquals(c) => Or(a, b) + case (a And b) Or c if a.semanticEquals(Not(c)) => Or(b, c) + case (a And b) Or c if b.semanticEquals(Not(c)) => Or(a, c) + + // Common factor elimination for conjunction + case and @ (left And right) => + // 1. Split left and right to get the disjunctive predicates, + // i.e. lhs = (a, b), rhs = (a, c) + // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) + // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) + // 4. Apply the formula, get the optimized predicate: common || (ldiff && rdiff) + val lhs = splitDisjunctivePredicates(left) + val rhs = splitDisjunctivePredicates(right) + val common = lhs.filter(e => rhs.exists(e.semanticEquals)) + if (common.isEmpty) { + // No common factors, return the original predicate + and + } else { + val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals)) + val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals)) + if (ldiff.isEmpty || rdiff.isEmpty) { + // (a || b || c || ...) && (a || b) => (a || b) + common.reduce(Or) } else { - val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals(_))) - val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals(_))) - if (ldiff.isEmpty || rdiff.isEmpty) { - // (a || b || c || ...) && (a || b) => (a || b) - common.reduce(Or) - } else { - // (a || b || c || ...) && (a || b || d || ...) => - // ((c || ...) && (d || ...)) || a || b - (common :+ And(ldiff.reduce(Or), rdiff.reduce(Or))).reduce(Or) - } + // (a || b || c || ...) && (a || b || d || ...) => + // ((c || ...) && (d || ...)) || a || b + (common :+ And(ldiff.reduce(Or), rdiff.reduce(Or))).reduce(Or) } - } // end of And(left, right) - - case or @ Or(left, right) => (left, right) match { - // true || r => true - case (Literal(true, BooleanType), _) => Literal(true) - // r || true => true - case (_, Literal(true, BooleanType)) => Literal(true) - // false || r => r - case (Literal(false, BooleanType), r) => r - // l || false => l - case (l, Literal(false, BooleanType)) => l - // a || a => a - case (l, r) if l fastEquals r => l - // (a && b) || (a && c) => a && (b || c) - case _ => - // 1. Split left and right to get the conjunctive predicates, - // i.e. lhs = (a, b), rhs = (a, c) - // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) - // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) - // 4. Apply the formula, get the optimized predicate: common && (ldiff || rdiff) - val lhs = splitConjunctivePredicates(left) - val rhs = splitConjunctivePredicates(right) - val common = lhs.filter(e => rhs.exists(e.semanticEquals(_))) - if (common.isEmpty) { - // No common factors, return the original predicate - or + } + + // Common factor elimination for disjunction + case or @ (left Or right) => + // 1. Split left and right to get the conjunctive predicates, + // i.e. lhs = (a, b), rhs = (a, c) + // 2. Find the common predict between lhsSet and rhsSet, i.e. common = (a) + // 3. Remove common predict from lhsSet and rhsSet, i.e. ldiff = (b), rdiff = (c) + // 4. Apply the formula, get the optimized predicate: common && (ldiff || rdiff) + val lhs = splitConjunctivePredicates(left) + val rhs = splitConjunctivePredicates(right) + val common = lhs.filter(e => rhs.exists(e.semanticEquals)) + if (common.isEmpty) { + // No common factors, return the original predicate + or + } else { + val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals)) + val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals)) + if (ldiff.isEmpty || rdiff.isEmpty) { + // (a && b) || (a && b && c && ...) => a && b + common.reduce(And) } else { - val ldiff = lhs.filterNot(e => common.exists(e.semanticEquals(_))) - val rdiff = rhs.filterNot(e => common.exists(e.semanticEquals(_))) - if (ldiff.isEmpty || rdiff.isEmpty) { - // (a && b) || (a && b && c && ...) => a && b - common.reduce(And) - } else { - // (a && b && c && ...) || (a && b && d && ...) => - // ((c && ...) || (d && ...)) && a && b - (common :+ Or(ldiff.reduce(And), rdiff.reduce(And))).reduce(And) - } + // (a && b && c && ...) || (a && b && d && ...) => + // ((c && ...) || (d && ...)) && a && b + (common :+ Or(ldiff.reduce(And), rdiff.reduce(And))).reduce(And) } - } // end of Or(left, right) - - case not @ Not(exp) => exp match { - // not(true) => false - case Literal(true, BooleanType) => Literal(false) - // not(false) => true - case Literal(false, BooleanType) => Literal(true) - // not(l > r) => l <= r - case GreaterThan(l, r) => LessThanOrEqual(l, r) - // not(l >= r) => l < r - case GreaterThanOrEqual(l, r) => LessThan(l, r) - // not(l < r) => l >= r - case LessThan(l, r) => GreaterThanOrEqual(l, r) - // not(l <= r) => l > r - case LessThanOrEqual(l, r) => GreaterThan(l, r) - // not(l || r) => not(l) && not(r) - case Or(l, r) => And(Not(l), Not(r)) - // not(l && r) => not(l) or not(r) - case And(l, r) => Or(Not(l), Not(r)) - // not(not(e)) => e - case Not(e) => e - case _ => not - } // end of Not(exp) - - // if (true) a else b => a - // if (false) a else b => b - case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue + } + + case Not(TrueLiteral) => FalseLiteral + case Not(FalseLiteral) => TrueLiteral + + case Not(a GreaterThan b) => LessThanOrEqual(a, b) + case Not(a GreaterThanOrEqual b) => LessThan(a, b) + + case Not(a LessThan b) => GreaterThanOrEqual(a, b) + case Not(a LessThanOrEqual b) => GreaterThan(a, b) + + case Not(a Or b) => And(Not(a), Not(b)) + case Not(a And b) => Or(Not(a), Not(b)) + + case Not(Not(e)) => e + } + } +} + +/** + * Simplifies conditional expressions (if / case). + */ +object SimplifyConditionals extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsUp { + case If(TrueLiteral, trueValue, _) => trueValue + case If(FalseLiteral, _, falseValue) => falseValue } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala index 0e93af8b92cd2..f8e4f21451192 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala @@ -28,7 +28,25 @@ import org.apache.spark.sql.AnalysisException * This is based on Hive's org.apache.hadoop.hive.ql.parse.ParseDriver */ object ParseDriver extends Logging { - def parse(command: String, conf: ParserConf): ASTNode = { + /** Create an LogicalPlan ASTNode from a SQL command. */ + def parsePlan(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.statement().getTree + } + + /** Create an Expression ASTNode from a SQL command. */ + def parseExpression(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.namedExpression().getTree + } + + /** Create an TableIdentifier ASTNode from a SQL command. */ + def parseTableName(command: String, conf: ParserConf): ASTNode = parse(command, conf) { parser => + parser.tableName().getTree + } + + private def parse( + command: String, + conf: ParserConf)( + toTree: SparkSqlParser => CommonTree): ASTNode = { logInfo(s"Parsing command: $command") // Setup error collection. @@ -44,7 +62,7 @@ object ParseDriver extends Logging { parser.configure(conf, reporter) try { - val result = parser.statement() + val result = toTree(parser) // Check errors. reporter.checkForErrors() @@ -57,7 +75,7 @@ object ParseDriver extends Logging { if (tree.token != null || tree.getChildCount == 0) tree else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree]) } - val tree = nonNullToken(result.getTree) + val tree = nonNullToken(result) // Make sure all boundaries are set. tree.setUnknownTokenBoundaries() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index d412b3291e864..b43b7ee71e7aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, VirtualColumn} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.sql.catalyst.util._ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType => @@ -85,14 +83,6 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } def recursiveTransform(arg: Any): AnyRef = arg match { - case e: ExpressionEncoder[_] => - val newEncoder = new ExpressionEncoder( - e.schema, - e.flat, - e.toRowExpressions.map(transformExpressionDown), - transformExpressionDown(e.fromRowExpression), - e.clsTag) - newEncoder case e: Expression => transformExpressionDown(e) case Some(e: Expression) => Some(transformExpressionDown(e)) case m: Map[_, _] => m 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 4729e0e2733d7..2a1b1b131d813 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.catalyst.plans.logical import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.Encoder -import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -480,122 +478,3 @@ case object OneRowRelation extends LeafNode { */ override def statistics: Statistics = Statistics(sizeInBytes = 1) } - -trait ObjectOperator - -/** - * A relation produced by applying `func` to each partition of the `child`. tEncoder/uEncoder are - * used respectively to decode/encode from the JVM object representation expected by `func.` - */ -case class MapPartitions[T, U]( - func: Iterator[T] => Iterator[U], - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode with ObjectOperator { - override def producedAttributes: AttributeSet = outputSet -} - -/** Factory for constructing new `AppendColumn` nodes. */ -object AppendColumns { - def apply[T, U : Encoder]( - func: T => U, - tEncoder: ExpressionEncoder[T], - child: LogicalPlan): AppendColumns[T, U] = { - val attrs = encoderFor[U].schema.toAttributes - new AppendColumns[T, U](func, tEncoder, encoderFor[U], attrs, child) - } -} - -/** - * A relation produced by applying `func` to each partition of the `child`, concatenating the - * resulting columns at the end of the input row. tEncoder/uEncoder are used respectively to - * decode/encode from the JVM object representation expected by `func.` - */ -case class AppendColumns[T, U]( - func: T => U, - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - newColumns: Seq[Attribute], - child: LogicalPlan) extends UnaryNode with ObjectOperator { - override def output: Seq[Attribute] = child.output ++ newColumns - override def producedAttributes: AttributeSet = AttributeSet(newColumns) -} - -/** Factory for constructing new `MapGroups` nodes. */ -object MapGroups { - def apply[K, T, U : Encoder]( - func: (K, Iterator[T]) => TraversableOnce[U], - kEncoder: ExpressionEncoder[K], - tEncoder: ExpressionEncoder[T], - groupingAttributes: Seq[Attribute], - child: LogicalPlan): MapGroups[K, T, U] = { - new MapGroups( - func, - kEncoder, - tEncoder, - encoderFor[U], - groupingAttributes, - encoderFor[U].schema.toAttributes, - child) - } -} - -/** - * Applies func to each unique group in `child`, based on the evaluation of `groupingAttributes`. - * Func is invoked with an object representation of the grouping key an iterator containing the - * object representation of all the rows with that key. - */ -case class MapGroups[K, T, U]( - func: (K, Iterator[T]) => TraversableOnce[U], - kEncoder: ExpressionEncoder[K], - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - groupingAttributes: Seq[Attribute], - output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode with ObjectOperator { - override def producedAttributes: AttributeSet = outputSet -} - -/** Factory for constructing new `CoGroup` nodes. */ -object CoGroup { - def apply[Key, Left, Right, Result : Encoder]( - func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], - keyEnc: ExpressionEncoder[Key], - leftEnc: ExpressionEncoder[Left], - rightEnc: ExpressionEncoder[Right], - leftGroup: Seq[Attribute], - rightGroup: Seq[Attribute], - left: LogicalPlan, - right: LogicalPlan): CoGroup[Key, Left, Right, Result] = { - CoGroup( - func, - keyEnc, - leftEnc, - rightEnc, - encoderFor[Result], - encoderFor[Result].schema.toAttributes, - leftGroup, - rightGroup, - left, - right) - } -} - -/** - * A relation produced by applying `func` to each grouping key and associated values from left and - * right children. - */ -case class CoGroup[Key, Left, Right, Result]( - func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], - keyEnc: ExpressionEncoder[Key], - leftEnc: ExpressionEncoder[Left], - rightEnc: ExpressionEncoder[Right], - resultEnc: ExpressionEncoder[Result], - output: Seq[Attribute], - leftGroup: Seq[Attribute], - rightGroup: Seq[Attribute], - left: LogicalPlan, - right: LogicalPlan) extends BinaryNode with ObjectOperator { - override def producedAttributes: AttributeSet = outputSet -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala new file mode 100644 index 0000000000000..760348052739c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -0,0 +1,185 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.catalyst.encoders._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types.ObjectType + +/** + * A trait for logical operators that apply user defined functions to domain objects. + */ +trait ObjectOperator extends LogicalPlan { + + /** The serializer that is used to produce the output of this operator. */ + def serializer: Seq[NamedExpression] + + /** + * The object type that is produced by the user defined function. Note that the return type here + * is the same whether or not the operator is output serialized data. + */ + def outputObject: NamedExpression = + Alias(serializer.head.collect { case b: BoundReference => b }.head, "obj")() + + /** + * Returns a copy of this operator that will produce an object instead of an encoded row. + * Used in the optimizer when transforming plans to remove unneeded serialization. + */ + def withObjectOutput: LogicalPlan = if (output.head.dataType.isInstanceOf[ObjectType]) { + this + } else { + withNewSerializer(outputObject) + } + + /** Returns a copy of this operator with a different serializer. */ + def withNewSerializer(newSerializer: NamedExpression): LogicalPlan = makeCopy { + productIterator.map { + case c if c == serializer => newSerializer :: Nil + case other: AnyRef => other + }.toArray + } +} + +object MapPartitions { + def apply[T : Encoder, U : Encoder]( + func: Iterator[T] => Iterator[U], + child: LogicalPlan): MapPartitions = { + MapPartitions( + func.asInstanceOf[Iterator[Any] => Iterator[Any]], + encoderFor[T].fromRowExpression, + encoderFor[U].namedExpressions, + child) + } +} + +/** + * A relation produced by applying `func` to each partition of the `child`. + * @param input used to extract the input to `func` from an input row. + * @param serializer use to serialize the output of `func`. + */ +case class MapPartitions( + func: Iterator[Any] => Iterator[Any], + input: Expression, + serializer: Seq[NamedExpression], + child: LogicalPlan) extends UnaryNode with ObjectOperator { + override def output: Seq[Attribute] = serializer.map(_.toAttribute) +} + +/** Factory for constructing new `AppendColumn` nodes. */ +object AppendColumns { + def apply[T : Encoder, U : Encoder]( + func: T => U, + child: LogicalPlan): AppendColumns = { + new AppendColumns( + func.asInstanceOf[Any => Any], + encoderFor[T].fromRowExpression, + encoderFor[U].namedExpressions, + child) + } +} + +/** + * A relation produced by applying `func` to each partition of the `child`, concatenating the + * resulting columns at the end of the input row. + * @param input used to extract the input to `func` from an input row. + * @param serializer use to serialize the output of `func`. + */ +case class AppendColumns( + func: Any => Any, + input: Expression, + serializer: Seq[NamedExpression], + child: LogicalPlan) extends UnaryNode with ObjectOperator { + override def output: Seq[Attribute] = child.output ++ newColumns + def newColumns: Seq[Attribute] = serializer.map(_.toAttribute) +} + +/** Factory for constructing new `MapGroups` nodes. */ +object MapGroups { + def apply[K : Encoder, T : Encoder, U : Encoder]( + func: (K, Iterator[T]) => TraversableOnce[U], + groupingAttributes: Seq[Attribute], + child: LogicalPlan): MapGroups = { + new MapGroups( + func.asInstanceOf[(Any, Iterator[Any]) => TraversableOnce[Any]], + encoderFor[K].fromRowExpression, + encoderFor[T].fromRowExpression, + encoderFor[U].namedExpressions, + groupingAttributes, + child) + } +} + +/** + * Applies func to each unique group in `child`, based on the evaluation of `groupingAttributes`. + * Func is invoked with an object representation of the grouping key an iterator containing the + * object representation of all the rows with that key. + * @param keyObject used to extract the key object for each group. + * @param input used to extract the items in the iterator from an input row. + * @param serializer use to serialize the output of `func`. + */ +case class MapGroups( + func: (Any, Iterator[Any]) => TraversableOnce[Any], + keyObject: Expression, + input: Expression, + serializer: Seq[NamedExpression], + groupingAttributes: Seq[Attribute], + child: LogicalPlan) extends UnaryNode with ObjectOperator { + + def output: Seq[Attribute] = serializer.map(_.toAttribute) +} + +/** Factory for constructing new `CoGroup` nodes. */ +object CoGroup { + def apply[Key : Encoder, Left : Encoder, Right : Encoder, Result : Encoder]( + func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan): CoGroup = { + CoGroup( + func.asInstanceOf[(Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any]], + encoderFor[Key].fromRowExpression, + encoderFor[Left].fromRowExpression, + encoderFor[Right].fromRowExpression, + encoderFor[Result].namedExpressions, + leftGroup, + rightGroup, + left, + right) + } +} + +/** + * A relation produced by applying `func` to each grouping key and associated values from left and + * right children. + */ +case class CoGroup( + func: (Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any], + keyObject: Expression, + leftObject: Expression, + rightObject: Expression, + serializer: Seq[NamedExpression], + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + left: LogicalPlan, + right: LogicalPlan) extends BinaryNode with ObjectOperator { + override def producedAttributes: AttributeSet = outputSet + + override def output: Seq[Attribute] = serializer.map(_.toAttribute) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 1bfe0ecb1e20b..d6e10c412ca1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder, Unevaluable} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{DataType, IntegerType} /** @@ -249,6 +249,11 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } + /** + * Returns an expression that will produce a valid partition ID(i.e. non-negative and is less + * than numPartitions) based on hashing expressions. + */ + def partitionIdExpression: Expression = Pmod(new Murmur3Hash(expressions), Literal(numPartitions)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index d4be545a35ab2..d0b29aa01f640 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -315,6 +315,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } else { arg } + case tuple @ (arg1: TreeNode[_], arg2: TreeNode[_]) => + val newChild1 = nextOperation(arg1.asInstanceOf[BaseType], rule) + val newChild2 = nextOperation(arg2.asInstanceOf[BaseType], rule) + if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { + changed = true + (newChild1, newChild2) + } else { + tuple + } case other => other } case nonChild: AnyRef => nonChild diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 9fefc5656aac0..e4417e0955143 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -122,7 +122,7 @@ object NumberConverter { * unsigned, otherwise it is signed. * NB: This logic is borrowed from org.apache.hadoop.hive.ql.ud.UDFConv */ - def convert(n: Array[Byte] , fromBase: Int, toBase: Int ): UTF8String = { + def convert(n: Array[Byte], fromBase: Int, toBase: Int ): UTF8String = { if (fromBase < Character.MIN_RADIX || fromBase > Character.MAX_RADIX || Math.abs(toBase) < Character.MIN_RADIX || Math.abs(toBase) > Character.MAX_RADIX) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 9b5c86a8984be..3bd733fa2d26c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -25,8 +25,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} -import org.apache.spark.sql.catalyst.util.{LegacyTypeStringParser, DataTypeParser} - +import org.apache.spark.sql.catalyst.util.{DataTypeParser, LegacyTypeStringParser} /** * :: DeveloperApi :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala index d7204c3488313..ba9d2524a9551 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystQlSuite.scala @@ -17,36 +17,157 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} +import org.apache.spark.unsafe.types.CalendarInterval class CatalystQlSuite extends PlanTest { val parser = new CatalystQl() + test("test case insensitive") { + val result = Project(UnresolvedAlias(Literal(1)):: Nil, OneRowRelation) + assert(result === parser.parsePlan("seLect 1")) + assert(result === parser.parsePlan("select 1")) + assert(result === parser.parsePlan("SELECT 1")) + } + + test("test NOT operator with comparison operations") { + val parsed = parser.parsePlan("SELECT NOT TRUE > TRUE") + val expected = Project( + UnresolvedAlias( + Not( + GreaterThan(Literal(true), Literal(true))) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + test("support hive interval literal") { + def checkInterval(sql: String, result: CalendarInterval): Unit = { + val parsed = parser.parsePlan(sql) + val expected = Project( + UnresolvedAlias( + Literal(result) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + def checkYearMonth(lit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' YEAR TO MONTH", + CalendarInterval.fromYearMonthString(lit)) + } + + def checkDayTime(lit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' DAY TO SECOND", + CalendarInterval.fromDayTimeString(lit)) + } + + def checkSingleUnit(lit: String, unit: String): Unit = { + checkInterval( + s"SELECT INTERVAL '$lit' $unit", + CalendarInterval.fromSingleUnitString(unit, lit)) + } + + checkYearMonth("123-10") + checkYearMonth("496-0") + checkYearMonth("-2-3") + checkYearMonth("-123-0") + + checkDayTime("99 11:22:33.123456789") + checkDayTime("-99 11:22:33.123456789") + checkDayTime("10 9:8:7.123456789") + checkDayTime("1 0:0:0") + checkDayTime("-1 0:0:0") + checkDayTime("1 0:0:1") + + for (unit <- Seq("year", "month", "day", "hour", "minute", "second")) { + checkSingleUnit("7", unit) + checkSingleUnit("-7", unit) + checkSingleUnit("0", unit) + } + + checkSingleUnit("13.123456789", "second") + checkSingleUnit("-13.123456789", "second") + } + + test("support scientific notation") { + def assertRight(input: String, output: Double): Unit = { + val parsed = parser.parsePlan("SELECT " + input) + val expected = Project( + UnresolvedAlias( + Literal(output) + ) :: Nil, + OneRowRelation) + comparePlans(parsed, expected) + } + + assertRight("9.0e1", 90) + assertRight("0.9e+2", 90) + assertRight("900e-1", 90) + assertRight("900.0E-1", 90) + assertRight("9.e+1", 90) + + intercept[AnalysisException](parser.parsePlan("SELECT .e3")) + } + + test("parse expressions") { + compareExpressions( + parser.parseExpression("prinln('hello', 'world')"), + UnresolvedFunction( + "prinln", Literal("hello") :: Literal("world") :: Nil, false)) + + compareExpressions( + parser.parseExpression("1 + r.r As q"), + Alias(Add(Literal(1), UnresolvedAttribute("r.r")), "q")()) + + compareExpressions( + parser.parseExpression("1 - f('o', o(bar))"), + Subtract(Literal(1), + UnresolvedFunction("f", + Literal("o") :: + UnresolvedFunction("o", UnresolvedAttribute("bar") :: Nil, false) :: + Nil, false))) + } + + test("table identifier") { + assert(TableIdentifier("q") === parser.parseTableIdentifier("q")) + assert(TableIdentifier("q", Some("d")) === parser.parseTableIdentifier("d.q")) + intercept[AnalysisException](parser.parseTableIdentifier("")) + // TODO parser swallows third identifier. + // intercept[AnalysisException](parser.parseTableIdentifier("d.q.g")) + } + test("parse union/except/intersect") { - parser.createPlan("select * from t1 union all select * from t2") - parser.createPlan("select * from t1 union distinct select * from t2") - parser.createPlan("select * from t1 union select * from t2") - parser.createPlan("select * from t1 except select * from t2") - parser.createPlan("select * from t1 intersect select * from t2") - parser.createPlan("(select * from t1) union all (select * from t2)") - parser.createPlan("(select * from t1) union distinct (select * from t2)") - parser.createPlan("(select * from t1) union (select * from t2)") - parser.createPlan("select * from ((select * from t1) union (select * from t2)) t") + parser.parsePlan("select * from t1 union all select * from t2") + parser.parsePlan("select * from t1 union distinct select * from t2") + parser.parsePlan("select * from t1 union select * from t2") + parser.parsePlan("select * from t1 except select * from t2") + parser.parsePlan("select * from t1 intersect select * from t2") + parser.parsePlan("(select * from t1) union all (select * from t2)") + parser.parsePlan("(select * from t1) union distinct (select * from t2)") + parser.parsePlan("(select * from t1) union (select * from t2)") + parser.parsePlan("select * from ((select * from t1) union (select * from t2)) t") } test("window function: better support of parentheses") { - parser.createPlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " + + parser.parsePlan("select sum(product + 1) over (partition by ((1) + (product / 2)) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by (1 + (product / 2)) " + + parser.parsePlan("select sum(product + 1) over (partition by (1 + (product / 2)) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product / 2) + 1) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product / 2) + 1) " + "order by 2) from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product) + (1)) order by 2) " + "from windowData") - parser.createPlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by ((product) + 1) order by 2) " + "from windowData") - parser.createPlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + + parser.parsePlan("select sum(product + 1) over (partition by (product + (1)) order by 2) " + "from windowData") } } 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 cf84855885a37..975cd87d090e4 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 @@ -239,7 +239,7 @@ class AnalysisSuite extends AnalysisTest { test("SPARK-12102: Ignore nullablity when comparing two sides of case") { val relation = LocalRelation('a.struct('x.int), 'b.struct('x.int.withNullability(false))) - val plan = relation.select(CaseWhen(Seq(Literal(true), 'a, 'b)).as("val")) + val plan = relation.select(CaseWhen(Seq((Literal(true), 'a.attr)), 'b).as("val")) assertAnalysisSuccess(plan) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 0521ed848c793..59549e3998e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -132,13 +132,13 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertErrorForDifferingTypes(If('booleanField, 'intField, 'booleanField)) assertError( - CaseWhen(Seq('booleanField, 'intField, 'booleanField, 'mapField)), + CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('booleanField.attr, 'mapField.attr))), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( CaseKeyWhen('intField, Seq('intField, 'stringField, 'intField, 'mapField)), "THEN and ELSE expressions should all be same type or coercible to a common type") assertError( - CaseWhen(Seq('booleanField, 'intField, 'intField, 'intField)), + CaseWhen(Seq(('booleanField.attr, 'intField.attr), ('intField.attr, 'intField.attr))), "WHEN expressions in CaseWhen should all be boolean type") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 58d808c55860d..b1f6c0b802d8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -299,7 +299,7 @@ class HiveTypeCoercionSuite extends PlanTest { } test("type coercion for CaseKeyWhen") { - ruleTest(HiveTypeCoercion.CaseWhenCoercion, + ruleTest(HiveTypeCoercion.ImplicitTypeCasts, CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a"))) ) @@ -308,19 +308,44 @@ class HiveTypeCoercionSuite extends PlanTest { CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) ruleTest(HiveTypeCoercion.CaseWhenCoercion, - CaseWhen(Seq(Literal(true), Literal(1.2), Literal.create(1, DecimalType(7, 2)))), - CaseWhen(Seq( - Literal(true), Literal(1.2), Cast(Literal.create(1, DecimalType(7, 2)), DoubleType))) + CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Literal(1.2))), + Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) ) ruleTest(HiveTypeCoercion.CaseWhenCoercion, - CaseWhen(Seq(Literal(true), Literal(100L), Literal.create(1, DecimalType(7, 2)))), - CaseWhen(Seq( - Literal(true), Cast(Literal(100L), DecimalType(22, 2)), - Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2)))) + CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))), + Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) + ) + } + + test("BooleanEquality type cast") { + val be = HiveTypeCoercion.BooleanEquality + // Use something more than a literal to avoid triggering the simplification rules. + val one = Add(Literal(Decimal(1)), Literal(Decimal(0))) + + ruleTest(be, + EqualTo(Literal(true), one), + EqualTo(Cast(Literal(true), one.dataType), one) + ) + + ruleTest(be, + EqualTo(one, Literal(true)), + EqualTo(one, Cast(Literal(true), one.dataType)) + ) + + ruleTest(be, + EqualNullSafe(Literal(true), one), + EqualNullSafe(Cast(Literal(true), one.dataType), one) + ) + + ruleTest(be, + EqualNullSafe(one, Literal(true)), + EqualNullSafe(one, Cast(Literal(true), one.dataType)) ) } - test("type coercion simplification for equal to") { + test("BooleanEquality simplification") { val be = HiveTypeCoercion.BooleanEquality ruleTest(be, @@ -426,7 +451,7 @@ class HiveTypeCoercionSuite extends PlanTest { val expectedTypes = Seq(DecimalType(10, 5), DecimalType(10, 5), DecimalType(15, 5), DecimalType(25, 5), DoubleType, DoubleType) - rightTypes.zip(expectedTypes).map { case (rType, expectedType) => + rightTypes.zip(expectedTypes).foreach { case (rType, expectedType) => val plan2 = LocalRelation( AttributeReference("r", rType)()) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index 4029da5925580..3c581ecdaf068 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -80,38 +80,39 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper val c5 = 'a.string.at(4) val c6 = 'a.string.at(5) - checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row) - - checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row) - checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) - checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) - - assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) + checkEvaluation(CaseWhen(Seq((c1, c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c2, c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c3, c4)), c6), "a", row) + checkEvaluation(CaseWhen(Seq((Literal.create(null, BooleanType), c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((Literal.create(false, BooleanType), c4)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((Literal.create(true, BooleanType), c4)), c6), "a", row) + + checkEvaluation(CaseWhen(Seq((c3, c4), (c2, c5)), c6), "a", row) + checkEvaluation(CaseWhen(Seq((c2, c4), (c3, c5)), c6), "b", row) + checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5)), c6), "c", row) + checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5))), null, row) + + assert(CaseWhen(Seq((c2, c4)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5))).nullable === true) val c4_notNull = 'a.boolean.notNull.at(3) val c5_notNull = 'a.boolean.notNull.at(4) val c6_notNull = 'a.boolean.notNull.at(5) - assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull)), c6_notNull).nullable === false) + assert(CaseWhen(Seq((c2, c4)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull)), c6).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6_notNull).nullable === false) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5)), c6_notNull).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull))).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5))).nullable === true) } test("case key when") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala index 64161bebdcbe8..75131a6170222 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala @@ -79,7 +79,8 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { .add("long", LongType) .add("float", FloatType) .add("double", DoubleType) - .add("decimal", DecimalType.SYSTEM_DEFAULT) + .add("bigDecimal", DecimalType.SYSTEM_DEFAULT) + .add("smallDecimal", DecimalType.USER_DEFAULT) .add("string", StringType) .add("binary", BinaryType) .add("date", DateType) @@ -126,7 +127,8 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { val literals = input.toSeq(inputSchema).zip(inputSchema.map(_.dataType)).map { case (value, dt) => Literal.create(value, dt) } - checkEvaluation(Murmur3Hash(literals, seed), input.hashCode(seed)) + // Only test the interpreted version has same result with codegen version. + checkEvaluation(Murmur3Hash(literals, seed), Murmur3Hash(literals, seed).eval()) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 000a3b7ecb7c6..6932f185b9d62 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -80,7 +80,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition(('a < 2 || 'a > 3 || 'b > 5) && 'a < 2, 'a < 2) - checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5) , 'a < 2) + checkCondition('a < 2 && ('a < 2 || 'a > 3 || 'b > 5), 'a < 2) checkCondition(('a < 2 || 'b > 3) && ('a < 2 || 'c > 5), 'a < 2 || ('b > 3 && 'c > 5)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 9fe2b2d1f48ca..87ad81db11b64 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -34,7 +34,8 @@ class CombiningLimitsSuite extends PlanTest { Batch("Constant Folding", FixedPoint(10), NullPropagation, ConstantFolding, - BooleanSimplification) :: Nil + BooleanSimplification, + SimplifyConditionals) :: Nil } val testRelation = LocalRelation('a.int, 'b.int, 'c.int) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala new file mode 100644 index 0000000000000..91777375608fd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSerializationSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.NewInstance +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, MapPartitions} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +case class OtherTuple(_1: Int, _2: Int) + +class EliminateSerializationSuite extends PlanTest { + private object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Serialization", FixedPoint(100), + EliminateSerialization) :: Nil + } + + implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() + private val func = identity[Iterator[(Int, Int)]] _ + private val func2 = identity[Iterator[OtherTuple]] _ + + def assertObjectCreations(count: Int, plan: LogicalPlan): Unit = { + val newInstances = plan.flatMap(_.expressions.collect { + case n: NewInstance => n + }) + + if (newInstances.size != count) { + fail( + s""" + |Wrong number of object creations in plan: ${newInstances.size} != $count + |$plan + """.stripMargin) + } + } + + test("back to back MapPartitions") { + val input = LocalRelation('_1.int, '_2.int) + val plan = + MapPartitions(func, + MapPartitions(func, input)) + + val optimized = Optimize.execute(plan.analyze) + assertObjectCreations(1, optimized) + } + + test("back to back with object change") { + val input = LocalRelation('_1.int, '_2.int) + val plan = + MapPartitions(func, + MapPartitions(func2, input)) + + val optimized = Optimize.execute(plan.analyze) + assertObjectCreations(2, optimized) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala new file mode 100644 index 0000000000000..8e5d7ef3c9d49 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalSuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + + +class SimplifyConditionalSuite extends PlanTest with PredicateHelper { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("SimplifyConditionals", FixedPoint(50), SimplifyConditionals) :: Nil + } + + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { + val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation).analyze + val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation).analyze) + comparePlans(actual, correctAnswer) + } + + test("simplify if") { + assertEquivalent( + If(TrueLiteral, Literal(10), Literal(20)), + Literal(10)) + + assertEquivalent( + If(FalseLiteral, Literal(10), Literal(20)), + Literal(20)) + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala similarity index 96% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala index 41455221cfdc6..24413e7a2a3f0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyStringCaseConversionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -/* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -25,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules._ -class SimplifyCaseConversionExpressionsSuite extends PlanTest { +class SimplifyStringCaseConversionSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java new file mode 100644 index 0000000000000..d9dde92ceb6d7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.vectorized; + +import org.apache.spark.sql.types.DataType; + +/** + * This class represents a column of values and provides the main APIs to access the data + * values. It supports all the types and contains get/put APIs as well as their batched versions. + * The batched versions are preferable whenever possible. + * + * Most of the APIs take the rowId as a parameter. This is the local 0-based row id for values + * in the current RowBatch. + * + * A ColumnVector should be considered immutable once originally created. In other words, it is not + * valid to call put APIs after reads until reset() is called. + */ +public abstract class ColumnVector { + /** + * Allocates a column with each element of size `width` either on or off heap. + */ + public static ColumnVector allocate(int capacity, DataType type, boolean offHeap) { + if (offHeap) { + return new OffHeapColumnVector(capacity, type); + } else { + return new OnHeapColumnVector(capacity, type); + } + } + + public final DataType dataType() { return type; } + + /** + * Resets this column for writing. The currently stored values are no longer accessible. + */ + public void reset() { + numNulls = 0; + if (anyNullsSet) { + putNotNulls(0, capacity); + anyNullsSet = false; + } + } + + /** + * Cleans up memory for this column. The column is not usable after this. + * TODO: this should probably have ref-counted semantics. + */ + public abstract void close(); + + /** + * Returns the number of nulls in this column. + */ + public final int numNulls() { return numNulls; } + + /** + * Returns true if any of the nulls indicator are set for this column. This can be used + * as an optimization to prevent setting nulls. + */ + public final boolean anyNullsSet() { return anyNullsSet; } + + /** + * Returns the off heap ptr for the arrays backing the NULLs and values buffer. Only valid + * to call for off heap columns. + */ + public abstract long nullsNativeAddress(); + public abstract long valuesNativeAddress(); + + /** + * Sets the value at rowId to null/not null. + */ + public abstract void putNotNull(int rowId); + public abstract void putNull(int rowId); + + /** + * Sets the values from [rowId, rowId + count) to null/not null. + */ + public abstract void putNulls(int rowId, int count); + public abstract void putNotNulls(int rowId, int count); + + /** + * Returns whether the value at rowId is NULL. + */ + public abstract boolean getIsNull(int rowId); + + /** + * Sets the value at rowId to `value`. + */ + public abstract void putInt(int rowId, int value); + + /** + * Sets values from [rowId, rowId + count) to value. + */ + public abstract void putInts(int rowId, int count, int value); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + */ + public abstract void putInts(int rowId, int count, int[] src, int srcIndex); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * The data in src must be 4-byte little endian ints. + */ + public abstract void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex); + + /** + * Returns the integer for rowId. + */ + public abstract int getInt(int rowId); + + /** + * Sets the value at rowId to `value`. + */ + public abstract void putDouble(int rowId, double value); + + /** + * Sets values from [rowId, rowId + count) to value. + */ + public abstract void putDoubles(int rowId, int count, double value); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * src should contain `count` doubles written as ieee format. + */ + public abstract void putDoubles(int rowId, int count, double[] src, int srcIndex); + + /** + * Sets values from [rowId, rowId + count) to [src + srcIndex, src + srcIndex + count) + * The data in src must be ieee formated doubles. + */ + public abstract void putDoubles(int rowId, int count, byte[] src, int srcIndex); + + /** + * Returns the double for rowId. + */ + public abstract double getDouble(int rowId); + + /** + * Maximum number of rows that can be stored in this column. + */ + protected final int capacity; + + /** + * Number of nulls in this column. This is an optimization for the reader, to skip NULL checks. + */ + protected int numNulls; + + /** + * True if there is at least one NULL byte set. This is an optimization for the writer, to skip + * having to clear NULL bits. + */ + protected boolean anyNullsSet; + + /** + * Data type for this column. + */ + protected final DataType type; + + protected ColumnVector(int capacity, DataType type) { + this.capacity = capacity; + this.type = type; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java new file mode 100644 index 0000000000000..47defac4534dc --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.vectorized; + +import java.util.Arrays; +import java.util.Iterator; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; + +import org.apache.commons.lang.NotImplementedException; + +/** + * This class is the in memory representation of rows as they are streamed through operators. It + * is designed to maximize CPU efficiency and not storage footprint. Since it is expected that + * each operator allocates one of thee objects, the storage footprint on the task is negligible. + * + * The layout is a columnar with values encoded in their native format. Each RowBatch contains + * a horizontal partitioning of the data, split into columns. + * + * The ColumnarBatch supports either on heap or offheap modes with (mostly) the identical API. + * + * TODO: + * - There are many TODOs for the existing APIs. They should throw a not implemented exception. + * - Compaction: The batch and columns should be able to compact based on a selection vector. + */ +public final class ColumnarBatch { + private static final int DEFAULT_BATCH_SIZE = 4 * 1024; + + private final StructType schema; + private final int capacity; + private int numRows; + private final ColumnVector[] columns; + + // True if the row is filtered. + private final boolean[] filteredRows; + + // Total number of rows that have been filtered. + private int numRowsFiltered = 0; + + public static ColumnarBatch allocate(StructType schema, boolean offHeap) { + return new ColumnarBatch(schema, DEFAULT_BATCH_SIZE, offHeap); + } + + public static ColumnarBatch allocate(StructType schema, boolean offHeap, int maxRows) { + return new ColumnarBatch(schema, maxRows, offHeap); + } + + /** + * Called to close all the columns in this batch. It is not valid to access the data after + * calling this. This must be called at the end to clean up memory allcoations. + */ + public void close() { + for (ColumnVector c: columns) { + c.close(); + } + } + + /** + * Adapter class to interop with existing components that expect internal row. A lot of + * performance is lost with this translation. + */ + public final class Row extends InternalRow { + private int rowId; + + /** + * Marks this row as being filtered out. This means a subsequent iteration over the rows + * in this batch will not include this row. + */ + public final void markFiltered() { + ColumnarBatch.this.markFiltered(rowId); + } + + @Override + public final int numFields() { + return ColumnarBatch.this.numCols(); + } + + @Override + public final InternalRow copy() { + throw new NotImplementedException(); + } + + @Override + public final boolean anyNull() { + throw new NotImplementedException(); + } + + @Override + public final boolean isNullAt(int ordinal) { + return ColumnarBatch.this.column(ordinal).getIsNull(rowId); + } + + @Override + public final boolean getBoolean(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final byte getByte(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final short getShort(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final int getInt(int ordinal) { + return ColumnarBatch.this.column(ordinal).getInt(rowId); + } + + @Override + public final long getLong(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final float getFloat(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final double getDouble(int ordinal) { + return ColumnarBatch.this.column(ordinal).getDouble(rowId); + } + + @Override + public final Decimal getDecimal(int ordinal, int precision, int scale) { + throw new NotImplementedException(); + } + + @Override + public final UTF8String getUTF8String(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final byte[] getBinary(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final CalendarInterval getInterval(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final InternalRow getStruct(int ordinal, int numFields) { + throw new NotImplementedException(); + } + + @Override + public final ArrayData getArray(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final MapData getMap(int ordinal) { + throw new NotImplementedException(); + } + + @Override + public final Object get(int ordinal, DataType dataType) { + throw new NotImplementedException(); + } + } + + /** + * Returns an iterator over the rows in this batch. This skips rows that are filtered out. + */ + public Iterator rowIterator() { + final int maxRows = ColumnarBatch.this.numRows(); + final Row row = new Row(); + return new Iterator() { + int rowId = 0; + + @Override + public boolean hasNext() { + while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) { + ++rowId; + } + return rowId < maxRows; + } + + @Override + public Row next() { + assert(hasNext()); + while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) { + ++rowId; + } + row.rowId = rowId++; + return row; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Resets the batch for writing. + */ + public void reset() { + for (int i = 0; i < numCols(); ++i) { + columns[i].reset(); + } + if (this.numRowsFiltered > 0) { + Arrays.fill(filteredRows, false); + } + this.numRows = 0; + this.numRowsFiltered = 0; + } + + /** + * Sets the number of rows that are valid. + */ + public void setNumRows(int numRows) { + assert(numRows <= this.capacity); + this.numRows = numRows; + } + + /** + * Returns the number of columns that make up this batch. + */ + public int numCols() { return columns.length; } + + /** + * Returns the number of rows for read, including filtered rows. + */ + public int numRows() { return numRows; } + + /** + * Returns the number of valid rowss. + */ + public int numValidRows() { + assert(numRowsFiltered <= numRows); + return numRows - numRowsFiltered; + } + + /** + * Returns the max capacity (in number of rows) for this batch. + */ + public int capacity() { return capacity; } + + /** + * Returns the column at `ordinal`. + */ + public ColumnVector column(int ordinal) { return columns[ordinal]; } + + /** + * Marks this row as being filtered out. This means a subsequent iteration over the rows + * in this batch will not include this row. + */ + public final void markFiltered(int rowId) { + assert(filteredRows[rowId] == false); + filteredRows[rowId] = true; + ++numRowsFiltered; + } + + private ColumnarBatch(StructType schema, int maxRows, boolean offHeap) { + this.schema = schema; + this.capacity = maxRows; + this.columns = new ColumnVector[schema.size()]; + this.filteredRows = new boolean[maxRows]; + + for (int i = 0; i < schema.fields().length; ++i) { + StructField field = schema.fields()[i]; + columns[i] = ColumnVector.allocate(maxRows, field.dataType(), offHeap); + } + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java new file mode 100644 index 0000000000000..2a9a2d1104b22 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.vectorized; + +import java.nio.ByteOrder; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.unsafe.Platform; + + +import org.apache.commons.lang.NotImplementedException; + +/** + * Column data backed using offheap memory. + */ +public final class OffHeapColumnVector extends ColumnVector { + // The data stored in these two allocations need to maintain binary compatible. We can + // directly pass this buffer to external components. + private long nulls; + private long data; + + protected OffHeapColumnVector(int capacity, DataType type) { + super(capacity, type); + if (!ByteOrder.nativeOrder().equals(ByteOrder.LITTLE_ENDIAN)) { + throw new NotImplementedException("Only little endian is supported."); + } + + this.nulls = Platform.allocateMemory(capacity); + if (type instanceof IntegerType) { + this.data = Platform.allocateMemory(capacity * 4); + } else if (type instanceof DoubleType) { + this.data = Platform.allocateMemory(capacity * 8); + } else { + throw new RuntimeException("Unhandled " + type); + } + reset(); + } + + @Override + public final long valuesNativeAddress() { + return data; + } + + @Override + public long nullsNativeAddress() { + return nulls; + } + + @Override + public final void close() { + Platform.freeMemory(nulls); + Platform.freeMemory(data); + nulls = 0; + data = 0; + } + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + Platform.putByte(null, nulls + rowId, (byte) 0); + } + + @Override + public final void putNull(int rowId) { + Platform.putByte(null, nulls + rowId, (byte) 1); + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + long offset = nulls + rowId; + for (int i = 0; i < count; ++i, ++offset) { + Platform.putByte(null, offset, (byte) 1); + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + long offset = nulls + rowId; + for (int i = 0; i < count; ++i, ++offset) { + Platform.putByte(null, offset, (byte) 0); + } + } + + @Override + public final boolean getIsNull(int rowId) { + return Platform.getByte(null, nulls + rowId) == 1; + } + + // + // APIs dealing with ints + // + + @Override + public final void putInt(int rowId, int value) { + Platform.putInt(null, data + 4 * rowId, value); + } + + @Override + public final void putInts(int rowId, int count, int value) { + long offset = data + 4 * rowId; + for (int i = 0; i < count; ++i, offset += 4) { + Platform.putInt(null, offset, value); + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, + null, data + 4 * rowId, count * 4); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, + null, data + 4 * rowId, count * 4); + } + + @Override + public final int getInt(int rowId) { + return Platform.getInt(null, data + 4 * rowId); + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { + Platform.putDouble(null, data + rowId * 8, value); + } + + @Override + public final void putDoubles(int rowId, int count, double value) { + long offset = data + 8 * rowId; + for (int i = 0; i < count; ++i, offset += 8) { + Platform.putDouble(null, offset, value); + } + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, + null, data + 8 * rowId, count * 8); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex, + null, data + rowId * 8, count * 8); + } + + @Override + public final double getDouble(int rowId) { + return Platform.getDouble(null, data + rowId * 8); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java new file mode 100644 index 0000000000000..a7b3addf11b14 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.vectorized; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.unsafe.Platform; + +import java.nio.ByteBuffer; +import java.nio.DoubleBuffer; +import java.util.Arrays; + +/** + * A column backed by an in memory JVM array. This stores the NULLs as a byte per value + * and a java array for the values. + */ +public final class OnHeapColumnVector extends ColumnVector { + // The data stored in these arrays need to maintain binary compatible. We can + // directly pass this buffer to external components. + + // This is faster than a boolean array and we optimize this over memory footprint. + private byte[] nulls; + + // Array for each type. Only 1 is populated for any type. + private int[] intData; + private double[] doubleData; + + protected OnHeapColumnVector(int capacity, DataType type) { + super(capacity, type); + if (type instanceof IntegerType) { + this.intData = new int[capacity]; + } else if (type instanceof DoubleType) { + this.doubleData = new double[capacity]; + } else { + throw new RuntimeException("Unhandled " + type); + } + this.nulls = new byte[capacity]; + reset(); + } + + @Override + public final long valuesNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + @Override + public final long nullsNativeAddress() { + throw new RuntimeException("Cannot get native address for on heap column"); + } + + @Override + public final void close() { + nulls = null; + intData = null; + doubleData = null; + } + + + // + // APIs dealing with nulls + // + + @Override + public final void putNotNull(int rowId) { + nulls[rowId] = (byte)0; + } + + @Override + public final void putNull(int rowId) { + nulls[rowId] = (byte)1; + ++numNulls; + anyNullsSet = true; + } + + @Override + public final void putNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)1; + } + anyNullsSet = true; + numNulls += count; + } + + @Override + public final void putNotNulls(int rowId, int count) { + for (int i = 0; i < count; ++i) { + nulls[rowId + i] = (byte)0; + } + } + + @Override + public final boolean getIsNull(int rowId) { + return nulls[rowId] == 1; + } + + // + // APIs dealing with Ints + // + + @Override + public final void putInt(int rowId, int value) { + intData[rowId] = value; + } + + @Override + public final void putInts(int rowId, int count, int value) { + for (int i = 0; i < count; ++i) { + intData[i + rowId] = value; + } + } + + @Override + public final void putInts(int rowId, int count, int[] src, int srcIndex) { + System.arraycopy(src, srcIndex, intData, rowId, count); + } + + @Override + public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; + for (int i = 0; i < count; ++i) { + intData[i + rowId] = Platform.getInt(src, srcOffset);; + srcIndex += 4; + srcOffset += 4; + } + } + + @Override + public final int getInt(int rowId) { + return intData[rowId]; + } + + // + // APIs dealing with doubles + // + + @Override + public final void putDouble(int rowId, double value) { + doubleData[rowId] = value; + } + + @Override + public final void putDoubles(int rowId, int count, double value) { + Arrays.fill(doubleData, rowId, rowId + count, value); + } + + @Override + public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + System.arraycopy(src, srcIndex, doubleData, rowId, count); + } + + @Override + public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, + Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); + } + + @Override + public final double getDouble(int rowId) { + return doubleData[rowId]; + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index e8c61d6e01dc3..6a020f9f2883e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -437,8 +437,11 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.4.0 */ def when(condition: Column, value: Any): Column = this.expr match { - case CaseWhen(branches: Seq[Expression]) => - withExpr { CaseWhen(branches ++ Seq(lit(condition).expr, lit(value).expr)) } + case CaseWhen(branches, None) => + withExpr { CaseWhen(branches :+ (condition.expr, lit(value).expr)) } + case CaseWhen(branches, Some(_)) => + throw new IllegalArgumentException( + "when() cannot be applied once otherwise() is applied") case _ => throw new IllegalArgumentException( "when() can only be applied on a Column previously generated by when() function") @@ -466,13 +469,11 @@ class Column(protected[sql] val expr: Expression) extends Logging { * @since 1.4.0 */ def otherwise(value: Any): Column = this.expr match { - case CaseWhen(branches: Seq[Expression]) => - if (branches.size % 2 == 0) { - withExpr { CaseWhen(branches :+ lit(value).expr) } - } else { - throw new IllegalArgumentException( - "otherwise() can only be applied once on a Column previously generated by when()") - } + case CaseWhen(branches, None) => + withExpr { CaseWhen(branches, Option(lit(value).expr)) } + case CaseWhen(branches, Some(_)) => + throw new IllegalArgumentException( + "otherwise() can only be applied once on a Column previously generated by when()") case _ => throw new IllegalArgumentException( "otherwise() can only be applied on a Column previously generated by when()") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 2770ecbcb1909..c22e720cc8d2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1735,7 +1735,7 @@ class DataFrame private[sql]( */ def inputFiles: Array[String] = { val files: Seq[String] = logicalPlan.collect { - case LogicalRelation(fsBasedRelation: FileRelation, _) => + case LogicalRelation(fsBasedRelation: FileRelation, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => fr.inputFiles diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 42f01e9359c64..9a9f7d111cf4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -336,12 +336,7 @@ class Dataset[T] private[sql]( def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sqlContext, - MapPartitions[T, U]( - func, - resolvedTEncoder, - encoderFor[U], - encoderFor[U].schema.toAttributes, - logicalPlan)) + MapPartitions[T, U](func, logicalPlan)) } /** @@ -434,7 +429,7 @@ class Dataset[T] private[sql]( */ def groupBy[K : Encoder](func: T => K): GroupedDataset[K, T] = { val inputPlan = logicalPlan - val withGroupingKey = AppendColumns(func, resolvedTEncoder, inputPlan) + val withGroupingKey = AppendColumns(func, inputPlan) val executed = sqlContext.executePlan(withGroupingKey) new GroupedDataset( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index 717709e4f9312..deed45d273c33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule /** * :: Experimental :: @@ -42,4 +44,7 @@ class ExperimentalMethods protected[sql](sqlContext: SQLContext) { @Experimental var extraStrategies: Seq[Strategy] = Nil + @Experimental + var extraOptimizations: Seq[Rule[LogicalPlan]] = Nil + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala index a819ddceb1b1b..b3f8284364782 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala @@ -115,8 +115,6 @@ class GroupedDataset[K, V] private[sql]( sqlContext, MapGroups( f, - resolvedKEncoder, - resolvedVEncoder, groupingAttributes, logicalPlan)) } @@ -305,13 +303,11 @@ class GroupedDataset[K, V] private[sql]( def cogroup[U, R : Encoder]( other: GroupedDataset[K, U])( f: (K, Iterator[V], Iterator[U]) => TraversableOnce[R]): Dataset[R] = { + implicit val uEncoder = other.unresolvedVEncoder new Dataset[R]( sqlContext, CoGroup( f, - this.resolvedKEncoder, - this.resolvedVEncoder, - other.resolvedVEncoder, this.groupingAttributes, other.groupingAttributes, this.logicalPlan, 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 4b4cba330356c..0d2d0002fb4f9 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 @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.errors.DialectException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} +import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ @@ -202,7 +202,7 @@ class SQLContext private[sql]( } @transient - protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer + protected[sql] lazy val optimizer: Optimizer = new SparkOptimizer(this) @transient protected[sql] val ddlParser = new DDLParser(sqlParser.parse(_)) @@ -954,7 +954,7 @@ class SQLContext private[sql]( } } - // Register a succesfully instantiatd context to the singleton. This should be at the end of + // Register a successfully instantiated context to the singleton. This should be at the end of // the class definition so that the singleton is updated only if there is no exception in the // construction of the instance. sparkContext.addSparkListener(new SparkListener { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 6b100577077c6..3770883af1e2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -143,7 +143,13 @@ case class Exchange( val rdd = child.execute() val part: Partitioner = newPartitioning match { case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) - case HashPartitioning(expressions, numPartitions) => new HashPartitioner(numPartitions) + case HashPartitioning(_, n) => + new Partitioner { + override def numPartitions: Int = n + // For HashPartitioning, the partitioning key is already a valid partition ID, as we use + // `HashPartitioning.partitionIdExpression` to produce partitioning key. + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + } case RangePartitioning(sortingExpressions, numPartitions) => // Internally, RangePartitioner runs a job on the RDD that samples keys to compute // partition bounds. To get accurate samples, we need to copy the mutable keys. @@ -173,7 +179,9 @@ case class Exchange( position += 1 position } - case HashPartitioning(expressions, _) => newMutableProjection(expressions, child.output)() + case h: HashPartitioning => + val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, child.output) + row => projection(row).getInt(0) case RangePartitioning(_, _) | SinglePartition => identity case _ => sys.error(s"Exchange not implemented for $newPartitioning") } @@ -223,7 +231,7 @@ case class Exchange( new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) } - protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") { + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { coordinator match { case Some(exchangeCoordinator) => val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala new file mode 100644 index 0000000000000..edaf3b36aa52e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.optimizer._ + +class SparkOptimizer(val sqlContext: SQLContext) + extends Optimizer { + override def batches: Seq[Batch] = super.batches :+ Batch( + "User Provided Optimizers", FixedPoint(100), sqlContext.experimental.extraOptimizations: _*) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index a322688a259e2..f3e89ef4a71f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.sql.execution +import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} -import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier} private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends CatalystQl(conf) { /** Check if a command should not be explained. */ 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 482130a18d939..910519d0e6814 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 @@ -309,16 +309,15 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new IllegalStateException( "logical distinct operator should have been replaced by aggregate in the optimizer") - case logical.MapPartitions(f, tEnc, uEnc, output, child) => - execution.MapPartitions(f, tEnc, uEnc, output, planLater(child)) :: Nil - case logical.AppendColumns(f, tEnc, uEnc, newCol, child) => - execution.AppendColumns(f, tEnc, uEnc, newCol, planLater(child)) :: Nil - case logical.MapGroups(f, kEnc, tEnc, uEnc, grouping, output, child) => - execution.MapGroups(f, kEnc, tEnc, uEnc, grouping, output, planLater(child)) :: Nil - case logical.CoGroup(f, kEnc, leftEnc, rightEnc, rEnc, output, - leftGroup, rightGroup, left, right) => - execution.CoGroup(f, kEnc, leftEnc, rightEnc, rEnc, output, leftGroup, rightGroup, - planLater(left), planLater(right)) :: Nil + case logical.MapPartitions(f, in, out, child) => + execution.MapPartitions(f, in, out, planLater(child)) :: Nil + case logical.AppendColumns(f, in, out, child) => + execution.AppendColumns(f, in, out, planLater(child)) :: Nil + case logical.MapGroups(f, key, in, out, grouping, child) => + execution.MapGroups(f, key, in, out, grouping, planLater(child)) :: Nil + case logical.CoGroup(f, keyObj, lObj, rObj, out, lGroup, rGroup, left, right) => + execution.CoGroup( + f, keyObj, lObj, rObj, out, lGroup, rGroup, planLater(left), planLater(right)) :: Nil case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala index be885397a7d40..168b5ab0316d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Window.scala @@ -22,6 +22,7 @@ import java.util import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -29,7 +30,6 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.types.IntegerType import org.apache.spark.util.collection.unsafe.sort.{UnsafeExternalSorter, UnsafeSorterIterator} -import org.apache.spark.{SparkEnv, TaskContext} /** * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 95bef683238a7..92c9a561312ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -21,9 +21,7 @@ import org.apache.spark.{HashPartitioner, SparkEnv} import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD, ShuffledRDD} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.LongType @@ -329,128 +327,3 @@ case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPl protected override def doExecute(): RDD[InternalRow] = child.execute() } - -/** - * Applies the given function to each input row and encodes the result. - */ -case class MapPartitions[T, U]( - func: Iterator[T] => Iterator[U], - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - output: Seq[Attribute], - child: SparkPlan) extends UnaryNode { - override def producedAttributes: AttributeSet = outputSet - - override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => - val tBoundEncoder = tEncoder.bind(child.output) - func(iter.map(tBoundEncoder.fromRow)).map(uEncoder.toRow) - } - } -} - -/** - * Applies the given function to each input row, appending the encoded result at the end of the row. - */ -case class AppendColumns[T, U]( - func: T => U, - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - newColumns: Seq[Attribute], - child: SparkPlan) extends UnaryNode { - override def producedAttributes: AttributeSet = AttributeSet(newColumns) - - override def output: Seq[Attribute] = child.output ++ newColumns - - override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => - val tBoundEncoder = tEncoder.bind(child.output) - val combiner = GenerateUnsafeRowJoiner.create(tEncoder.schema, uEncoder.schema) - iter.map { row => - val newColumns = uEncoder.toRow(func(tBoundEncoder.fromRow(row))) - combiner.join(row.asInstanceOf[UnsafeRow], newColumns.asInstanceOf[UnsafeRow]): InternalRow - } - } - } -} - -/** - * Groups the input rows together and calls the function with each group and an iterator containing - * all elements in the group. The result of this function is encoded and flattened before - * being output. - */ -case class MapGroups[K, T, U]( - func: (K, Iterator[T]) => TraversableOnce[U], - kEncoder: ExpressionEncoder[K], - tEncoder: ExpressionEncoder[T], - uEncoder: ExpressionEncoder[U], - groupingAttributes: Seq[Attribute], - output: Seq[Attribute], - child: SparkPlan) extends UnaryNode { - override def producedAttributes: AttributeSet = outputSet - - override def requiredChildDistribution: Seq[Distribution] = - ClusteredDistribution(groupingAttributes) :: Nil - - override def requiredChildOrdering: Seq[Seq[SortOrder]] = - Seq(groupingAttributes.map(SortOrder(_, Ascending))) - - override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => - val grouped = GroupedIterator(iter, groupingAttributes, child.output) - val groupKeyEncoder = kEncoder.bind(groupingAttributes) - val groupDataEncoder = tEncoder.bind(child.output) - - grouped.flatMap { case (key, rowIter) => - val result = func( - groupKeyEncoder.fromRow(key), - rowIter.map(groupDataEncoder.fromRow)) - result.map(uEncoder.toRow) - } - } - } -} - -/** - * Co-groups the data from left and right children, and calls the function with each group and 2 - * iterators containing all elements in the group from left and right side. - * The result of this function is encoded and flattened before being output. - */ -case class CoGroup[Key, Left, Right, Result]( - func: (Key, Iterator[Left], Iterator[Right]) => TraversableOnce[Result], - keyEnc: ExpressionEncoder[Key], - leftEnc: ExpressionEncoder[Left], - rightEnc: ExpressionEncoder[Right], - resultEnc: ExpressionEncoder[Result], - output: Seq[Attribute], - leftGroup: Seq[Attribute], - rightGroup: Seq[Attribute], - left: SparkPlan, - right: SparkPlan) extends BinaryNode { - override def producedAttributes: AttributeSet = outputSet - - override def requiredChildDistribution: Seq[Distribution] = - ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil - - override def requiredChildOrdering: Seq[Seq[SortOrder]] = - leftGroup.map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil - - override protected def doExecute(): RDD[InternalRow] = { - left.execute().zipPartitions(right.execute()) { (leftData, rightData) => - val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) - val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) - val boundKeyEnc = keyEnc.bind(leftGroup) - val boundLeftEnc = leftEnc.bind(left.output) - val boundRightEnc = rightEnc.bind(right.output) - - new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { - case (key, leftResult, rightResult) => - val result = func( - boundKeyEnc.fromRow(key), - leftResult.map(boundLeftEnc.fromRow), - rightResult.map(boundRightEnc.fromRow)) - result.map(resultEnc.toRow) - } - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 1d6290e027f3d..da9320ffb61c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} */ private[sql] object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => pruneFilterProjectRaw( l, projects, @@ -49,14 +49,14 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => pruneFilterProject( l, projects, filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _)) => pruneFilterProject( l, projects, @@ -64,7 +64,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil // Scanning partitioned HadoopFsRelation - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _)) + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _, _)) if t.partitionSpec.partitionColumns.nonEmpty => // We divide the filter expressions into 3 parts val partitionColumns = AttributeSet( @@ -118,7 +118,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { ).getOrElse(scan) :: Nil // Scanning non-partitioned HadoopFsRelation - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: HadoopFsRelation, _, _)) => // See buildPartitionedTableScan for the reason that we need to create a shard // broadcast HadoopConf. val sharedHadoopConf = SparkHadoopUtil.get.conf @@ -130,16 +130,16 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { filters, (a, f) => t.buildInternalScan(a.map(_.name).toArray, f, t.paths, confBroadcast)) :: Nil - case l @ LogicalRelation(baseRelation: TableScan, _) => + case l @ LogicalRelation(baseRelation: TableScan, _, _) => execution.PhysicalRDD.createFromDataSource( l.output, toCatalystRDD(l, baseRelation.buildScan()), baseRelation) :: Nil - case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _), + case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _), part, query, overwrite, false) => + l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) => val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append execution.ExecutedCommand(InsertIntoHadoopFsRelation(t, query, mode)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 219dae88e515d..fa97f3d7199ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} @@ -30,7 +31,8 @@ import org.apache.spark.sql.sources.BaseRelation */ case class LogicalRelation( relation: BaseRelation, - expectedOutputAttributes: Option[Seq[Attribute]] = None) + expectedOutputAttributes: Option[Seq[Attribute]] = None, + metastoreTableIdentifier: Option[TableIdentifier] = None) extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { @@ -49,7 +51,7 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _) => relation == otherRelation && output == l.output + case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output case _ => false } @@ -58,7 +60,7 @@ case class LogicalRelation( } override def sameResult(otherPlan: LogicalPlan): Boolean = otherPlan match { - case LogicalRelation(otherRelation, _) => relation == otherRelation + case LogicalRelation(otherRelation, _, _) => relation == otherRelation case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala index 4f8524f4b967c..fc77529b7db32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.datasources import java.util.{Date, UUID} -import scala.collection.JavaConverters._ - import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter} @@ -30,10 +28,11 @@ import org.apache.spark._ import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.UnsafeKVExternalSorter import org.apache.spark.sql.sources.{HadoopFsRelation, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.types.{IntegerType, StructType, StringType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.SerializableConfiguration @@ -322,9 +321,12 @@ private[sql] class DynamicPartitionWriterContainer( spec => spec.sortColumnNames.map(c => inputSchema.find(_.name == c).get) } - private def bucketIdExpression: Option[Expression] = for { - BucketSpec(numBuckets, _, _) <- bucketSpec - } yield Pmod(new Murmur3Hash(bucketColumns), Literal(numBuckets)) + private def bucketIdExpression: Option[Expression] = bucketSpec.map { spec => + // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can + // guarantee the data distribution is same between shuffle and bucketed data source, which + // enables us to only shuffle one side when join a bucketed table and a normal one. + HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression + } // Expressions that given a partition key build a string like: col1=val/col2=val/... private def partitionStringExpression: Seq[Expression] = { @@ -341,73 +343,8 @@ private[sql] class DynamicPartitionWriterContainer( } } - private def getBucketIdFromKey(key: InternalRow): Option[Int] = { - if (bucketSpec.isDefined) { - Some(key.getInt(partitionColumns.length)) - } else { - None - } - } - - private def sameBucket(key1: UnsafeRow, key2: UnsafeRow): Boolean = { - val bucketIdIndex = partitionColumns.length - if (key1.getInt(bucketIdIndex) != key2.getInt(bucketIdIndex)) { - false - } else { - var i = partitionColumns.length - 1 - while (i >= 0) { - val dt = partitionColumns(i).dataType - if (key1.get(i, dt) != key2.get(i, dt)) return false - i -= 1 - } - true - } - } - - private def sortBasedWrite( - sorter: UnsafeKVExternalSorter, - iterator: Iterator[InternalRow], - getSortingKey: UnsafeProjection, - getOutputRow: UnsafeProjection, - getPartitionString: UnsafeProjection, - outputWriters: java.util.HashMap[InternalRow, OutputWriter]): Unit = { - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - - logInfo(s"Sorting complete. Writing out partition files one at a time.") - - val needNewWriter: (UnsafeRow, UnsafeRow) => Boolean = if (sortColumns.isEmpty) { - (key1, key2) => key1 != key2 - } else { - (key1, key2) => key1 == null || !sameBucket(key1, key2) - } - - val sortedIterator = sorter.sortedIterator() - var currentKey: UnsafeRow = null - var currentWriter: OutputWriter = null - try { - while (sortedIterator.next()) { - if (needNewWriter(currentKey, sortedIterator.getKey)) { - if (currentWriter != null) { - currentWriter.close() - } - currentKey = sortedIterator.getKey.copy() - logDebug(s"Writing partition: $currentKey") - - // Either use an existing file from before, or open a new one. - currentWriter = outputWriters.remove(currentKey) - if (currentWriter == null) { - currentWriter = newOutputWriter(currentKey, getPartitionString) - } - } - - currentWriter.writeInternal(sortedIterator.getValue) - } - } finally { - if (currentWriter != null) { currentWriter.close() } - } + private def getBucketIdFromKey(key: InternalRow): Option[Int] = bucketSpec.map { _ => + key.getInt(partitionColumns.length) } /** @@ -435,22 +372,18 @@ private[sql] class DynamicPartitionWriterContainer( } def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - val outputWriters = new java.util.HashMap[InternalRow, OutputWriter] executorSideSetup(taskContext) - var outputWritersCleared = false - // We should first sort by partition columns, then bucket id, and finally sorting columns. - val getSortingKey = - UnsafeProjection.create(partitionColumns ++ bucketIdExpression ++ sortColumns, inputSchema) - - val sortingKeySchema = if (bucketSpec.isEmpty) { - StructType.fromAttributes(partitionColumns) - } else { // If it's bucketed, we should also consider bucket id as part of the key. - val fields = StructType.fromAttributes(partitionColumns) - .add("bucketId", IntegerType, nullable = false) ++ StructType.fromAttributes(sortColumns) - StructType(fields) - } + val sortingExpressions: Seq[Expression] = partitionColumns ++ bucketIdExpression ++ sortColumns + + val getSortingKey = UnsafeProjection.create(sortingExpressions, inputSchema) + + val sortingKeySchema = StructType(sortingExpressions.map { + case a: Attribute => StructField(a.name, a.dataType, a.nullable) + // The sorting expressions are all `Attribute` except bucket id. + case _ => StructField("bucketId", IntegerType, nullable = false) + }) // Returns the data columns to be written given an input row val getOutputRow = UnsafeProjection.create(dataColumns, inputSchema) @@ -461,54 +394,49 @@ private[sql] class DynamicPartitionWriterContainer( // If anything below fails, we should abort the task. try { - // If there is no sorting columns, we set sorter to null and try the hash-based writing first, - // and fill the sorter if there are too many writers and we need to fall back on sorting. - // If there are sorting columns, then we have to sort the data anyway, and no need to try the - // hash-based writing first. - var sorter: UnsafeKVExternalSorter = if (sortColumns.nonEmpty) { - new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) + // Sorts the data before write, so that we only need one writer at the same time. + // TODO: inject a local sort operator in planning. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(dataColumns), + SparkEnv.get.blockManager, + TaskContext.get().taskMemoryManager().pageSizeBytes) + + while (iterator.hasNext) { + val currentRow = iterator.next() + sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) + } + + logInfo(s"Sorting complete. Writing out partition files one at a time.") + + val getBucketingKey: InternalRow => InternalRow = if (sortColumns.isEmpty) { + identity } else { - null + UnsafeProjection.create(sortingExpressions.dropRight(sortColumns.length).zipWithIndex.map { + case (expr, ordinal) => BoundReference(ordinal, expr.dataType, expr.nullable) + }) } - while (iterator.hasNext && sorter == null) { - val inputRow = iterator.next() - // When we reach here, the `sortColumns` must be empty, so the sorting key is hashing key. - val currentKey = getSortingKey(inputRow) - var currentWriter = outputWriters.get(currentKey) - - if (currentWriter == null) { - if (outputWriters.size < maxOpenFiles) { + + val sortedIterator = sorter.sortedIterator() + var currentKey: UnsafeRow = null + var currentWriter: OutputWriter = null + try { + while (sortedIterator.next()) { + val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] + if (currentKey != nextKey) { + if (currentWriter != null) { + currentWriter.close() + } + currentKey = nextKey.copy() + logDebug(s"Writing partition: $currentKey") + currentWriter = newOutputWriter(currentKey, getPartitionString) - outputWriters.put(currentKey.copy(), currentWriter) - currentWriter.writeInternal(getOutputRow(inputRow)) - } else { - logInfo(s"Maximum partitions reached, falling back on sorting.") - sorter = new UnsafeKVExternalSorter( - sortingKeySchema, - StructType.fromAttributes(dataColumns), - SparkEnv.get.blockManager, - TaskContext.get().taskMemoryManager().pageSizeBytes) - sorter.insertKV(currentKey, getOutputRow(inputRow)) } - } else { - currentWriter.writeInternal(getOutputRow(inputRow)) - } - } - // If the sorter is not null that means that we reached the maxFiles above and need to finish - // using external sort, or there are sorting columns and we need to sort the whole data set. - if (sorter != null) { - sortBasedWrite( - sorter, - iterator, - getSortingKey, - getOutputRow, - getPartitionString, - outputWriters) + currentWriter.writeInternal(sortedIterator.getValue) + } + } finally { + if (currentWriter != null) { currentWriter.close() } } commitTask() @@ -518,31 +446,5 @@ private[sql] class DynamicPartitionWriterContainer( abortTask() throw new SparkException("Task failed while writing rows.", cause) } - - def clearOutputWriters(): Unit = { - if (!outputWritersCleared) { - outputWriters.asScala.values.foreach(_.close()) - outputWriters.clear() - outputWritersCleared = true - } - } - - def commitTask(): Unit = { - try { - clearOutputWriters() - super.commitTask() - } catch { - case cause: Throwable => - throw new RuntimeException("Failed to commit task", cause) - } - } - - def abortTask(): Unit = { - try { - clearOutputWriters() - } finally { - super.abortTask() - } - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala index 82287c8967134..9976829638d70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/bucket.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.mapreduce.TaskAttemptContext + import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.sources.{OutputWriter, OutputWriterFactory, HadoopFsRelationProvider, HadoopFsRelation} +import org.apache.spark.sql.sources.{HadoopFsRelation, HadoopFsRelationProvider, OutputWriter, OutputWriterFactory} import org.apache.spark.sql.types.StructType /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala index 2e3fe3da15389..b2f5c1e96421d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonParser.scala @@ -90,7 +90,7 @@ object JacksonParser { DateTimeUtils.stringToTime(parser.getText).getTime * 1000L case (VALUE_NUMBER_INT, TimestampType) => - parser.getLongValue * 1000L + parser.getLongValue * 1000000L case (_, StringType) => val writer = new ByteArrayOutputStream() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index ca8d010090401..991a5d5aef2db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.parquet import java.net.URI -import java.util.logging.{Logger => JLogger} import java.util.{List => JList} +import java.util.logging.{Logger => JLogger} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -32,24 +32,24 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.task.JobContextImpl +import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.metadata.CompressionCodecName import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType -import org.apache.parquet.{Log => ApacheParquetLog} import org.slf4j.bridge.SLF4JBridgeHandler +import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{RDD, SqlNewHadoopPartition, SqlNewHadoopRDD} import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.LegacyTypeStringParser -import org.apache.spark.sql.catalyst.{InternalRow, SqlParser, TableIdentifier} import org.apache.spark.sql.execution.datasources.{PartitionSpec, _} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} -import org.apache.spark.{Logging, Partition => SparkPartition, SparkException} private[sql] class DefaultSource extends BucketedHadoopFsRelationProvider with DataSourceRegister { @@ -147,12 +147,6 @@ private[sql] class ParquetRelation( .get(ParquetRelation.METASTORE_SCHEMA) .map(DataType.fromJson(_).asInstanceOf[StructType]) - // If this relation is converted from a Hive metastore table, this method returns the name of the - // original Hive metastore table. - private[sql] def metastoreTableName: Option[TableIdentifier] = { - parameters.get(ParquetRelation.METASTORE_TABLE_NAME).map(SqlParser.parseTableIdentifier) - } - private lazy val metadataCache: MetadataCache = { val meta = new MetadataCache meta.refresh() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index d484403d1c641..dd3e66d8a9434 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{AnalysisException, SaveMode, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{RowOrdering, Alias, Attribute, Cast} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, RowOrdering} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -61,7 +61,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation or HadoopFsRelation. case i @ InsertIntoTable( - l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _), _, child, _, _) => { + l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _), _, child, _, _) => // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -70,7 +70,6 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { s"statement generates the same number of columns as its schema.") } castAndRenameChildOutput(i, l.output, child) - } } /** If necessary, cast data types and rename fields to the expected types and names. */ @@ -108,14 +107,15 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation, _), partition, query, overwrite, ifNotExists) => + l @ LogicalRelation(t: InsertableRelation, _, _), + partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") } else { // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(t)) { failAnalysis( @@ -126,7 +126,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case logical.InsertIntoTable( - LogicalRelation(r: HadoopFsRelation, _), part, query, overwrite, _) => + LogicalRelation(r: HadoopFsRelation, _, _), part, query, overwrite, _) => // We need to make sure the partition columns specified by users do match partition // columns of the relation. val existingPartitionColumns = r.partitionColumns.fieldNames.toSet @@ -145,7 +145,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(r)) { failAnalysis( @@ -173,10 +173,10 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => EliminateSubQueries(catalog.lookupRelation(c.tableIdent)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). - case l @ LogicalRelation(dest: BaseRelation, _) => + case l @ LogicalRelation(dest: BaseRelation, _, _) => // Get all input data source relations of the query. val srcRelations = c.child.collect { - case LogicalRelation(src: BaseRelation, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(dest)) { failAnalysis( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala new file mode 100644 index 0000000000000..2acca1743cbb9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection, GenerateUnsafeRowJoiner} +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.types.ObjectType + +/** + * Helper functions for physical operators that work with user defined objects. + */ +trait ObjectOperator extends SparkPlan { + def generateToObject(objExpr: Expression, inputSchema: Seq[Attribute]): InternalRow => Any = { + val objectProjection = GenerateSafeProjection.generate(objExpr :: Nil, inputSchema) + (i: InternalRow) => objectProjection(i).get(0, objExpr.dataType) + } + + def generateToRow(serializer: Seq[Expression]): Any => InternalRow = { + val outputProjection = if (serializer.head.dataType.isInstanceOf[ObjectType]) { + GenerateSafeProjection.generate(serializer) + } else { + GenerateUnsafeProjection.generate(serializer) + } + val inputType = serializer.head.collect { case b: BoundReference => b.dataType }.head + val outputRow = new SpecificMutableRow(inputType :: Nil) + (o: Any) => { + outputRow(0) = o + outputProjection(outputRow) + } + } +} + +/** + * Applies the given function to each input row and encodes the result. + */ +case class MapPartitions( + func: Iterator[Any] => Iterator[Any], + input: Expression, + serializer: Seq[NamedExpression], + child: SparkPlan) extends UnaryNode with ObjectOperator { + override def output: Seq[Attribute] = serializer.map(_.toAttribute) + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { iter => + val getObject = generateToObject(input, child.output) + val outputObject = generateToRow(serializer) + func(iter.map(getObject)).map(outputObject) + } + } +} + +/** + * Applies the given function to each input row, appending the encoded result at the end of the row. + */ +case class AppendColumns( + func: Any => Any, + input: Expression, + serializer: Seq[NamedExpression], + child: SparkPlan) extends UnaryNode with ObjectOperator { + + override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) + + private def newColumnSchema = serializer.map(_.toAttribute).toStructType + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { iter => + val getObject = generateToObject(input, child.output) + val combiner = GenerateUnsafeRowJoiner.create(child.schema, newColumnSchema) + val outputObject = generateToRow(serializer) + + iter.map { row => + val newColumns = outputObject(func(getObject(row))) + + // This operates on the assumption that we always serialize the result... + combiner.join(row.asInstanceOf[UnsafeRow], newColumns.asInstanceOf[UnsafeRow]): InternalRow + } + } + } +} + +/** + * Groups the input rows together and calls the function with each group and an iterator containing + * all elements in the group. The result of this function is encoded and flattened before + * being output. + */ +case class MapGroups( + func: (Any, Iterator[Any]) => TraversableOnce[Any], + keyObject: Expression, + input: Expression, + serializer: Seq[NamedExpression], + groupingAttributes: Seq[Attribute], + child: SparkPlan) extends UnaryNode with ObjectOperator { + + override def output: Seq[Attribute] = serializer.map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(groupingAttributes) :: Nil + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(groupingAttributes.map(SortOrder(_, Ascending))) + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { iter => + val grouped = GroupedIterator(iter, groupingAttributes, child.output) + + val getKey = generateToObject(keyObject, groupingAttributes) + val getValue = generateToObject(input, child.output) + val outputObject = generateToRow(serializer) + + grouped.flatMap { case (key, rowIter) => + val result = func( + getKey(key), + rowIter.map(getValue)) + result.map(outputObject) + } + } + } +} + +/** + * Co-groups the data from left and right children, and calls the function with each group and 2 + * iterators containing all elements in the group from left and right side. + * The result of this function is encoded and flattened before being output. + */ +case class CoGroup( + func: (Any, Iterator[Any], Iterator[Any]) => TraversableOnce[Any], + keyObject: Expression, + leftObject: Expression, + rightObject: Expression, + serializer: Seq[NamedExpression], + leftGroup: Seq[Attribute], + rightGroup: Seq[Attribute], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with ObjectOperator { + + override def output: Seq[Attribute] = serializer.map(_.toAttribute) + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftGroup) :: ClusteredDistribution(rightGroup) :: Nil + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + leftGroup.map(SortOrder(_, Ascending)) :: rightGroup.map(SortOrder(_, Ascending)) :: Nil + + override protected def doExecute(): RDD[InternalRow] = { + left.execute().zipPartitions(right.execute()) { (leftData, rightData) => + val leftGrouped = GroupedIterator(leftData, leftGroup, left.output) + val rightGrouped = GroupedIterator(rightData, rightGroup, right.output) + + val getKey = generateToObject(keyObject, leftGroup) + val getLeft = generateToObject(leftObject, left.output) + val getRight = generateToObject(rightObject, right.output) + val outputObject = generateToRow(serializer) + + new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { + case (key, leftResult, rightResult) => + val result = func( + getKey(key), + leftResult.map(getLeft), + rightResult.map(getRight)) + result.map(outputObject) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 592d79df3109a..b8ea2261e94e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -308,6 +308,46 @@ object functions extends LegacyFunctions { def countDistinct(columnName: String, columnNames: String*): Column = countDistinct(Column(columnName), columnNames.map(Column.apply) : _*) + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(column1: Column, column2: Column): Column = withAggregateFunction { + CovPopulation(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the population covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_pop(columnName1: String, columnName2: String): Column = { + covar_pop(Column(columnName1), Column(columnName2)) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(column1: Column, column2: Column): Column = withAggregateFunction { + CovSample(column1.expr, column2.expr) + } + + /** + * Aggregate function: returns the sample covariance for two columns. + * + * @group agg_funcs + * @since 2.0.0 + */ + def covar_samp(columnName1: String, columnName2: String): Column = { + covar_samp(Column(columnName1), Column(columnName2)) + } + /** * Aggregate function: returns the first value in a group. * @@ -1002,7 +1042,7 @@ object functions extends LegacyFunctions { * @since 1.4.0 */ def when(condition: Column, value: Any): Column = withExpr { - CaseWhen(Seq(condition.expr, lit(value).expr)) + CaseWhen(Seq((condition.expr, lit(value).expr))) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 040402ed48adb..a55c10ad60661 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -183,7 +183,6 @@ trait HadoopFsRelationProvider { partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation - // TODO: expose bucket API to users. private[sql] def createRelation( sqlContext: SQLContext, paths: Array[String], @@ -391,7 +390,6 @@ abstract class OutputWriterFactory extends Serializable { dataSchema: StructType, context: TaskAttemptContext): OutputWriter - // TODO: expose bucket API to users. private[sql] def newInstance( path: String, bucketId: Option[Int], @@ -481,7 +479,6 @@ abstract class HadoopFsRelation private[sql]( private var _partitionSpec: PartitionSpec = _ - // TODO: expose bucket API to users. private[sql] def bucketSpec: Option[BucketSpec] = None private class FileStatusCache { diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 8e0b2dbca4a98..ac1607ba3521a 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -237,8 +237,8 @@ public void testCrosstab() { DataFrame crosstab = df.stat().crosstab("a", "b"); String[] columnNames = crosstab.schema().fieldNames(); Assert.assertEquals("a_b", columnNames[0]); - Assert.assertEquals("1", columnNames[1]); - Assert.assertEquals("2", columnNames[2]); + Assert.assertEquals("2", columnNames[1]); + Assert.assertEquals("1", columnNames[2]); Row[] rows = crosstab.collect(); Arrays.sort(rows, crosstabRowComparator); Integer count = 1; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 9f8db39e33d7e..1a3df1b117b68 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -187,7 +187,7 @@ public String call(Integer key, Iterator values) throws Exception { } }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), mapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); Dataset flatMapped = grouped.flatMapGroups( new FlatMapGroupsFunction() { @@ -202,7 +202,7 @@ public Iterable call(Integer key, Iterator values) throws Except }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), flatMapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList())); Dataset> reduced = grouped.reduce(new ReduceFunction() { @Override @@ -212,8 +212,8 @@ public String call(String v1, String v2) throws Exception { }); Assert.assertEquals( - Arrays.asList(tuple2(1, "a"), tuple2(3, "foobar")), - reduced.collectAsList()); + asSet(tuple2(1, "a"), tuple2(3, "foobar")), + toSet(reduced.collectAsList())); List data2 = Arrays.asList(2, 6, 10); Dataset ds2 = context.createDataset(data2, Encoders.INT()); @@ -245,7 +245,7 @@ public Iterable call( }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a#2", "3foobar#6", "5#10"), cogrouped.collectAsList()); + Assert.assertEquals(asSet("1a#2", "3foobar#6", "5#10"), toSet(cogrouped.collectAsList())); } @Test @@ -268,7 +268,7 @@ public String call(Integer key, Iterator data) throws Exception { }, Encoders.STRING()); - Assert.assertEquals(Arrays.asList("1a", "3foobar"), mapped.collectAsList()); + Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); } @Test @@ -290,9 +290,7 @@ public void testSetOperation() { List data = Arrays.asList("abc", "abc", "xyz"); Dataset ds = context.createDataset(data, Encoders.STRING()); - Assert.assertEquals( - Arrays.asList("abc", "xyz"), - sort(ds.distinct().collectAsList().toArray(new String[0]))); + Assert.assertEquals(asSet("abc", "xyz"), toSet(ds.distinct().collectAsList())); List data2 = Arrays.asList("xyz", "foo", "foo"); Dataset ds2 = context.createDataset(data2, Encoders.STRING()); @@ -302,16 +300,23 @@ public void testSetOperation() { Dataset unioned = ds.union(ds2); Assert.assertEquals( - Arrays.asList("abc", "abc", "foo", "foo", "xyz", "xyz"), - sort(unioned.collectAsList().toArray(new String[0]))); + Arrays.asList("abc", "abc", "xyz", "xyz", "foo", "foo"), + unioned.collectAsList()); Dataset subtracted = ds.subtract(ds2); Assert.assertEquals(Arrays.asList("abc", "abc"), subtracted.collectAsList()); } - private > List sort(T[] data) { - Arrays.sort(data); - return Arrays.asList(data); + private Set toSet(List records) { + Set set = new HashSet(); + for (T record : records) { + set.add(record); + } + return set; + } + + private Set asSet(T... records) { + return toSet(Arrays.asList(records)); } @Test diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 983dfbdedeefe..d6c140dfea9ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1083,17 +1083,20 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // Walk each partition and verify that it is sorted descending and does not contain all // the values. df4.rdd.foreachPartition { p => - var previousValue: Int = -1 - var allSequential: Boolean = true - p.foreach { r => - val v: Int = r.getInt(1) - if (previousValue != -1) { - if (previousValue < v) throw new SparkException("Partition is not ordered.") - if (v + 1 != previousValue) allSequential = false + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v } - previousValue = v + if (allSequential) throw new SparkException("Partition should not be globally ordered") } - if (allSequential) throw new SparkException("Partition should not be globally ordered") } // Distribute and order by with multiple order bys diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 3a283a4e1f610..848f1af65508b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -27,7 +27,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("persist and unpersist") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS().select(expr("_2 + 1").as[Int]) val cached = ds.cache() // count triggers the caching action. It should not throw. cached.count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 53b5f45c2d4a6..b69bb21db532b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -26,11 +26,13 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +case class OtherTuple(_1: String, _2: Int) + class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("toDS") { - val data = Seq(("a", 1) , ("b", 2), ("c", 3)) + val data = Seq(("a", 1), ("b", 2), ("c", 3)) checkAnswer( data.toDS(), data: _*) @@ -87,7 +89,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("as case class / collect") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("a", "b").as[ClassData] checkAnswer( ds, ClassData("a", 1), ClassData("b", 2), ClassData("c", 3)) @@ -105,12 +107,22 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("map") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.map(v => (v._1, v._2 + 1)), ("a", 2), ("b", 3), ("c", 4)) } + test("map with type change") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + + checkAnswer( + ds.map(identity[(String, Int)]) + .as[OtherTuple] + .map(identity[OtherTuple]), + OtherTuple("a", 1), OtherTuple("b", 2), OtherTuple("c", 3)) + } + test("map and group by with class data") { // We inject a group by here to make sure this test case is future proof // when we implement better pipelining and local execution mode. @@ -124,14 +136,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select(expr("_2 + 1").as[Int]), 2, 3, 4) } test("select 2") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -140,7 +152,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and tuple") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -149,7 +161,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.select( expr("_1").as[String], @@ -158,7 +170,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("select 2, primitive and class, fields reordered") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDecoding( ds.select( expr("_1").as[String], @@ -167,28 +179,28 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("filter") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkAnswer( ds.filter(_._1 == "b"), ("b", 2)) } test("foreach") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreach(v => acc += v._2) assert(acc.value == 6) } test("foreachPartition") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() val acc = sparkContext.accumulator(0) ds.foreachPartition(_.foreach(v => acc += v._2)) assert(acc.value == 6) } test("reduce") { - val ds = Seq(("a", 1) , ("b", 2), ("c", 3)).toDS() + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() assert(ds.reduce((a, b) => ("sum", a._2 + b._2)) == ("sum", 6)) } @@ -456,8 +468,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { implicit val kryoEncoder = Encoders.kryo[KryoData] val ds = Seq(KryoData(1), KryoData(2)).toDS() - assert(ds.groupBy(p => p).count().collect().toSeq == - Seq((KryoData(1), 1L), (KryoData(2), 1L))) + assert(ds.groupBy(p => p).count().collect().toSet == + Set((KryoData(1), 1L), (KryoData(2), 1L))) } test("Kryo encoder self join") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 84003a0952f1d..c7bdf1d382ad9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -196,10 +196,10 @@ abstract class QueryTest extends PlanTest { val logicalPlan = df.queryExecution.analyzed // bypass some cases that we can't handle currently. logicalPlan.transform { - case _: MapPartitions[_, _] => return - case _: MapGroups[_, _, _] => return - case _: AppendColumns[_, _] => return - case _: CoGroup[_, _, _, _] => return + case _: MapPartitions => return + case _: MapGroups => return + case _: AppendColumns => return + case _: CoGroup => return case _: LogicalRelation => return }.transformAllExpressions { case a: ImperativeAggregate => return diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala index 1994dacfc4dfa..14b9448d260f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLContextSuite.scala @@ -18,9 +18,15 @@ package org.apache.spark.sql import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ + object DummyRule extends Rule[LogicalPlan] { + def apply(p: LogicalPlan): LogicalPlan = p + } + test("getOrCreate instantiates SQLContext") { val sqlContext = SQLContext.getOrCreate(sc) assert(sqlContext != null, "SQLContext.getOrCreate returned null") @@ -65,4 +71,10 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext{ session2.sql("select myadd(1, 2)").explain() } } + + test("Catalyst optimization passes are modifiable at runtime") { + val sqlContext = SQLContext.getOrCreate(sc) + sqlContext.experimental.extraOptimizations = Seq(DummyRule) + assert(sqlContext.optimizer.batches.flatMap(_.rules).contains(DummyRule)) + } } 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 5de0979606b88..03d67c4e91f7f 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 @@ -806,7 +806,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") - sql("SELECT DISTINCT n FROM lowerCaseData") + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n ASC") .limit(2) .registerTempTable("subset2") checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala index b3b6b7df0c1d1..860e07c68cef1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala @@ -83,9 +83,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { val doubleNumber: Double = 1.7976931348623157E308d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber * 1000L)), enforceCorrectType(intNumber, TimestampType)) - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong)), + checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), @@ -206,7 +206,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { StructType( StructField("f1", IntegerType, true) :: StructField("f2", IntegerType, true) :: Nil), - StructType(StructField("f1", LongType, true) :: Nil) , + StructType(StructField("f1", LongType, true) :: Nil), StructType( StructField("f1", LongType, true) :: StructField("f2", IntegerType, true) :: Nil)) @@ -1465,4 +1465,17 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } + test("Casting long as timestamp") { + withTempTable("jsonTable") { + val schema = (new StructType).add("ts", TimestampType) + val jsonDF = sqlContext.read.schema(schema).json(timestampAsLong) + + jsonDF.registerTempTable("jsonTable") + + checkAnswer( + sql("select ts from jsonTable"), + Row(java.sql.Timestamp.valueOf("2016-01-02 03:04:05")) + ) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index cb61f7eeca0de..a0836058d3c74 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -205,6 +205,10 @@ private[json] trait TestJsonData { """{"b": [{"c": {}}]}""" :: """]""" :: Nil) + def timestampAsLong: RDD[String] = + sqlContext.sparkContext.parallelize( + """{"ts":1451732645}""" :: Nil) + lazy val singleRow: RDD[String] = sqlContext.sparkContext.parallelize("""{"a":123}""" :: Nil) def empty: RDD[String] = sqlContext.sparkContext.parallelize(Seq[String]()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 587aa5fd30d2d..97c5313f0feff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -59,7 +59,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[ParquetRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: ParquetRelation, _)) => + case PhysicalOperation(_, filters, LogicalRelation(relation: ParquetRelation, _, _)) => maybeRelation = Some(relation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 0feb945fbb79a..3d1677bed4770 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -563,7 +563,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = sqlContext.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: ParquetRelation, _) => + case LogicalRelation(relation: ParquetRelation, _, _) => assert(relation.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala index cab6abde6da23..ae95b50e1ee76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadBenchmark.scala @@ -21,9 +21,9 @@ import java.io.File import scala.collection.JavaConverters._ import scala.util.Try +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.{SQLConf, SQLContext} import org.apache.spark.util.{Benchmark, Utils} -import org.apache.spark.{SparkConf, SparkContext} /** * Benchmark to measure parquet read performance. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala new file mode 100644 index 0000000000000..e28153d12a354 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -0,0 +1,320 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.execution.vectorized.ColumnVector +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.Benchmark +import org.apache.spark.util.collection.BitSet + +/** + * Benchmark to low level memory access using different ways to manage buffers. + */ +object ColumnarBatchBenchmark { + + // This benchmark reads and writes an array of ints. + // TODO: there is a big (2x) penalty for a random access API for off heap. + // Note: carefully if modifying this code. It's hard to reason about the JIT. + def intAccess(iters: Long): Unit = { + val count = 8 * 1000 + + // Accessing a java array. + val javaArray = { i: Int => + val data = new Array[Int](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data(i) = i + i += 1 + } + i = 0 + while (i < count) { + sum += data(i) + i += 1 + } + } + } + + // Accessing ByteBuffers + val byteBufferUnsafe = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + Platform.putInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4, i) + i += 1 + } + i = 0 + while (i < count) { + sum += Platform.getInt(data.array(), Platform.BYTE_ARRAY_OFFSET + i * 4) + i += 1 + } + } + } + + // Accessing offheap byte buffers + val directByteBuffer = { i: Int => + val data = ByteBuffer.allocateDirect(count * 4).asIntBuffer() + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.put(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.get() + i += 1 + } + data.rewind() + } + } + + // Accessing ByteBuffer using the typed APIs + val byteBufferApi = { i: Int => + val data = ByteBuffer.allocate(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + data.putInt(i) + i += 1 + } + data.rewind() + i = 0 + while (i < count) { + sum += data.getInt() + i += 1 + } + data.rewind() + } + } + + // Using unsafe memory + val unsafeBuffer = { i: Int => + val data: Long = Platform.allocateMemory(count * 4) + var sum = 0L + for (n <- 0L until iters) { + var ptr = data + var i = 0 + while (i < count) { + Platform.putInt(null, ptr, i) + ptr += 4 + i += 1 + } + ptr = data + i = 0 + while (i < count) { + sum += Platform.getInt(null, ptr) + ptr += 4 + i += 1 + } + } + } + + // Access through the column API with on heap memory + val columnOnHeap = { i: Int => + val col = ColumnVector.allocate(count, IntegerType, false) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + } + + // Access through the column API with off heap memory + def columnOffHeap = { i: Int => { + val col = ColumnVector.allocate(count, IntegerType, true) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + col.putInt(i, i) + i += 1 + } + i = 0 + while (i < count) { + sum += col.getInt(i) + i += 1 + } + } + col.close + }} + + // Access by directly getting the buffer backing the column. + val columnOffheapDirect = { i: Int => + val col = ColumnVector.allocate(count, IntegerType, true) + var sum = 0L + for (n <- 0L until iters) { + var addr = col.valuesNativeAddress() + var i = 0 + while (i < count) { + Platform.putInt(null, addr, i) + addr += 4 + i += 1 + } + i = 0 + addr = col.valuesNativeAddress() + while (i < count) { + sum += Platform.getInt(null, addr) + addr += 4 + i += 1 + } + } + col.close + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOnheap = { i: Int => + val buffer = new Array[Byte](count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(buffer, Platform.BYTE_ARRAY_OFFSET + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + } + + // Access by going through a batch of unsafe rows. + val unsafeRowOffheap = { i: Int => + val buffer = Platform.allocateMemory(count * 16) + var sum = 0L + for (n <- 0L until iters) { + val row = new UnsafeRow(1) + var i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + row.setInt(0, i) + i += 1 + } + i = 0 + while (i < count) { + row.pointTo(null, buffer + i * 16, 16) + sum += row.getInt(0) + i += 1 + } + } + Platform.freeMemory(buffer) + } + + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Int Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Java Array 248.8 1317.04 1.00 X + ByteBuffer Unsafe 435.6 752.25 0.57 X + ByteBuffer API 1752.0 187.03 0.14 X + DirectByteBuffer 595.4 550.35 0.42 X + Unsafe Buffer 235.2 1393.20 1.06 X + Column(on heap) 189.8 1726.45 1.31 X + Column(off heap) 408.4 802.35 0.61 X + Column(off heap direct) 237.6 1379.12 1.05 X + UnsafeRow (on heap) 414.6 790.35 0.60 X + UnsafeRow (off heap) 487.2 672.58 0.51 X + */ + val benchmark = new Benchmark("Int Read/Write", count * iters) + benchmark.addCase("Java Array")(javaArray) + benchmark.addCase("ByteBuffer Unsafe")(byteBufferUnsafe) + benchmark.addCase("ByteBuffer API")(byteBufferApi) + benchmark.addCase("DirectByteBuffer")(directByteBuffer) + benchmark.addCase("Unsafe Buffer")(unsafeBuffer) + benchmark.addCase("Column(on heap)")(columnOnHeap) + benchmark.addCase("Column(off heap)")(columnOffHeap) + benchmark.addCase("Column(off heap direct)")(columnOffheapDirect) + benchmark.addCase("UnsafeRow (on heap)")(unsafeRowOnheap) + benchmark.addCase("UnsafeRow (off heap)")(unsafeRowOffheap) + benchmark.run() + } + + def booleanAccess(iters: Int): Unit = { + val count = 8 * 1024 + val benchmark = new Benchmark("Boolean Read/Write", iters * count) + benchmark.addCase("Bitset") { i: Int => { + val b = new BitSet(count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b.set(i) + i += 1 + } + i = 0 + while (i < count) { + if (b.get(i)) sum += 1 + i += 1 + } + } + }} + + benchmark.addCase("Byte Array") { i: Int => { + val b = new Array[Byte](count) + var sum = 0L + for (n <- 0L until iters) { + var i = 0 + while (i < count) { + if (i % 2 == 0) b(i) = 1; + i += 1 + } + i = 0 + while (i < count) { + if (b(i) == 1) sum += 1 + i += 1 + } + } + }} + /* + Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz + Boolean Read/Write: Avg Time(ms) Avg Rate(M/s) Relative Rate + ------------------------------------------------------------------------- + Bitset 895.88 374.54 1.00 X + Byte Array 578.96 579.56 1.55 X + */ + benchmark.run() + } + + def main(args: Array[String]): Unit = { + intAccess(1024 * 40) + booleanAccess(1024 * 40) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala new file mode 100644 index 0000000000000..305a83e3e45c9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.vectorized + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.{DoubleType, IntegerType, StructType} +import org.apache.spark.unsafe.Platform + +class ColumnarBatchSuite extends SparkFunSuite { + test("Null Apis") { + (false :: true :: Nil).foreach { offHeap => { + val reference = mutable.ArrayBuffer.empty[Boolean] + + val column = ColumnVector.allocate(1024, IntegerType, offHeap) + var idx = 0 + assert(column.anyNullsSet() == false) + + column.putNotNull(idx) + reference += false + idx += 1 + assert(column.anyNullsSet() == false) + + column.putNull(idx) + reference += true + idx += 1 + assert(column.anyNullsSet() == true) + assert(column.numNulls() == 1) + + column.putNulls(idx, 3) + reference += true + reference += true + reference += true + idx += 3 + assert(column.anyNullsSet() == true) + + column.putNotNulls(idx, 4) + reference += false + reference += false + reference += false + reference += false + idx += 4 + assert(column.anyNullsSet() == true) + assert(column.numNulls() == 4) + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getIsNull(v._2)) + if (offHeap) { + val addr = column.nullsNativeAddress() + assert(v._1 == (Platform.getByte(null, addr + v._2) == 1), "index=" + v._2) + } + } + column.close + }} + } + + test("Int Apis") { + (false :: true :: Nil).foreach { offHeap => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Int] + + val column = ColumnVector.allocate(1024, IntegerType, offHeap) + var idx = 0 + + val values = (1 :: 2 :: 3 :: 4 :: 5 :: Nil).toArray + column.putInts(idx, 2, values, 0) + reference += 1 + reference += 2 + idx += 2 + + column.putInts(idx, 3, values, 2) + reference += 3 + reference += 4 + reference += 5 + idx += 3 + + val littleEndian = new Array[Byte](8) + littleEndian(0) = 7 + littleEndian(1) = 1 + littleEndian(4) = 6 + littleEndian(6) = 1 + + column.putIntsLittleEndian(idx, 1, littleEndian, 4) + column.putIntsLittleEndian(idx + 1, 1, littleEndian, 0) + reference += 6 + (1 << 16) + reference += 7 + (1 << 8) + idx += 2 + + column.putIntsLittleEndian(idx, 2, littleEndian, 0) + reference += 7 + (1 << 8) + reference += 6 + (1 << 16) + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextInt() + column.putInt(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + column.putInts(idx, n, n + 1) + var i = 0 + while (i < n) { + reference += (n + 1) + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getInt(v._2), "Seed = " + seed + " Off Heap=" + offHeap) + if (offHeap) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getInt(null, addr + 4 * v._2)) + } + } + column.close + }} + } + + test("Double APIs") { + (false :: true :: Nil).foreach { offHeap => { + val seed = System.currentTimeMillis() + val random = new Random(seed) + val reference = mutable.ArrayBuffer.empty[Double] + + val column = ColumnVector.allocate(1024, DoubleType, offHeap) + var idx = 0 + + val values = (1.0 :: 2.0 :: 3.0 :: 4.0 :: 5.0 :: Nil).toArray + column.putDoubles(idx, 2, values, 0) + reference += 1.0 + reference += 2.0 + idx += 2 + + column.putDoubles(idx, 3, values, 2) + reference += 3.0 + reference += 4.0 + reference += 5.0 + idx += 3 + + val buffer = new Array[Byte](16) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET, 2.234) + Platform.putDouble(buffer, Platform.BYTE_ARRAY_OFFSET + 8, 1.123) + + column.putDoubles(idx, 1, buffer, 8) + column.putDoubles(idx + 1, 1, buffer, 0) + reference += 1.123 + reference += 2.234 + idx += 2 + + column.putDoubles(idx, 2, buffer, 0) + reference += 2.234 + reference += 1.123 + idx += 2 + + while (idx < column.capacity) { + val single = random.nextBoolean() + if (single) { + val v = random.nextDouble() + column.putDouble(idx, v) + reference += v + idx += 1 + } else { + val n = math.min(random.nextInt(column.capacity / 20), column.capacity - idx) + val v = random.nextDouble() + column.putDoubles(idx, n, v) + var i = 0 + while (i < n) { + reference += v + i += 1 + } + idx += n + } + } + + reference.zipWithIndex.foreach { v => + assert(v._1 == column.getDouble(v._2), "Seed = " + seed + " Off Heap=" + offHeap) + if (offHeap) { + val addr = column.valuesNativeAddress() + assert(v._1 == Platform.getDouble(null, addr + 8 * v._2)) + } + } + column.close + }} + } + + test("ColumnarBatch basic") { + (false :: true :: Nil).foreach { offHeap => { + val schema = new StructType() + .add("intCol", IntegerType) + .add("doubleCol", DoubleType) + .add("intCol2", IntegerType) + + val batch = ColumnarBatch.allocate(schema, offHeap) + assert(batch.numCols() == 3) + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.capacity() > 0) + assert(batch.rowIterator().hasNext == false) + + // Add a row [1, 1.1, NULL] + batch.column(0).putInt(0, 1) + batch.column(1).putDouble(0, 1.1) + batch.column(2).putNull(0) + batch.setNumRows(1) + + // Verify the results of the row. + assert(batch.numCols() == 3) + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 1) + assert(batch.rowIterator().hasNext == true) + assert(batch.rowIterator().hasNext == true) + + assert(batch.column(0).getInt(0) == 1) + assert(batch.column(0).getIsNull(0) == false) + assert(batch.column(1).getDouble(0) == 1.1) + assert(batch.column(1).getIsNull(0) == false) + assert(batch.column(2).getIsNull(0) == true) + + // Verify the iterator works correctly. + val it = batch.rowIterator() + assert(it.hasNext()) + val row = it.next() + assert(row.getInt(0) == 1) + assert(row.isNullAt(0) == false) + assert(row.getDouble(1) == 1.1) + assert(row.isNullAt(1) == false) + assert(row.isNullAt(2) == true) + assert(it.hasNext == false) + assert(it.hasNext == false) + + // Filter out the row. + row.markFiltered() + assert(batch.numRows() == 1) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Reset and add 3 throws + batch.reset() + assert(batch.numRows() == 0) + assert(batch.numValidRows() == 0) + assert(batch.rowIterator().hasNext == false) + + // Add rows [NULL, 2.2, 2], [3, NULL, 3], [4, 4.4, 4] + batch.column(0).putNull(0) + batch.column(1).putDouble(0, 2.2) + batch.column(2).putInt(0, 2) + + batch.column(0).putInt(1, 3) + batch.column(1).putNull(1) + batch.column(2).putInt(1, 3) + + batch.column(0).putInt(2, 4) + batch.column(1).putDouble(2, 4.4) + batch.column(2).putInt(2, 4) + batch.setNumRows(3) + + def rowEquals(x: InternalRow, y: Row): Unit = { + assert(x.isNullAt(0) == y.isNullAt(0)) + if (!x.isNullAt(0)) assert(x.getInt(0) == y.getInt(0)) + + assert(x.isNullAt(1) == y.isNullAt(1)) + if (!x.isNullAt(1)) assert(x.getDouble(1) == y.getDouble(1)) + + assert(x.isNullAt(2) == y.isNullAt(2)) + if (!x.isNullAt(2)) assert(x.getInt(2) == y.getInt(2)) + } + // Verify + assert(batch.numRows() == 3) + assert(batch.numValidRows() == 3) + val it2 = batch.rowIterator() + rowEquals(it2.next(), Row(null, 2.2, 2)) + rowEquals(it2.next(), Row(3, null, 3)) + rowEquals(it2.next(), Row(4, 4.4, 4)) + assert(!it.hasNext) + + // Filter out some rows and verify + batch.markFiltered(1) + assert(batch.numValidRows() == 2) + val it3 = batch.rowIterator() + rowEquals(it3.next(), Row(null, 2.2, 2)) + rowEquals(it3.next(), Row(4, 4.4, 4)) + assert(!it.hasNext) + + batch.markFiltered(2) + assert(batch.numValidRows() == 1) + val it4 = batch.rowIterator() + rowEquals(it4.next(), Row(null, 2.2, 2)) + + batch.close + }} + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 398b8a1a661c6..7196b6dc13394 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -317,7 +317,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val table = caseInsensitiveContext.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _) => r + case LogicalRelation(r, _, _) => r }.get assert( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 03bc830df2034..f279b78f47c7d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -369,7 +369,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (counter != 0) { responseMsg += s", Fetched $counter row(s)" } - console.printInfo(responseMsg , null) + console.printInfo(responseMsg, null) // Destroy the driver to release all the locks. driver.destroy() } else { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index e598284ab22f8..ba3b26e1b7d49 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -347,7 +347,9 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { ) } - test("test jdbc cancel") { + // This test often hangs and then times out, leaving the hanging processes. + // Let's ignore it and improve the test. + ignore("test jdbc cancel") { withJdbcStatement { statement => val queries = Seq( "DROP TABLE IF EXISTS test_map", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala index 395c8bff53f47..b22f424981325 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -38,7 +38,7 @@ private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser { protected lazy val hiveQl: Parser[LogicalPlan] = restInput ^^ { - case statement => HiveQl.createPlan(statement.trim) + case statement => HiveQl.parsePlan(statement.trim) } protected lazy val dfs: Parser[LogicalPlan] = 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 43d84d507b20e..daaa5a5709bdc 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 @@ -184,7 +184,9 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive table.properties("spark.sql.sources.provider"), options) - LogicalRelation(resolvedRelation.relation) + LogicalRelation( + resolvedRelation.relation, + metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database)))) } } @@ -414,8 +416,8 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive alias match { // because hive use things like `_c0` to build the expanded text // currently we cannot support view from "create view v1(c1) as ..." - case None => Subquery(table.name, HiveQl.createPlan(viewText)) - case Some(aliasText) => Subquery(aliasText, HiveQl.createPlan(viewText)) + case None => Subquery(table.name, HiveQl.parsePlan(viewText)) + case Some(aliasText) => Subquery(aliasText, HiveQl.parsePlan(viewText)) } } else { MetastoreRelation(qualifiedTableName.database, qualifiedTableName.name, alias)(table)(hive) @@ -447,7 +449,7 @@ private[hive] class HiveMetastoreCatalog(val client: ClientInterface, hive: Hive partitionSpecInMetastore: Option[PartitionSpec]): Option[LogicalRelation] = { cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => + case logical @ LogicalRelation(parquetRelation: ParquetRelation, _, _) => // If we have the same paths, same schema, and same partition spec, // we will use the cached Parquet Relation. val useCached = 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 5b13dbe47370e..ca9ddf94c11a7 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 @@ -24,11 +24,12 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} +import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.parse.EximUtil import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe + import org.apache.spark.Logging import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions._ @@ -38,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.SparkQl import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.hive.client._ -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, AnalyzeTable, DropTable, HiveScriptIOSchema} +import org.apache.spark.sql.hive.execution.{AnalyzeTable, DropTable, HiveNativeCommand, HiveScriptIOSchema} import org.apache.spark.sql.types._ import org.apache.spark.sql.AnalysisException @@ -229,15 +230,16 @@ private[hive] object HiveQl extends SparkQl with Logging { CreateViewAsSelect(tableDesc, nodeToPlan(query), allowExist, replace, sql) } - protected override def createPlan( - sql: String, - node: ASTNode): LogicalPlan = { - if (nativeCommands.contains(node.text)) { - HiveNativeCommand(sql) - } else { - nodeToPlan(node) match { - case NativePlaceholder => HiveNativeCommand(sql) - case plan => plan + /** Creates LogicalPlan for a given SQL string. */ + override def parsePlan(sql: String): LogicalPlan = { + safeParse(sql, ParseDriver.parsePlan(sql, conf)) { ast => + if (nativeCommands.contains(ast.text)) { + HiveNativeCommand(sql) + } else { + nodeToPlan(ast) match { + case NativePlaceholder => HiveNativeCommand(sql) + case plan => plan + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index b8cced0b80969..087b0c087c111 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -26,11 +26,13 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.{Input, Output} +import com.google.common.base.Objects import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils} import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector @@ -45,6 +47,7 @@ private[hive] object HiveShim { // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs) val UNLIMITED_DECIMAL_PRECISION = 38 val UNLIMITED_DECIMAL_SCALE = 18 + val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro" /* * This function in hive-0.13 become private, but we have to do this to walkaround hive bug @@ -123,6 +126,26 @@ private[hive] object HiveShim { // for Serialization def this() = this(null) + override def hashCode(): Int = { + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody()) + } else { + functionClassName.hashCode() + } + } + + override def equals(other: Any): Boolean = other match { + case a: HiveFunctionWrapper if functionClassName == a.functionClassName => + // In case of udf macro, check to make sure they point to the same underlying UDF + if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) { + a.instance.asInstanceOf[GenericUDFMacro].getBody() == + instance.asInstanceOf[GenericUDFMacro].getBody() + } else { + true + } + case _ => false + } + @transient def deserializeObjectByKryo[T: ClassTag]( kryo: Kryo, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala index 1c910051faccf..e83b4bffff857 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/SQLBuilder.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.hive import java.util.concurrent.atomic.AtomicLong import org.apache.spark.Logging +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.optimizer.ProjectCollapsing import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation -import org.apache.spark.sql.{DataFrame, SQLContext} /** * A builder class used to convert a resolved logical plan into a SQL query string. Note that this @@ -135,13 +135,9 @@ class SQLBuilder(logicalPlan: LogicalPlan, sqlContext: SQLContext) extends Loggi rightSQL <- toSQL(right) } yield s"$leftSQL UNION ALL $rightSQL" - // ParquetRelation converted from Hive metastore table - case Subquery(alias, LogicalRelation(r: ParquetRelation, _)) => - // There seems to be a bug related to `ParquetConversions` analysis rule. The problem is - // that, the metastore database name and table name are not always propagated to converted - // `ParquetRelation` instances via data source options. Here we use subquery alias as a - // workaround. - Some(s"`$alias`") + // Persisted data source relation + case Subquery(alias, LogicalRelation(_, _, Some(TableIdentifier(table, Some(database))))) => + Some(s"`$database`.`$table`") case Subquery(alias, child) => toSQL(child).map(childSQL => s"($childSQL) AS $alias") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 612f01cda88ba..07a352873d087 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -216,7 +216,7 @@ case class CreateMetastoreDataSourceAsSelect( sqlContext, Some(query.schema.asNullable), partitionColumns, provider, optionsWithPath) val createdRelation = LogicalRelation(resolved.relation) EliminateSubQueries(sqlContext.catalog.lookupRelation(tableIdent)) match { - case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _) => + case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => if (l.relation != createdRelation.relation) { val errorDescription = s"Cannot append to table $tableName because the resolved relation does not " + 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 e76c18fa528f3..56cab1aee89df 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 @@ -22,21 +22,21 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.hive.ql.exec._ +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} +import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper -import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.hadoop.hive.serde2.objectinspector.{ConstantObjectInspector, ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{analysis, InternalRow} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.util.sequenceOption -import org.apache.spark.sql.catalyst.{InternalRow, analysis} import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.hive.client.ClientWrapper import org.apache.spark.sql.types._ 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 d26cb48479066..033746d42f557 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 @@ -37,8 +37,8 @@ import org.apache.spark.sql.catalyst.expressions.ExpressionInfo 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.hive.client.ClientWrapper +import org.apache.spark.sql.hive.execution.HiveNativeCommand import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index a2d283622ca52..14a466cfe9486 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -21,8 +21,8 @@ import scala.util.Try import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.catalyst.parser.ParseDriver import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.parser.ParseDriver import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -117,9 +117,8 @@ class ErrorPositionSuite extends QueryTest with TestHiveSingleton with BeforeAnd * @param token a unique token in the string that should be indicated by the exception */ def positionTest(name: String, query: String, token: String): Unit = { - def ast = ParseDriver.parse(query, hiveContext.conf) - def parseTree = - Try(quietly(ast.treeString)).getOrElse("") + def ast = ParseDriver.parsePlan(query, hiveContext.conf) + def parseTree = Try(quietly(ast.treeString)).getOrElse("") test(name) { val error = intercept[AnalysisException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala index f4a1a17422483..53d15c14cb3d5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveQlSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.hive.client.{ExternalTable, HiveColumn, HiveTable, M class HiveQlSuite extends SparkFunSuite with BeforeAndAfterAll { private def extractTableDesc(sql: String): (HiveTable, Boolean) = { - HiveQl.createPlan(sql).collect { + HiveQl.parsePlan(sql).collect { case CreateTableAsSelect(desc, child, allowExisting) => (desc, allowExisting) }.head } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index da7303c791064..40e9c9362cf5e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -154,8 +154,8 @@ class InsertIntoHiveTableSuite extends QueryTest with TestHiveSingleton with Bef } val expected = List( "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=2"::Nil, - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=3"::Nil , - "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil , + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=3"::Nil, + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil, "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=4"::Nil ) assert(listFolders(tmpDir, List()).sortBy(_.toString()) === expected.sortBy(_.toString)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala index 0e81acf532a03..0604d9f47c5da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/LogicalPlanToSQLSuite.scala @@ -17,13 +17,16 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SQLTestUtils class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { import testImplicits._ protected override def beforeAll(): Unit = { + sql("DROP TABLE IF EXISTS t0") + sql("DROP TABLE IF EXISTS t1") + sql("DROP TABLE IF EXISTS t2") sqlContext.range(10).write.saveAsTable("t0") sqlContext @@ -143,4 +146,14 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { ignore("distinct and non-distinct aggregation") { checkHiveQl("SELECT a, COUNT(DISTINCT b), COUNT(DISTINCT c), SUM(d) FROM t2 GROUP BY a") } + + test("persisted data source relations") { + Seq("orc", "json", "parquet").foreach { format => + val tableName = s"${format}_t0" + withTable(tableName) { + sqlContext.range(10).write.format(format).saveAsTable(tableName) + checkHiveQl(s"SELECT id FROM $tableName") + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 202851ae1366e..253f13c598520 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -571,7 +571,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: ParquetRelation, _) => // OK + case LogicalRelation(p: ParquetRelation, _, _) => // OK case _ => fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation]}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala index cf4a3fdd88806..a5e209ac9db3b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/SQLBuilderTest.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.hive +import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.{DataFrame, QueryTest} abstract class SQLBuilderTest extends QueryTest with TestHiveSingleton { protected def checkSQL(e: Expression, expectedSQL: String): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index 5550198c02fbf..76b36aa89182e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -807,6 +807,38 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te assert(math.abs(corr7 - 0.6633880657639323) < 1e-12) } + test("covariance: covar_pop and covar_samp") { + // non-trivial example. To reproduce in python, use: + // >>> import numpy as np + // >>> a = np.array(range(20)) + // >>> b = np.array([x * x - 2 * x + 3.5 for x in range(20)]) + // >>> np.cov(a, b, bias = 0)[0][1] + // 595.0 + // >>> np.cov(a, b, bias = 1)[0][1] + // 565.25 + val df = Seq.tabulate(20)(x => (1.0 * x, x * x - 2 * x + 3.5)).toDF("a", "b") + val cov_samp = df.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp - 595.0) < 1e-12) + + val cov_pop = df.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop - 565.25) < 1e-12) + + val df2 = Seq.tabulate(20)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + val cov_samp2 = df2.groupBy().agg(covar_samp("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_samp2 - 11564.0) < 1e-12) + + val cov_pop2 = df2.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(math.abs(cov_pop2 - 10985.799999999999) < 1e-12) + + // one row test + val df3 = Seq.tabulate(1)(x => (1 * x, x * x * x - 2)).toDF("a", "b") + val cov_samp3 = df3.groupBy().agg(covar_samp("a", "b")).collect()(0).get(0) + assert(cov_samp3 == null) + + val cov_pop3 = df3.groupBy().agg(covar_pop("a", "b")).collect()(0).getDouble(0) + assert(cov_pop3 == 0.0) + } + test("no aggregation function (SPARK-11486)") { val df = sqlContext.range(20).selectExpr("id", "repeat(id, 1) as s") .groupBy("s").count() 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 57358a07840e2..fd3339a66bec0 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 @@ -27,10 +27,10 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.datasources.DescribeCommand import org.apache.spark.sql.execution.{ExplainCommand, SetCommand} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.execution.datasources.DescribeCommand import org.apache.spark.sql.hive.{InsertIntoHiveTable => LogicalInsertIntoHiveTable, SQLBuilder} +import org.apache.spark.sql.hive.test.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index c5ff8825abd7f..dfe33ba8b0502 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -350,6 +350,13 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { sqlContext.dropTempTable("testUDF") } + test("Hive UDF in group by") { + Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1") + val count = sql("select date(cast(test_date as timestamp))" + + " from tab1 group by date(cast(test_date as timestamp))").count() + assert(count == 1) + } + test("SPARK-11522 select input_file_name from non-parquet table"){ withTempDir { tempDir => 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 593fac2c32817..f6c687aab7a1b 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 @@ -268,7 +268,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { def checkRelation(tableName: String, isDataSourceParquet: Boolean): Unit = { val relation = EliminateSubQueries(catalog.lookupRelation(TableIdentifier(tableName))) relation match { - case LogicalRelation(r: ParquetRelation, _) => + case LogicalRelation(r: ParquetRelation, _, _) => if (!isDataSourceParquet) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 5afc7e77ab775..c94e73c4aa300 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -42,7 +42,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[OrcRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: OrcRelation, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: OrcRelation, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) 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 2ceb836681901..ed544c638058c 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 @@ -282,7 +282,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: ParquetRelation, _) => // OK + case LogicalRelation(_: ParquetRelation, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + s"${classOf[ParquetRelation].getCanonicalName }") @@ -369,7 +369,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: ParquetRelation, _) => r + case r @ LogicalRelation(_: ParquetRelation, _, _) => r }.size } } @@ -378,7 +378,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { def collectParquetRelation(df: DataFrame): ParquetRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: ParquetRelation, _) => r + case LogicalRelation(r: ParquetRelation, _, _) => r }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$plan") } @@ -428,7 +428,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { // Converted test_parquet should be cached. catalog.cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case logical @ LogicalRelation(parquetRelation: ParquetRelation, _) => // OK + case logical @ LogicalRelation(parquetRelation: ParquetRelation, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index 579da0291f291..3ea9826544edb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.sources import java.io.File +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.functions._ -import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.{AnalysisException, QueryTest} class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { import testImplicits._ @@ -70,6 +71,8 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle } } + private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") + private def testBucketing( dataDir: File, source: String, @@ -82,27 +85,31 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle assert(groupedBucketFiles.size <= 8) for ((bucketId, bucketFiles) <- groupedBucketFiles) { - for (bucketFile <- bucketFiles) { - val df = sqlContext.read.format(source).load(bucketFile.getAbsolutePath) - .select((bucketCols ++ sortCols).map(col): _*) + for (bucketFilePath <- bucketFiles.map(_.getAbsolutePath)) { + val types = df.select((bucketCols ++ sortCols).map(col): _*).schema.map(_.dataType) + val columns = (bucketCols ++ sortCols).zip(types).map { + case (colName, dt) => col(colName).cast(dt) + } + val readBack = sqlContext.read.format(source).load(bucketFilePath).select(columns: _*) if (sortCols.nonEmpty) { - checkAnswer(df.sort(sortCols.map(col): _*), df.collect()) + checkAnswer(readBack.sort(sortCols.map(col): _*), readBack.collect()) } - val rows = df.select(bucketCols.map(col): _*).queryExecution.toRdd.map(_.copy()).collect() + val qe = readBack.select(bucketCols.map(col): _*).queryExecution + val rows = qe.toRdd.map(_.copy()).collect() + val getHashCode = UnsafeProjection.create( + HashPartitioning(qe.analyzed.output, 8).partitionIdExpression :: Nil, + qe.analyzed.output) for (row <- rows) { - assert(row.isInstanceOf[UnsafeRow]) - val actualBucketId = (row.hashCode() % 8 + 8) % 8 + val actualBucketId = getHashCode(row).getInt(0) assert(actualBucketId == bucketId) } } } } - private val df = (0 until 50).map(i => (i % 5, i % 13, i.toString)).toDF("i", "j", "k") - test("write bucketed data") { for (source <- Seq("parquet", "json", "orc")) { withTable("bucketed_table") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index efbf9988ddc13..3f9ecf6965e1d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@ -500,7 +500,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } val actualPaths = df.queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _) => + case LogicalRelation(relation: HadoopFsRelation, _, _) => relation.paths.toSet }.getOrElse { fail("Expect an FSBasedRelation, but none could be found") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index b186d297610e2..298cdc05acfa9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.Utils import org.apache.spark.streaming.scheduler.JobGenerator +import org.apache.spark.util.Utils private[streaming] class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) @@ -183,7 +183,7 @@ class CheckpointWriter( val executor = Executors.newFixedThreadPool(1) val compressionCodec = CompressionCodec.createCodec(conf) private var stopped = false - private var fs_ : FileSystem = _ + private var _fs: FileSystem = _ @volatile private var latestCheckpointTime: Time = null @@ -298,12 +298,12 @@ class CheckpointWriter( } private def fs = synchronized { - if (fs_ == null) fs_ = new Path(checkpointDir).getFileSystem(hadoopConf) - fs_ + if (_fs == null) _fs = new Path(checkpointDir).getFileSystem(hadoopConf) + _fs } private def reset() = synchronized { - fs_ = null + _fs = null } } @@ -370,8 +370,8 @@ object CheckpointReader extends Logging { } private[streaming] -class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) - extends ObjectInputStream(inputStream_) { +class ObjectInputStreamWithLoader(_inputStream: InputStream, loader: ClassLoader) + extends ObjectInputStream(_inputStream) { override def resolveClass(desc: ObjectStreamClass): Class[_] = { try { 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 ba509a1030af7..157ee92fd71b3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -58,9 +58,9 @@ import org.apache.spark.util.{AsynchronousListenerBus, CallSite, ShutdownHookMan * of the context by `stop()` or by an exception. */ class StreamingContext private[streaming] ( - sc_ : SparkContext, - cp_ : Checkpoint, - batchDur_ : Duration + _sc: SparkContext, + _cp: Checkpoint, + _batchDur: Duration ) extends Logging { /** @@ -126,18 +126,18 @@ class StreamingContext private[streaming] ( } - if (sc_ == null && cp_ == null) { + if (_sc == null && _cp == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") } - private[streaming] val isCheckpointPresent = (cp_ != null) + private[streaming] val isCheckpointPresent = (_cp != null) private[streaming] val sc: SparkContext = { - if (sc_ != null) { - sc_ + if (_sc != null) { + _sc } else if (isCheckpointPresent) { - SparkContext.getOrCreate(cp_.createSparkConf()) + SparkContext.getOrCreate(_cp.createSparkConf()) } else { throw new SparkException("Cannot create StreamingContext without a SparkContext") } @@ -154,13 +154,13 @@ class StreamingContext private[streaming] ( private[streaming] val graph: DStreamGraph = { if (isCheckpointPresent) { - cp_.graph.setContext(this) - cp_.graph.restoreCheckpointData() - cp_.graph + _cp.graph.setContext(this) + _cp.graph.restoreCheckpointData() + _cp.graph } else { - require(batchDur_ != null, "Batch duration for StreamingContext cannot be null") + require(_batchDur != null, "Batch duration for StreamingContext cannot be null") val newGraph = new DStreamGraph() - newGraph.setBatchDuration(batchDur_) + newGraph.setBatchDuration(_batchDur) newGraph } } @@ -169,15 +169,15 @@ class StreamingContext private[streaming] ( private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { - sc.setCheckpointDir(cp_.checkpointDir) - cp_.checkpointDir + sc.setCheckpointDir(_cp.checkpointDir) + _cp.checkpointDir } else { null } } private[streaming] val checkpointDuration: Duration = { - if (isCheckpointPresent) cp_.checkpointDuration else graph.batchDuration + if (isCheckpointPresent) _cp.checkpointDuration else graph.batchDuration } private[streaming] val scheduler = new JobScheduler(this) @@ -246,7 +246,7 @@ class StreamingContext private[streaming] ( } private[streaming] def initialCheckpoint: Checkpoint = { - if (isCheckpointPresent) cp_ else null + if (isCheckpointPresent) _cp else null } private[streaming] def getNewInputStreamId() = nextInputStreamId.getAndIncrement() @@ -460,7 +460,7 @@ class StreamingContext private[streaming] ( def binaryRecordsStream( directory: String, recordLength: Int): DStream[Array[Byte]] = withNamedScope("binary records stream") { - val conf = sc_.hadoopConfiguration + val conf = _sc.hadoopConfiguration conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat]( directory, FileInputDStream.defaultFilter: Path => Boolean, newFilesOnly = true, conf) 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 733147f63ea2e..a791a474c673d 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 @@ -101,7 +101,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of elements in a window over this DStream. windowDuration and slideDuration are as defined in * the window() operation. This is equivalent to window(windowDuration, slideDuration).count() */ - def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[jl.Long] = { + def countByWindow(windowDuration: Duration, slideDuration: Duration): JavaDStream[jl.Long] = { dstream.countByWindow(windowDuration, slideDuration) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index 953fe95177f02..8c9beccc2922c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -169,16 +169,6 @@ private[python] object PythonDStream { PythonTransformFunctionSerializer.register(ser) } - /** - * Update the port of callback client to `port` - */ - def updatePythonGatewayPort(gws: GatewayServer, port: Int): Unit = { - val cl = gws.getCallbackClient - val f = cl.getClass.getDeclaredField("port") - f.setAccessible(true) - f.setInt(cl, port) - } - /** * helper function for DStream.foreachRDD(), * cannot be `foreachRDD`, it will confusing py4j diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 695384deb32d7..b5f86fe7794fc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -25,8 +25,8 @@ import org.apache.spark.streaming.{StreamingContext, Time} /** * An input stream that always returns the same RDD on each timestep. Useful for testing. */ -class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) - extends InputDStream[T](ssc_) { +class ConstantInputDStream[T: ClassTag](_ssc: StreamingContext, rdd: RDD[T]) + extends InputDStream[T](_ssc) { require(rdd != null, "parameter rdd null is illegal, which will lead to NPE in the following transformation") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 3eff174c2b66c..a9ce1131ce0c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -39,7 +39,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) // in that batch's checkpoint data @transient private var timeToOldestCheckpointFileTime = new HashMap[Time, Time] - @transient private var fileSystem : FileSystem = null + @transient private var fileSystem: FileSystem = null protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index cb5b1f252e90c..1c2325409b53e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -73,13 +73,13 @@ import org.apache.spark.util.{SerializableConfiguration, TimeStampedHashMap, Uti */ private[streaming] class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( - ssc_ : StreamingContext, + _ssc: StreamingContext, directory: String, filter: Path => Boolean = FileInputDStream.defaultFilter, newFilesOnly: Boolean = true, conf: Option[Configuration] = None) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) - extends InputDStream[(K, V)](ssc_) { + extends InputDStream[(K, V)](_ssc) { private val serializableConfOpt = conf.map(new SerializableConfiguration(_)) @@ -128,8 +128,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( // Timestamp of the last round of finding files @transient private var lastNewFileFindingTime = 0L - @transient private var path_ : Path = null - @transient private var fs_ : FileSystem = null + @transient private var _path: Path = null + @transient private var _fs: FileSystem = null override def start() { } @@ -289,17 +289,17 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( } private def directoryPath: Path = { - if (path_ == null) path_ = new Path(directory) - path_ + if (_path == null) _path = new Path(directory) + _path } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) - fs_ + if (_fs == null) _fs = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) + _fs } private def reset() { - fs_ = null + _fs = null } @throws(classOf[IOException]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index d60f418e5c4de..76f6230f36226 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -38,10 +38,10 @@ import org.apache.spark.util.Utils * that requires running a receiver on the worker nodes, use * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream */ -abstract class InputDStream[T: ClassTag] (ssc_ : StreamingContext) - extends DStream[T](ssc_) { +abstract class InputDStream[T: ClassTag] (_ssc: StreamingContext) + extends DStream[T](_ssc) { private[streaming] var lastValidTime: Time = null diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 2442e4c01a0c0..e003ddb96c860 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -24,8 +24,8 @@ import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( - ssc_ : StreamingContext, - receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { + _ssc: StreamingContext, + receiver: Receiver[T]) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { receiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index ac73dca05a674..409c565380f06 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -38,11 +38,11 @@ import org.apache.spark.streaming.receiver.Receiver */ private[streaming] class RawInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](_ssc) with Logging { def getReceiver(): Receiver[T] = { new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 565b137228d00..49d8f14f4c390 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -35,11 +35,11 @@ import org.apache.spark.streaming.util.WriteAheadLogUtils * define [[getReceiver]] function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream * @tparam T Class type of the object of this stream */ -abstract class ReceiverInputDStream[T: ClassTag](ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { +abstract class ReceiverInputDStream[T: ClassTag](_ssc: StreamingContext) + extends InputDStream[T](_ssc) { /** * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index e70fc87c39d95..441477479167a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -31,12 +31,12 @@ import org.apache.spark.util.NextIterator private[streaming] class SocketInputDStream[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, host: String, port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends ReceiverInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](_ssc) { def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index ebbe139a2cdf8..fedffb23952a4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -31,7 +31,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)], partitioner: Partitioner, preservePartitioning: Boolean, - initialRDD : Option[RDD[(K, S)]] + initialRDD: Option[RDD[(K, S)]] ) extends DStream[(K, S)](parent.ssc) { super.persist(StorageLevel.MEMORY_ONLY_SER) @@ -43,7 +43,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( override val mustCheckpoint = true private [this] def computeUsingPreviousRDD ( - parentRDD : RDD[(K, V)], prevStateRDD : RDD[(K, S)]) = { + parentRDD: RDD[(K, V)], prevStateRDD: RDD[(K, S)]) = { // Define the function for the mapPartition operation on cogrouped RDD; // first map the cogrouped tuple to tuples of required type, // and then apply the update function @@ -98,7 +98,7 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator : Iterator[(K, Iterable[V])]) => { + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { updateFuncLocal (iterator.map (tuple => (tuple._1, tuple._2.toSeq, None))) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 639f4259e2e73..3376cd557d72f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -108,7 +108,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable def onStop() /** Override this to specify a preferred location (hostname). */ - def preferredLocation : Option[String] = None + def preferredLocation: Option[String] = None /** * Store a single item of received data to Spark's memory. @@ -257,11 +257,11 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable private var id: Int = -1 /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ - @transient private var _supervisor : ReceiverSupervisor = null + @transient private var _supervisor: ReceiverSupervisor = null /** Set the ID of the DStream that this receiver is associated with. */ - private[streaming] def setReceiverId(id_ : Int) { - id = id_ + private[streaming] def setReceiverId(_id: Int) { + id = _id } /** Attach Network Receiver executor to this receiver. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 4e5baebaae04b..4ccc905b275d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -25,7 +25,7 @@ import com.esotericsoftware.kryo.{Kryo, KryoSerializable} import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.SparkConf -import org.apache.spark.serializer.{KryoOutputObjectOutputBridge, KryoInputObjectInputBridge} +import org.apache.spark.serializer.{KryoInputObjectInputBridge, KryoOutputObjectOutputBridge} import org.apache.spark.streaming.util.OpenHashMapBasedStateMap._ import org.apache.spark.util.collection.OpenHashMap 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 9d296c6d3ef8b..25e7ae8262a5f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -186,7 +186,7 @@ class BasicOperationsSuite extends TestSuiteBase { val output = Seq(1 to 8, 101 to 108, 201 to 208) testOperation( input, - (s: DStream[Int]) => s.union(s.map(_ + 4)) , + (s: DStream[Int]) => s.union(s.map(_ + 4)), output ) } 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 4d04138da01f7..4a6b91fbc745e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.{Clock, ManualClock, MutableURLClassLoader, ResetSy * A input stream that records the times of restore() invoked */ private[streaming] -class CheckpointInputDStream(ssc_ : StreamingContext) extends InputDStream[Int](ssc_) { +class CheckpointInputDStream(_ssc: StreamingContext) extends InputDStream[Int](_ssc) { protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData override def start(): Unit = { } override def stop(): Unit = { } 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 4e56dfbd424b0..7bbbdebd9b19f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -200,12 +200,12 @@ object MasterFailureTest extends Logging { * the last expected output is generated. Finally, return */ private def runStreams[T: ClassTag]( - ssc_ : StreamingContext, + _ssc: StreamingContext, lastExpectedOutput: T, maxTimeToRun: Long ): Seq[T] = { - var ssc = ssc_ + var ssc = _ssc var totalTimeRan = 0L var isLastOutputGenerated = false var isTimedOut = false diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index ea32bbf95ce59..7a76cafc9a11c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.rdd.MapWithStateRDDRecord - import scala.collection.{immutable, mutable, Map} import scala.reflect.ClassTag import scala.util.Random import com.esotericsoftware.kryo.{Kryo, KryoSerializable} -import com.esotericsoftware.kryo.io.{Output, Input} +import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer._ +import org.apache.spark.streaming.rdd.MapWithStateRDDRecord import org.apache.spark.streaming.util.{EmptyStateMap, OpenHashMapBasedStateMap, StateMap} class StateMapSuite extends SparkFunSuite { @@ -281,7 +280,7 @@ class StateMapSuite extends SparkFunSuite { testSerialization(new KryoSerializer(conf), map, msg) } - private def testSerialization[T : ClassTag]( + private def testSerialization[T: ClassTag]( serializer: Serializer, map: OpenHashMapBasedStateMap[T, T], msg: String): OpenHashMapBasedStateMap[T, T] = { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 0ae4c45988032..197b3d143995a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -896,7 +896,7 @@ object SlowTestReceiver { package object testPackage extends Assertions { def test() { val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") - val ssc = new StreamingContext(conf , Milliseconds(100)) + val ssc = new StreamingContext(conf, Milliseconds(100)) try { val inputStream = ssc.receiverStream(new TestReceiver) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 54eff2b214290..239b10894ad2c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -58,8 +58,8 @@ private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputD * replayable, reliable message queue like Kafka. It requires a sequence as input, and * returns the i_th element at the i_th batch unde manual clock. */ -class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int) - extends InputDStream[T](ssc_) { +class TestInputStream[T: ClassTag](_ssc: StreamingContext, input: Seq[Seq[T]], numPartitions: Int) + extends InputDStream[T](_ssc) { def start() {} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index 3bd8d086abf7f..b67189fbd7f03 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -107,8 +107,8 @@ class ReceiverTrackerSuite extends TestSuiteBase { } /** An input DStream with for testing rate controlling */ -private[streaming] class RateTestInputDStream(@transient ssc_ : StreamingContext) - extends ReceiverInputDStream[Int](ssc_) { +private[streaming] class RateTestInputDStream(@transient _ssc: StreamingContext) + extends ReceiverInputDStream[Int](_ssc) { override def getReceiver(): Receiver[Int] = new RateTestReceiver(id) 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 b5d6a24ce8dd6..734dd93cda471 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 @@ -154,7 +154,7 @@ abstract class CommonWriteAheadLogTests( // Recover old files and generate a second set of log files val dataToWrite2 = generateRandomData() manualClock.advance(100000) - writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching , + writeDataUsingWriteAheadLog(testDir, dataToWrite2, closeFileAfterWrite, allowBatching, manualClock) val logFiles2 = getLogFilesInDirectory(testDir) assert(logFiles2.size > logFiles1.size) diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 4276f25c2165b..5e7ee480cafd1 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -38,6 +38,10 @@ public String toString() { } public int hashInt(int input) { + return hashInt(input, seed); + } + + public static int hashInt(int input, int seed) { int k1 = mixK1(input); int h1 = mixH1(seed, k1); @@ -51,16 +55,38 @@ public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; + int h1 = hashBytesByInt(base, offset, lengthInBytes, seed); + return fmix(h1, lengthInBytes); + } + + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; + int lengthAligned = lengthInBytes - lengthInBytes % 4; + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); + for (int i = lengthAligned; i < lengthInBytes; i++) { + int halfWord = Platform.getByte(base, offset + i); + int k1 = mixK1(halfWord); + h1 = mixH1(h1, k1); + } + return fmix(h1, lengthInBytes); + } + + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { + assert (lengthInBytes % 4 == 0); int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { int halfWord = Platform.getInt(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } - return fmix(h1, lengthInBytes); + return h1; } public int hashLong(long input) { + return hashLong(input, seed); + } + + public static int hashLong(long input, int seed) { int low = (int) input; int high = (int) (input >>> 32); diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8cf438be587dc..d4ca255953a48 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1044,9 +1044,9 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), "pyspark.zip not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.9-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.9.1-src.zip") require(py4jFile.exists(), - "py4j-0.9-src.zip not found; cannot run pyspark application in YARN mode.") + "py4j-0.9.1-src.zip not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) } } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e286aed9f9781..272f1299e0ea9 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -357,7 +357,7 @@ object YarnSparkHadoopUtil { * * @return The correct OOM Error handler JVM option, platform dependent. */ - def getOutOfMemoryErrorArgument : String = { + def getOutOfMemoryErrorArgument: String = { if (Utils.isWindows) { escapeForShell("-XX:OnOutOfMemoryError=taskkill /F /PID %%%%p") } else { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 6db012a77a936..b91c4be2ea875 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -151,9 +151,9 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // When running tests, let's not assume the user has built the assembly module, which also // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the // needed locations. - val sparkHome = sys.props("spark.test.home"); + val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.9-src.zip", + s"$sparkHome/python/lib/py4j-0.9.1-src.zip", s"$sparkHome/python") val extraEnv = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From bc3ee55abde9735f6d8f63b972cabfcd1f072ef9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 15 Jan 2016 00:13:38 -0800 Subject: [PATCH 2/3] STYLE --- .../streaming/kafka/KafkaInputDStream.scala | 2 +- .../apache/spark/streaming/kafka/KafkaRDD.scala | 4 ++-- .../spark/streaming/kafka/KafkaSource.scala | 2 +- .../spark/streaming/kafka/KafkaTestUtils.scala | 4 ++-- .../spark/streaming/kafka/KafkaUtils.scala | 10 +++++----- .../spark/streaming/kafka/KafkaSourceSuite.scala | 3 +-- .../org/apache/spark/sql/DataStreamReader.scala | 9 +++------ .../org/apache/spark/sql/DataStreamWriter.scala | 12 ++---------- .../datasources/ResolvedDataSource.scala | 6 +----- .../spark/sql/execution/datasources/rules.scala | 1 + .../datasources/text/DefaultSource.scala | 4 +--- .../sql/execution/streaming/FileStreamSink.scala | 3 ++- .../execution/streaming/FileStreamSource.scala | 9 +++++---- .../spark/sql/execution/streaming/Sink.scala | 4 ---- .../execution/streaming/StreamExecution.scala | 11 +++++------ .../spark/sql/execution/streaming/memory.scala | 16 ++++------------ .../apache/spark/sql/sources/interfaces.scala | 3 +-- .../scala/org/apache/spark/sql/QueryTest.scala | 4 ---- .../scala/org/apache/spark/sql/StreamTest.scala | 2 +- .../sql/streaming/MemorySourceStressSuite.scala | 5 +---- .../apache/spark/sql/streaming/OffsetSuite.scala | 3 +-- .../apache/spark/sql/streaming/StreamSuite.scala | 2 +- 22 files changed, 41 insertions(+), 78 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 902cd83c51bf9..89d1811c99971 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.kafka import java.util.Properties import scala.collection.Map -import scala.reflect.{ClassTag, classTag} +import scala.reflect.{classTag, ClassTag} import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} import kafka.serializer.Decoder diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 27919c09511fb..4eb155645867b 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import scala.collection.mutable.ArrayBuffer -import scala.reflect.{ClassTag, classTag} +import scala.reflect.{classTag, ClassTag} import kafka.api.{FetchRequestBuilder, FetchResponse} import kafka.common.{ErrorMapping, TopicAndPartition} @@ -27,10 +27,10 @@ import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator -import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} /** * A batch-oriented interface for consuming from Kafka. diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaSource.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaSource.scala index 05e2fb111c596..7b97829766f94 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaSource.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaSource.scala @@ -21,10 +21,10 @@ import kafka.common.TopicAndPartition import kafka.serializer._ import org.apache.spark.Logging +import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.streaming.{Batch, Offset, Source, StreamingRelation} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{DataFrame, Dataset, SQLContext} /** An [[Offset]] for the [[KafkaSource]]. */ diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index 7cec53b43890d..cda11b7d6634a 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -20,8 +20,8 @@ package org.apache.spark.streaming.kafka import java.io.File import java.lang.{Integer => JInt} import java.net.InetSocketAddress -import java.util.concurrent.{TimeUnit, TimeoutException} import java.util.{Map => JMap, Properties} +import java.util.concurrent.{TimeoutException, TimeUnit} import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -38,9 +38,9 @@ import org.apache.kafka.clients.producer._ import org.apache.kafka.common.serialization.StringSerializer import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.streaming.Time import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkConf} /** * This is a helper class for Kafka test suites. This has the functionality to set up diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 2de1b3ca7ae09..45a7db1719088 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -27,19 +27,19 @@ import scala.reflect.ClassTag import com.google.common.base.Charsets.UTF_8 import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} -import net.razorvine.pickle.{Opcodes, Pickler, IObjectPickler} +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} +import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.util.WriteAheadLogUtils import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java._ import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} +import org.apache.spark.streaming.util.WriteAheadLogUtils object KafkaUtils { /** diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaSourceSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaSourceSuite.scala index 832aaf38f92ba..879159ef43a7f 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaSourceSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaSourceSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.StreamTest import org.apache.spark.sql.execution.streaming.{Offset, Source} import org.apache.spark.sql.test.SharedSQLContext - class KafkaSourceSuite extends StreamTest with SharedSQLContext { import testImplicits._ @@ -72,7 +71,7 @@ class KafkaSourceSuite extends StreamTest with SharedSQLContext { testStream(mapped)( AddKafkaData(kafkaSource, Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), - StopStream , + StopStream, DropBatches(1), // Lose last batch in the sink StartStream, CheckAnswer(2, 3, 4), // Should get the data back on recovery diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataStreamReader.scala index bc29105e92332..d2ee1990a98b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataStreamReader.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql -import org.apache.spark.sql.execution.streaming.StreamingRelation - import scala.collection.JavaConverters._ -import org.apache.hadoop.util.StringUtils - -import org.apache.spark.Logging import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.datasources.{LogicalRelation, ResolvedDataSource} +import org.apache.spark.sql.execution.datasources.ResolvedDataSource +import org.apache.spark.sql.execution.streaming.StreamingRelation import org.apache.spark.sql.types.StructType +import org.apache.spark.Logging /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataStreamWriter.scala index db716f4c5d1f0..22943787c6359 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataStreamWriter.scala @@ -17,19 +17,11 @@ package org.apache.spark.sql -import java.util.Properties - -import org.apache.spark.sql.execution.streaming.{Offset, Sink, Batch, StreamExecution} - import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} -import org.apache.spark.sql.catalyst.plans.logical.{Project, InsertIntoTable} -import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils -import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, ResolvedDataSource} -import org.apache.spark.sql.sources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.ResolvedDataSource +import org.apache.spark.sql.execution.streaming.StreamExecution /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala index c7905ed009ddd..0639986fba6ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala @@ -19,9 +19,6 @@ package org.apache.spark.sql.execution.datasources import java.util.ServiceLoader -import com.sun.jersey.core.impl.provider.entity.DataSourceProvider -import org.apache.spark.sql.execution.streaming.{Sink, Source} - import scala.collection.JavaConverters._ import scala.language.{existentials, implicitConversions} import scala.util.{Failure, Success, Try} @@ -32,14 +29,13 @@ import org.apache.hadoop.util.StringUtils import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{CalendarIntervalType, StructType} import org.apache.spark.util.Utils - case class ResolvedDataSource(provider: Class[_], relation: BaseRelation) - object ResolvedDataSource extends Logging { /** A map to maintain backward compatibility in case we move data sources around. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index dd3e66d8a9434..e7af116552096 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.sources.{BaseRelation, HadoopFsRelation, InsertableR * Try to replaces [[UnresolvedRelation]]s with [[ResolvedDataSource]]. */ private[sql] class ResolveDataSource(sqlContext: SQLContext) extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UnresolvedRelation if u.tableIdentifier.database.isDefined => try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala index 628651162aecb..e422d369e4728 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/DefaultSource.scala @@ -30,11 +30,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.catalyst.expressions.codegen.{UnsafeRowWriter, BufferHolder} -import org.apache.spark.sql.execution.streaming.{FileStreamSink, Sink, FileStreamSource, Source} -import org.apache.spark.sql.{AnalysisException, Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.codegen.{BufferHolder, UnsafeRowWriter} import org.apache.spark.sql.execution.datasources.PartitionSpec +import org.apache.spark.sql.execution.streaming.{FileStreamSink, FileStreamSource, Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index da9b2eaf7ee65..2f576e04d3126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.execution.streaming import java.nio.ByteBuffer -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.{FileSystem, Path} + import org.apache.spark.Logging import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.SQLContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 25f0a9f08f17e..4fd7fb660816a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -17,16 +17,17 @@ package org.apache.spark.sql.execution.streaming -import java.io.{OutputStreamWriter, BufferedWriter} +import java.io.{BufferedWriter, OutputStreamWriter} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.fs.{FileStatus, Path, FileSystem} import org.apache.spark.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.collection.OpenHashSet -import scala.collection.mutable.ArrayBuffer - /** * A very simple source that reads text files from the given directory as they appear. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 68cf5f41e5e73..cafb4c0f09faf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -17,10 +17,6 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.{DataFrame, SQLContext} - /** * An interface for systems that can collect the results of a streaming query. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 5c6846be60d4b..d92ec92604a98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -19,15 +19,14 @@ package org.apache.spark.sql.execution.streaming import java.lang.Thread.UncaughtExceptionHandler +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries +import org.apache.spark.sql.{SQLContext, StandingQuery} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.{StandingQuery, DataFrame, SQLContext} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.execution.{SparkPlan, QueryExecution, LogicalRDD} - -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.QueryExecution /** * Manages the execution of a streaming Spark SQL query that is occuring in a separate thread. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index c2c8f6df72ce7..0882c02b32d0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -19,21 +19,13 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.atomic.AtomicInteger -import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.LogicalRDD -import org.apache.spark.sql.types.StructType - import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.api.java.StorageLevels - -import org.apache.spark.sql.{Dataset, DataFrame, SQLContext, Encoder, Row} -import org.apache.spark.sql.catalyst.encoders.{RowEncoder, encoderFor} -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Union, LeafNode} -import org.apache.spark.storage.{StreamBlockId, BlockId} +import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row, SQLContext} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.{encoderFor, RowEncoder} +import org.apache.spark.sql.types.StructType object MemoryStream { protected val currentBlockId = new AtomicInteger(0) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index a55c10ad60661..565ef7861acea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -34,9 +34,8 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.execution.{FileRelation, RDDConversions} -import org.apache.spark.sql.execution.datasources.{PartitioningUtils, PartitionSpec, Partition} -import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.execution.datasources.{BucketSpec, Partition, PartitioningUtils, PartitionSpec} +import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index c7bdf1d382ad9..405e5891ac976 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -22,10 +22,6 @@ import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ import scala.util.control.NonFatal -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate import org.apache.spark.sql.catalyst.plans._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index c005514e7e993..cadbc1f16168f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -26,7 +26,7 @@ import scala.util.Random import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder, encoderFor} +import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala index b6e1fdad8d43f..83795ac53bb43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala @@ -17,13 +17,10 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.{Dataset, StreamTest, Row} +import org.apache.spark.sql.StreamTest import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.test.SharedSQLContext -import scala.collection.mutable.ArrayBuffer -import scala.util.Random - class MemorySourceStressSuite extends StreamTest with SharedSQLContext { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala index faf9c70a6204f..7c3d4da9f2c3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.StreamTest -import org.apache.spark.sql.execution.streaming.{CompositeOffset, Offset, LongOffset} +import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, Offset} trait OffsetSuite extends SparkFunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index b70193bc0d60f..479af682a84a4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql.{StreamTest, Row} +import org.apache.spark.sql.{Row, StreamTest} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.test.SharedSQLContext From 8fc1d9bf38ce9e8c3bb78d184b477551388335a0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 15 Jan 2016 13:11:47 -0800 Subject: [PATCH 3/3] fix --- .../apache/spark/sql/execution/streaming/StreamExecution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index d92ec92604a98..42c529b182c98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -22,7 +22,7 @@ import java.lang.Thread.UncaughtExceptionHandler import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging -import org.apache.spark.sql.{SQLContext, StandingQuery} +import org.apache.spark.sql.{DataFrame, SQLContext, StandingQuery} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._