From 8009f216aa6a287a420176fff45a1fc866d2dfa8 Mon Sep 17 00:00:00 2001 From: Justin Uang Date: Tue, 13 Nov 2018 13:27:13 -0500 Subject: [PATCH] Revert "Merge partial upstream" --- R/pkg/NAMESPACE | 17 +- R/pkg/R/DataFrame.R | 86 +- R/pkg/R/SQLContext.R | 154 +- R/pkg/R/catalog.R | 99 +- R/pkg/R/functions.R | 55 +- R/pkg/R/generics.R | 19 +- R/pkg/R/mllib_fpm.R | 64 - R/pkg/R/sparkR.R | 142 +- R/pkg/tests/fulltests/test_context.R | 12 + R/pkg/tests/fulltests/test_mllib_fpm.R | 16 - R/pkg/tests/fulltests/test_sparkSQL.R | 100 +- R/pkg/tests/fulltests/test_sparkSQL_eager.R | 72 - R/pkg/vignettes/sparkr-vignettes.Rmd | 13 - bin/docker-image-tool.sh | 47 +- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- build/mvn | 10 +- .../spark/util/kvstore/KVStoreSerializer.java | 10 +- .../spark/util/kvstore/LevelDBSuite.java | 2 +- .../spark/network/crypto/AuthEngine.java | 95 +- .../spark/network/crypto/TransportCipher.java | 60 +- .../spark/network/util/TransportConf.java | 16 +- .../network/ChunkFetchIntegrationSuite.java | 54 +- .../spark/network/crypto/AuthEngineSuite.java | 17 - .../shuffle/ShuffleIndexInformation.java | 8 +- .../ExternalShuffleBlockResolverSuite.java | 28 +- .../ExternalShuffleIntegrationSuite.java | 50 +- .../shuffle/ExternalShuffleSecuritySuite.java | 18 +- .../spark/util/sketch/CountMinSketch.java | 7 +- .../spark/util/sketch/CountMinSketchImpl.java | 8 +- .../apache/spark/unsafe/types/UTF8String.java | 6 - .../spark/unsafe/types/UTF8StringSuite.java | 14 +- core/benchmarks/KryoBenchmark-results.txt | 29 - core/pom.xml | 2 +- .../JavaSparkContextVarargsWorkaround.java | 67 + .../apache/spark/io/ReadAheadInputStream.java | 102 +- .../java/org/apache/spark/package-info.java | 4 +- .../sort/BypassMergeSortShuffleWriter.java | 6 +- .../shuffle/sort/ShuffleExternalSorter.java | 63 +- .../org/apache/spark/ui/static/webui.css | 4 - .../org/apache/spark/ui/static/webui.js | 6 +- .../scala/org/apache/spark/Accumulable.scala | 226 ++ .../scala/org/apache/spark/Accumulator.scala | 117 + .../scala/org/apache/spark/Heartbeater.scala | 2 +- .../scala/org/apache/spark/SparkConf.scala | 6 +- .../scala/org/apache/spark/SparkContext.scala | 80 +- .../apache/spark/api/java/JavaPairRDD.scala | 9 +- .../spark/api/java/JavaSparkContext.scala | 155 +- .../spark/api/python/PythonRunner.scala | 26 +- .../apache/spark/api/python/PythonUtils.scala | 3 +- .../api/python/PythonWorkerFactory.scala | 74 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 46 +- .../org/apache/spark/deploy/SparkSubmit.scala | 17 +- .../deploy/history/FsHistoryProvider.scala | 22 +- .../history/HistoryServerArguments.scala | 34 +- .../deploy/rest/StandaloneRestServer.scala | 12 +- .../HadoopDelegationTokenManager.scala | 278 +- .../HadoopFSDelegationTokenProvider.scala | 5 +- .../spark/executor/ExecutorMetrics.scala | 2 +- .../spark/input/PortableDataStream.scala | 12 - .../spark/internal/config/package.scala | 9 - .../apache/spark/io/CompressionCodec.scala | 63 +- .../spark/network/BlockTransferService.scala | 4 +- .../spark/scheduler/AccumulableInfo.scala | 2 +- .../spark/scheduler/AsyncEventQueue.scala | 3 +- .../apache/spark/scheduler/DAGScheduler.scala | 50 +- .../scheduler/EventLoggingListener.scala | 15 +- .../spark/scheduler/ReplayListenerBus.scala | 2 - .../spark/scheduler/SparkListener.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 40 +- .../spark/security/CryptoStreamUtils.scala | 137 +- .../spark/status/AppStatusListener.scala | 52 +- .../apache/spark/status/AppStatusSource.scala | 85 - .../apache/spark/status/AppStatusStore.scala | 15 +- .../org/apache/spark/status/LiveEntity.scala | 2 +- .../spark/status/api/v1/StagesResource.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 8 +- .../org/apache/spark/storage/DiskStore.scala | 5 +- .../org/apache/spark/ui/PagedTable.scala | 214 +- .../org/apache/spark/ui/jobs/StagePage.scala | 13 +- .../org/apache/spark/util/AccumulatorV2.scala | 31 + .../org/apache/spark/util/ListenerBus.scala | 8 - .../spark/util/io/ChunkedByteBuffer.scala | 22 +- .../org/apache/spark/JavaJdbcRDDSuite.java | 30 +- .../sort/UnsafeShuffleWriterSuite.java | 14 +- .../test/org/apache/spark/JavaAPISuite.java | 86 +- .../org/apache/spark/AccumulatorSuite.scala | 148 +- .../scala/org/apache/spark/FileSuite.scala | 13 - .../org/apache/spark/SparkContextSuite.scala | 53 +- .../apache/spark/benchmark/Benchmark.scala | 3 +- .../spark/benchmark/BenchmarkBase.scala | 4 +- .../spark/deploy/SparkSubmitSuite.scala | 17 +- .../history/FsHistoryProviderSuite.scala | 37 +- .../history/HistoryServerArgumentsSuite.scala | 12 + .../rest/StandaloneRestSubmitSuite.scala | 20 - .../HadoopDelegationTokenManagerSuite.scala | 142 +- .../CoarseGrainedSchedulerBackendSuite.scala | 8 +- .../spark/scheduler/DAGSchedulerSuite.scala | 51 +- .../security/CryptoStreamUtilsSuite.scala | 37 +- .../spark/serializer/KryoBenchmark.scala | 79 +- .../org/apache/spark/ui/PagedTableSuite.scala | 2 +- .../spark/util/AccumulatorV2Suite.scala | 53 + .../org/apache/sparktest/ImplicitSuite.scala | 20 + dev/appveyor-install-dependencies.ps1 | 3 +- dev/create-release/release-build.sh | 59 +- dev/deps/spark-deps-hadoop-palantir | 19 +- dev/lint-python | 28 - dev/make-distribution.sh | 10 +- dev/mima | 2 +- dev/run-tests-jenkins.py | 6 +- dev/sbt-checkstyle | 2 + dev/scalastyle | 4 +- dev/sparktestsupport/modules.py | 84 +- dev/test_functions.py | 27 +- dev/tox.ini | 2 - docs/_data/menu-ml.yaml | 2 - docs/_data/menu-sql.yaml | 81 - docs/_includes/nav-left-wrapper-sql.html | 6 - docs/_includes/nav-left.html | 3 +- docs/_layouts/global.html | 11 +- ...rces-avro.md => avro-data-source-guide.md} | 18 +- docs/building-spark.md | 57 +- docs/cluster-overview.md | 2 +- docs/configuration.md | 25 +- docs/graphx-programming-guide.md | 2 +- docs/index.md | 8 +- docs/js/api-docs.js | 3 +- docs/ml-datasource.md | 108 - docs/ml-frequent-pattern-mining.md | 53 - docs/ml-pipeline.md | 2 +- docs/quick-start.md | 5 - docs/running-on-kubernetes.md | 54 +- docs/running-on-mesos.md | 5 - docs/running-on-yarn.md | 45 +- docs/security.md | 92 +- docs/spark-standalone.md | 5 - docs/sparkr.md | 56 +- docs/sql-data-sources-hive-tables.md | 166 - docs/sql-data-sources-jdbc.md | 223 -- docs/sql-data-sources-json.md | 81 - docs/sql-data-sources-load-save-functions.md | 327 -- docs/sql-data-sources-orc.md | 26 - docs/sql-data-sources-parquet.md | 321 -- docs/sql-data-sources-troubleshooting.md | 9 - docs/sql-data-sources.md | 42 - docs/sql-distributed-sql-engine.md | 84 - docs/sql-getting-started.md | 369 -- .../sql-migration-guide-hive-compatibility.md | 152 - docs/sql-migration-guide-upgrade.md | 582 --- docs/sql-migration-guide.md | 23 - docs/sql-performance-tuning.md | 151 - docs/sql-programming-guide.md | 3120 ++++++++++++++++- docs/sql-pyspark-pandas-with-arrow.md | 167 - docs/sql-reference.md | 641 ---- docs/streaming-custom-receivers.md | 2 +- docs/streaming-flume-integration.md | 169 + docs/streaming-kafka-0-10-integration.md | 5 +- docs/streaming-kafka-0-8-integration.md | 196 ++ docs/streaming-kafka-integration.md | 53 +- docs/streaming-programming-guide.md | 37 +- .../structured-streaming-programming-guide.md | 331 +- .../examples/ml/JavaPrefixSpanExample.java | 67 - .../sql/JavaSQLDataSourceExample.java | 7 - .../src/main/python/ml/prefixspan_example.py | 48 - examples/src/main/python/sql/datasource.py | 9 - .../streaming/direct_kafka_wordcount.py | 56 + .../main/python/streaming/flume_wordcount.py | 56 + .../main/python/streaming/kafka_wordcount.py | 56 + examples/src/main/r/RSparkSQLExample.R | 6 +- examples/src/main/r/ml/prefixSpan.R | 42 - examples/src/main/resources/users.orc | Bin 547 -> 0 bytes .../org/apache/spark/examples/HdfsTest.scala | 2 - .../spark/examples/ml/PrefixSpanExample.scala | 62 - .../examples/sql/SQLDataSourceExample.scala | 7 - .../benchmarks/AvroWriteBenchmark-results.txt | 10 - .../spark/sql/avro/AvroDataToCatalyst.scala | 92 +- .../apache/spark/sql/avro/AvroOptions.scala | 16 +- .../spark/sql/avro/CatalystDataToAvro.scala | 8 +- .../spark/sql/avro/SchemaConverters.scala | 26 +- .../org/apache/spark/sql/avro/package.scala | 28 +- .../AvroCatalystDataConversionSuite.scala | 58 +- .../spark/sql/avro/AvroFunctionsSuite.scala | 36 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 65 - .../benchmark/AvroWriteBenchmark.scala | 27 +- external/flume-assembly/pom.xml | 167 + external/flume-sink/pom.xml | 140 + .../flume-sink/src/main/avro/sparkflume.avdl | 40 + .../spark/streaming/flume/sink/Logging.scala | 127 + .../flume/sink/SparkAvroCallbackHandler.scala | 166 + .../streaming/flume/sink/SparkSink.scala | 171 + .../flume/sink/SparkSinkThreadFactory.scala | 28 +- .../streaming/flume/sink/SparkSinkUtils.scala | 24 +- .../flume/sink/TransactionProcessor.scala | 252 ++ .../src/test/resources/log4j.properties | 28 + .../streaming/flume/sink/SparkSinkSuite.scala | 218 ++ external/flume/pom.xml | 89 + .../spark/examples/JavaFlumeEventCount.java | 67 + .../spark/examples/FlumeEventCount.scala | 68 + .../examples/FlumePollingEventCount.scala | 65 + .../streaming/flume/EventTransformer.scala | 72 + .../streaming/flume/FlumeBatchFetcher.scala | 166 + .../streaming/flume/FlumeInputDStream.scala | 208 ++ .../flume/FlumePollingInputDStream.scala | 123 + .../streaming/flume/FlumeTestUtils.scala | 117 + .../spark/streaming/flume/FlumeUtils.scala | 312 ++ .../flume/PollingFlumeTestUtils.scala | 209 ++ .../spark/streaming/flume/package-info.java | 21 + .../spark/streaming/flume/package.scala | 12 +- .../streaming/LocalJavaStreamingContext.java | 44 + .../flume/JavaFlumePollingStreamSuite.java | 44 + .../streaming/flume/JavaFlumeStreamSuite.java | 37 + .../flume/src/test/resources/log4j.properties | 28 + .../spark/streaming/TestOutputStream.scala | 48 + .../flume/FlumePollingStreamSuite.scala | 149 + .../streaming/flume/FlumeStreamSuite.scala | 105 + .../spark/sql/kafka010/KafkaWriter.scala | 2 +- .../KafkaDontFailOnDataLossSuite.scala | 4 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 4 +- .../spark/sql/kafka010/KafkaSinkSuite.scala | 1 - .../streaming/kafka010/ConsumerStrategy.scala | 35 +- .../kafka010/DirectKafkaInputDStream.scala | 1 + .../spark/streaming/kafka010/KafkaUtils.scala | 15 + .../streaming/kafka010/LocationStrategy.scala | 16 +- .../streaming/kafka010/OffsetRange.scala | 8 + .../kafka010/PerPartitionConfig.scala | 3 + .../kafka010/DirectKafkaStreamSuite.scala | 3 - .../streaming/kafka010/KafkaRDDSuite.scala | 4 - external/kafka-0-8-assembly/pom.xml | 170 + external/kafka-0-8/pom.xml | 109 + .../apache/spark/streaming/kafka/Broker.scala | 68 + .../kafka/DirectKafkaInputDStream.scala | 234 ++ .../spark/streaming/kafka/KafkaCluster.scala | 439 +++ .../streaming/kafka/KafkaInputDStream.scala | 142 + .../spark/streaming/kafka/KafkaRDD.scala | 273 ++ .../streaming/kafka/KafkaRDDPartition.scala | 42 + .../streaming/kafka/KafkaTestUtils.scala | 299 ++ .../spark/streaming/kafka/KafkaUtils.scala | 806 +++++ .../spark/streaming/kafka/OffsetRange.scala | 112 + .../kafka/ReliableKafkaReceiver.scala | 302 ++ .../spark/streaming/kafka/package-info.java | 21 + .../spark/streaming/kafka/package.scala | 20 +- .../kafka/JavaDirectKafkaStreamSuite.java | 170 + .../streaming/kafka/JavaKafkaRDDSuite.java | 156 + .../streaming/kafka/JavaKafkaStreamSuite.java | 144 + .../src/test/resources/log4j.properties | 28 + .../kafka/DirectKafkaStreamSuite.scala | 636 ++++ .../streaming/kafka/KafkaClusterSuite.scala | 86 + .../spark/streaming/kafka/KafkaRDDSuite.scala | 182 + .../streaming/kafka/KafkaStreamSuite.scala | 92 + .../kafka/ReliableKafkaStreamSuite.scala | 153 + external/kinesis-asl/pom.xml | 7 - .../streaming/JavaKinesisWordCountASL.java | 2 +- .../spark/metrics/sink/GangliaSink.scala | 7 +- hadoop-cloud/pom.xml | 51 +- .../DecisionTreeClassifier.scala | 14 +- .../ml/classification/GBTClassifier.scala | 6 +- .../RandomForestClassifier.scala | 6 +- .../ml/regression/DecisionTreeRegressor.scala | 13 +- .../spark/ml/regression/GBTRegressor.scala | 6 +- .../ml/regression/RandomForestRegressor.scala | 6 +- .../ml/source/image/ImageDataSource.scala | 17 +- .../scala/org/apache/spark/ml/tree/Node.scala | 247 +- .../spark/ml/tree/impl/RandomForest.scala | 10 +- .../org/apache/spark/ml/tree/treeModels.scala | 36 +- .../clustering/BisectingKMeansModel.scala | 6 +- .../org/apache/spark/mllib/feature/PCA.scala | 20 +- .../mllib/linalg/distributed/RowMatrix.scala | 33 +- .../DecisionTreeClassifierSuite.scala | 31 +- .../classification/GBTClassifierSuite.scala | 4 +- .../LogisticRegressionSuite.scala | 28 +- .../RandomForestClassifierSuite.scala | 5 +- .../DecisionTreeRegressorSuite.scala | 14 + .../ml/tree/impl/RandomForestSuite.scala | 22 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 12 +- .../clustering/BisectingKMeansSuite.scala | 3 +- .../apache/spark/mllib/feature/PCASuite.scala | 23 +- .../linalg/UDTSerializationBenchmark.scala | 2 +- pom.xml | 130 +- project/MimaExcludes.scala | 38 +- project/SparkBuild.scala | 24 +- python/README.md | 2 +- python/docs/Makefile | 2 +- python/docs/pyspark.streaming.rst | 14 + python/lib/py4j-0.10.7-src.zip | Bin 0 -> 42437 bytes python/lib/py4j-0.10.8.1-src.zip | Bin 41255 -> 0 bytes python/pyspark/accumulators.py | 12 +- python/pyspark/context.py | 8 +- python/pyspark/sql/functions.py | 223 +- python/pyspark/sql/session.py | 30 - python/pyspark/sql/tests.py | 843 ++--- python/pyspark/sql/window.py | 70 +- python/pyspark/streaming/context.py | 8 +- python/pyspark/streaming/dstream.py | 3 +- python/pyspark/streaming/flume.py | 156 + python/pyspark/streaming/kafka.py | 506 +++ python/pyspark/streaming/tests.py | 459 ++- python/setup.py | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- .../org/apache/spark/deploy/k8s/Config.scala | 64 +- .../apache/spark/deploy/k8s/Constants.scala | 31 +- .../spark/deploy/k8s/KubernetesConf.scala | 41 +- .../spark/deploy/k8s/KubernetesUtils.scala | 68 +- .../k8s/SparkKubernetesClientFactory.scala | 5 - .../HadoopConfExecutorFeatureStep.scala | 48 - .../HadoopSparkUserExecutorFeatureStep.scala | 43 - .../KerberosConfDriverFeatureStep.scala | 165 - .../KerberosConfExecutorFeatureStep.scala | 53 - .../features/MountVolumesFeatureStep.scala | 3 +- .../features/PodTemplateConfigMapStep.scala | 4 +- .../hadooputils/HadoopBootstrapUtil.scala | 283 -- .../hadooputils/HadoopKerberosLogin.scala | 64 - .../submit/KubernetesClientApplication.scala | 12 +- .../k8s/submit/KubernetesDriverBuilder.scala | 39 +- .../k8s/submit/LoggingPodStatusWatcher.scala | 5 +- .../k8s/ExecutorPodsLifecycleManager.scala | 2 +- .../k8s/KubernetesClusterManager.scala | 3 +- .../k8s/KubernetesExecutorBuilder.scala | 42 +- .../deploy/k8s/KubernetesConfSuite.scala | 21 +- .../deploy/k8s/KubernetesUtilsSuite.scala | 68 - .../BasicDriverFeatureStepSuite.scala | 17 +- .../BasicExecutorFeatureStepSuite.scala | 12 +- ...ubernetesCredentialsFeatureStepSuite.scala | 9 +- .../DriverServiceFeatureStepSuite.scala | 18 +- .../features/EnvSecretsFeatureStepSuite.scala | 3 +- .../features/LocalDirsFeatureStepSuite.scala | 3 +- .../MountLocalFilesFeatureStepSuite.scala | 8 +- .../MountSecretsFeatureStepSuite.scala | 3 +- .../MountVolumesFeatureStepSuite.scala | 3 +- .../PodTemplateConfigMapStepSuite.scala | 7 +- .../bindings/JavaDriverFeatureStepSuite.scala | 3 +- .../PythonDriverFeatureStepSuite.scala | 6 +- .../bindings/RDriverFeatureStepSuite.scala | 3 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 3 +- .../submit/KubernetesDriverBuilderSuite.scala | 105 +- .../k8s/ExecutorLifecycleTestUtils.scala | 2 +- .../k8s/ExecutorPodsAllocatorSuite.scala | 1 - .../k8s/KubernetesExecutorBuilderSuite.scala | 90 +- .../src/main/dockerfiles/spark/Dockerfile | 4 +- .../src/main/dockerfiles/spark/entrypoint.sh | 4 - .../kubernetes/integration-tests/README.md | 183 +- .../dev/dev-run-integration-tests.sh | 13 +- .../kubernetes/integration-tests/pom.xml | 12 +- .../scripts/setup-integration-test-env.sh | 43 +- .../src/test/resources/executor-template.yml | 1 + .../k8s/integrationtest/KubernetesSuite.scala | 9 +- .../KubernetesTestComponents.scala | 5 +- .../integrationtest/PodTemplateSuite.scala | 1 + .../k8s/integrationtest/ProcessUtils.scala | 8 +- .../k8s/integrationtest/TestConfig.scala | 6 +- .../k8s/integrationtest/TestConstants.scala | 15 +- .../backend/IntegrationTestBackend.scala | 21 +- .../backend/cloud/KubeConfigBackend.scala | 70 - .../MesosCoarseGrainedSchedulerBackend.scala | 19 +- .../MesosHadoopDelegationTokenManager.scala | 160 + .../spark/deploy/yarn/ApplicationMaster.scala | 24 +- .../org/apache/spark/deploy/yarn/Client.scala | 30 +- .../deploy/yarn/ResourceRequestHelper.scala | 157 - .../spark/deploy/yarn/YarnAllocator.scala | 38 +- .../org/apache/spark/deploy/yarn/config.scala | 8 +- .../yarn/security/AMCredentialRenewer.scala | 177 + .../YARNHadoopDelegationTokenManager.scala | 48 +- .../cluster/YarnSchedulerBackend.scala | 5 +- .../spark/deploy/yarn/ClientSuite.scala | 31 +- .../yarn/ResourceRequestHelperSuite.scala | 143 - .../yarn/ResourceRequestTestHelper.scala | 89 - .../deploy/yarn/YarnAllocatorSuite.scala | 36 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 4 +- ...ARNHadoopDelegationTokenManagerSuite.scala | 5 +- sbin/spark-config.sh | 2 +- sbin/start-history-server.sh | 17 +- sbin/start-master.sh | 3 +- sbin/start-slave.sh | 3 +- sbin/start-thriftserver.sh | 6 +- .../src/test/resources/ExpectedDockerfile | 4 +- .../benchmarks/HashBenchmark-results.txt | 70 - .../HashByteArrayBenchmark-results.txt | 77 - sql/catalyst/pom.xml | 6 - .../spark/sql/catalyst/parser/SqlBase.g4 | 5 +- .../expressions/RowBasedKeyValueBatch.java | 3 +- .../scala/org/apache/spark/sql/Encoders.scala | 8 +- .../sql/catalyst/JavaTypeInference.scala | 91 +- .../spark/sql/catalyst/ScalaReflection.scala | 205 +- .../sql/catalyst/analysis/Analyzer.scala | 62 +- .../catalyst/analysis/FunctionRegistry.scala | 9 +- .../sql/catalyst/analysis/unresolved.scala | 5 +- .../spark/sql/catalyst/analysis/view.scala | 8 +- .../spark/sql/catalyst/csv/CSVExprUtils.scala | 82 - .../sql/catalyst/csv/CSVHeaderChecker.scala | 131 - .../catalyst/encoders/ExpressionEncoder.scala | 215 +- .../sql/catalyst/encoders/RowEncoder.scala | 24 +- .../sql/catalyst/expressions/ExprUtils.scala | 70 - .../sql/catalyst/expressions/Expression.scala | 26 - .../sql/catalyst/expressions/ScalaUDF.scala | 14 +- .../expressions/aggregate/Average.scala | 33 +- .../expressions/aggregate/Count.scala | 28 +- .../aggregate/UnevaluableAggs.scala | 62 - .../catalyst/expressions/csvExpressions.scala | 176 - .../expressions/jsonExpressions.scala | 124 +- .../sql/catalyst/expressions/literals.scala | 46 +- .../expressions/objects/objects.scala | 55 +- .../sql/catalyst/expressions/predicates.scala | 35 - .../expressions/regexpExpressions.scala | 44 +- .../expressions/windowExpressions.scala | 10 +- .../sql/catalyst/json/JacksonParser.scala | 26 +- .../sql/catalyst/optimizer/Optimizer.scala | 6 +- .../optimizer/RewriteDistinctAggregates.scala | 4 +- .../sql/catalyst/optimizer/expressions.scala | 97 +- .../catalyst/optimizer/finishAnalysis.scala | 18 +- .../sql/catalyst/parser/AstBuilder.scala | 43 +- .../catalyst/plans/logical/LogicalPlan.scala | 22 +- .../plans/logical/basicLogicalOperators.scala | 4 +- .../apache/spark/sql/internal/SQLConf.scala | 55 +- .../RowBasedKeyValueBatchSuite.java | 112 +- .../org/apache/spark/sql/HashBenchmark.scala | 152 +- .../spark/sql/HashByteArrayBenchmark.scala | 120 +- .../spark/sql/UnsafeProjectionBenchmark.scala | 2 +- .../sql/catalyst/ScalaReflectionSuite.scala | 70 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 43 +- .../ExpressionTypeCheckingSuite.scala | 3 - .../catalyst/analysis/TypeCoercionSuite.scala | 30 +- .../encoders/ExpressionEncoderSuite.scala | 6 +- .../catalyst/encoders/RowEncoderSuite.scala | 10 +- .../sql/catalyst/expressions/CastSuite.scala | 2 +- .../expressions/CodeGenerationSuite.scala | 14 +- .../expressions/CsvExpressionsSuite.scala | 168 - .../expressions/ExpressionEvalHelper.scala | 46 +- .../ExpressionEvalHelperSuite.scala | 27 +- .../expressions/JsonExpressionsSuite.scala | 58 +- .../expressions/NullExpressionsSuite.scala | 6 +- .../expressions/RegexpExpressionsSuite.scala | 15 +- .../catalyst/expressions/ScalaUDFSuite.scala | 9 +- .../SortOrderExpressionsSuite.scala | 10 +- .../expressions/TimeWindowSuite.scala | 4 +- .../aggregate/PercentileSuite.scala | 11 +- .../BooleanSimplificationSuite.scala | 111 +- .../optimizer/ReplaceNullWithFalseSuite.scala | 323 -- .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 2 +- .../sql/catalyst/util/DateTimeTestUtils.scala | 10 - ...uiltInDataSourceWriteBenchmark-results.txt | 60 - sql/core/benchmarks/CSVBenchmark-results.txt | 27 - .../DataSourceReadBenchmark-results.txt | 269 -- sql/core/benchmarks/JSONBenchmark-results.txt | 37 - sql/core/benchmarks/JoinBenchmark-results.txt | 75 - sql/core/benchmarks/MiscBenchmark-results.txt | 120 - .../benchmarks/RangeBenchmark-results.txt | 16 - .../WideSchemaBenchmark-results.txt | 216 +- .../sql/execution/BufferedRowIterator.java | 10 + .../scala/org/apache/spark/sql/Column.scala | 14 +- .../apache/spark/sql/DataFrameReader.scala | 18 +- .../apache/spark/sql/DataFrameWriter.scala | 40 +- .../scala/org/apache/spark/sql/Dataset.scala | 24 +- .../org/apache/spark/sql/ForeachWriter.scala | 7 +- .../spark/sql/KeyValueGroupedDataset.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 312 +- .../org/apache/spark/sql/SparkSession.scala | 56 +- .../spark/sql/SparkSessionExtensions.scala | 37 +- .../apache/spark/sql/UDFRegistration.scala | 218 +- .../sql/execution/ColumnarBatchScan.scala | 14 +- .../sql/execution/DataSourceScanExec.scala | 7 +- .../spark/sql/execution/SQLExecution.scala | 34 +- .../apache/spark/sql/execution/SortExec.scala | 12 +- .../spark/sql/execution/SparkSqlParser.scala | 3 +- .../sql/execution/WholeStageCodegenExec.scala | 64 +- .../aggregate/HashAggregateExec.scala | 88 +- .../aggregate/TypedAggregateExpression.scala | 12 +- .../execution/basicPhysicalOperators.scala | 100 +- .../execution/columnar/InMemoryRelation.scala | 28 +- .../columnar/InMemoryTableScanExec.scala | 2 + .../command/AnalyzeColumnCommand.scala | 4 +- .../spark/sql/execution/command/cache.scala | 23 +- .../sql/execution/command/resources.scala | 2 +- .../spark/sql/execution/command/tables.scala | 52 +- .../datasources}/FailureSafeParser.scala | 7 +- .../datasources/FileFormatDataWriter.scala | 3 +- .../execution/datasources/FileScanRDD.scala | 14 +- .../datasources/InMemoryFileIndex.scala | 9 +- .../datasources/csv/CSVDataSource.scala | 162 +- .../datasources/csv/CSVFileFormat.scala | 12 +- .../datasources}/csv/CSVInferSchema.scala | 66 +- .../datasources}/csv/CSVOptions.scala | 4 +- .../execution/datasources/csv/CSVUtils.scala | 93 +- .../datasources/csv/UnivocityGenerator.scala | 1 - .../datasources}/csv/UnivocityParser.scala | 43 +- .../datasources/json/JsonDataSource.scala | 1 - .../datasources/json/JsonFileFormat.scala | 2 +- .../datasources/orc/OrcFilters.scala | 106 +- .../parquet/ParquetFileFormat.scala | 13 +- .../datasources/parquet/ParquetFilters.scala | 31 +- .../datasources/v2/DataSourceV2ScanExec.scala | 2 + .../execution/joins/SortMergeJoinExec.scala | 2 +- .../apache/spark/sql/execution/limit.scala | 31 +- .../execution/python/ArrowPythonRunner.scala | 4 +- .../execution/python/PythonUDFRunner.scala | 4 +- .../sql/execution/ui/AllExecutionsPage.scala | 404 +-- .../spark/sql/execution/ui/SQLListener.scala | 29 +- .../sql/expressions/UserDefinedFunction.scala | 24 +- .../apache/spark/sql/expressions/Window.scala | 51 + .../spark/sql/expressions/WindowSpec.scala | 53 + .../org/apache/spark/sql/functions.scala | 203 +- .../internal/BaseSessionStateBuilder.scala | 7 +- .../sql/streaming/DataStreamWriter.scala | 43 +- .../sql/util/QueryExecutionListener.scala | 108 +- .../sql/JavaBeanDeserializationSuite.java | 240 -- .../apache/spark/sql/JavaDataFrameSuite.java | 30 +- .../JavaDataStreamReaderWriterSuite.java | 89 - .../test/resources/sql-tests/inputs/count.sql | 27 - .../sql-tests/inputs/csv-functions.sql | 17 - .../resources/sql-tests/inputs/group-by.sql | 73 - .../sql-tests/inputs/json-functions.sql | 10 +- .../sql-tests/inputs/show-create-table.sql | 61 - .../sql-tests/inputs/string-functions.sql | 6 +- .../resources/sql-tests/results/count.sql.out | 55 - .../sql-tests/results/csv-functions.sql.out | 119 - .../sql-tests/results/group-by.sql.out | 239 +- .../sql-tests/results/json-functions.sql.out | 304 +- .../results/show-create-table.sql.out | 222 -- .../results/string-functions.sql.out | 18 +- .../native/stringCastAndExpressions.sql.out | 4 +- .../sql-tests/results/window.sql.out | 2 +- .../test-data/with-array-fields.json | 3 - .../resources/test-data/with-map-fields.json | 5 - .../apache/spark/sql/CachedTableSuite.scala | 60 - .../apache/spark/sql/CsvFunctionsSuite.scala | 77 - .../spark/sql/DataFrameAggregateSuite.scala | 30 +- .../spark/sql/DataFrameFunctionsSuite.scala | 15 +- .../spark/sql/DataFrameRangeSuite.scala | 76 +- .../org/apache/spark/sql/DataFrameSuite.scala | 22 +- .../sql/DataFrameWindowFramesSuite.scala | 49 +- .../apache/spark/sql/DatasetBenchmark.scala | 2 +- .../apache/spark/sql/DatasetCacheSuite.scala | 6 +- .../spark/sql/DatasetPrimitiveSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 11 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 35 +- .../ReplaceNullWithFalseEndToEndSuite.scala | 71 - .../apache/spark/sql/SessionStateSuite.scala | 3 - .../sql/SparkSessionExtensionSuite.scala | 24 +- .../spark/sql/StringFunctionsSuite.scala | 44 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 27 - .../sql/execution/SQLJsonProtocolSuite.scala | 33 +- .../execution/WholeStageCodegenSuite.scala | 11 - .../benchmark/AggregateBenchmark.scala | 2 +- .../benchmark/BloomFilterBenchmark.scala | 2 +- .../BuiltInDataSourceWriteBenchmark.scala | 68 +- .../benchmark/DataSourceReadBenchmark.scala | 300 +- .../benchmark/DataSourceWriteBenchmark.scala | 15 +- .../benchmark/FilterPushdownBenchmark.scala | 2 +- .../execution/benchmark/JoinBenchmark.scala | 216 +- .../execution/benchmark/MiscBenchmark.scala | 331 +- .../benchmark/PrimitiveArrayBenchmark.scala | 2 +- .../execution/benchmark/RangeBenchmark.scala | 65 - .../execution/benchmark/SortBenchmark.scala | 2 +- .../benchmark/UnsafeArrayDataBenchmark.scala | 2 +- .../benchmark/WideSchemaBenchmark.scala | 134 +- .../columnar/InMemoryColumnarQuerySuite.scala | 45 +- .../CompressionSchemeBenchmark.scala | 2 +- .../datasources/FileIndexSuite.scala | 39 +- .../SaveIntoDataSourceCommandSuite.scala | 2 +- ...CSVBenchmark.scala => CSVBenchmarks.scala} | 70 +- .../csv/CSVInferSchemaSuite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 16 - .../datasources/csv/CSVUtilsSuite.scala} | 36 +- .../csv/UnivocityParserSuite.scala | 3 +- ...onBenchmark.scala => JsonBenchmarks.scala} | 86 +- .../datasources/json/JsonSuite.scala | 39 +- .../datasources/orc/OrcFilterSuite.scala | 37 +- .../datasources/orc/OrcQuerySuite.scala | 28 +- .../datasources/orc/OrcSourceSuite.scala | 75 - .../execution/datasources/orc/OrcTest.scala | 10 - .../parquet/ParquetFilterSuite.scala | 67 +- .../execution/joins/BroadcastJoinSuite.scala | 2 +- .../execution/joins/ExistenceJoinSuite.scala | 2 +- .../sql/execution/joins/InnerJoinSuite.scala | 6 +- .../sql/execution/joins/OuterJoinSuite.scala | 2 +- .../execution/metric/SQLMetricsSuite.scala | 111 +- .../sources/ForeachBatchSinkSuite.scala | 5 +- .../vectorized/ColumnarBatchBenchmark.scala | 2 +- .../spark/sql/sources/BucketedReadSuite.scala | 181 +- .../continuous/ContinuousSuite.scala | 36 +- .../apache/spark/sql/test/SQLTestUtils.scala | 15 - .../sql/util/DataFrameCallbackSuite.scala | 13 - .../util/ExecutionListenerManagerSuite.scala | 14 +- .../hive/thriftserver/HiveThriftServer2.scala | 7 - ...jectHashAggregateExecBenchmark-results.txt | 45 - .../spark/sql/hive/HiveExternalCatalog.scala | 5 +- .../org/apache/spark/sql/hive/HiveUtils.scala | 2 +- .../apache/spark/sql/hive/TableReader.scala | 2 +- .../hive/client/IsolatedClientLoader.scala | 14 +- .../hive/execution/HiveTableScanExec.scala | 2 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 53 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 3 +- .../spark/sql/hive/orc/OrcFilters.scala | 69 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../src/test/resources/sample.json | 0 .../ObjectHashAggregateExecBenchmark.scala | 218 +- .../sql/hive/CompressionCodecSuite.scala | 54 +- .../sql/hive/HiveShowCreateTableSuite.scala | 198 -- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 4 +- .../sql/hive}/ShowCreateTableSuite.scala | 201 +- .../sql/hive/client/HiveClientSuite.scala | 70 +- .../sql/hive/execution/HiveCommandSuite.scala | 9 - .../sql/hive/execution/HiveQuerySuite.scala | 4 + .../sql/hive/execution/HiveUDFSuite.scala | 14 - .../execution/ObjectHashAggregateSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 8 - .../sql/hive/orc/HiveOrcFilterSuite.scala | 45 +- .../sql/hive/orc/HiveOrcSourceSuite.scala | 8 - .../spark/sql/hive/orc/OrcReadBenchmark.scala | 2 +- .../sql/sources/HadoopFsRelationTest.scala | 50 +- .../ParquetHadoopFsRelationSuite.scala | 31 +- .../spark/streaming/StreamingContext.scala | 2 +- .../streaming/api/java/JavaDStream.scala | 2 +- .../streaming/api/java/JavaPairDStream.scala | 8 +- .../api/java/JavaStreamingContext.scala | 27 +- .../spark/streaming/dstream/DStream.scala | 2 +- .../streaming/dstream/InputDStream.scala | 1 + .../apache/spark/streaming/Java8APISuite.java | 2 +- .../apache/spark/streaming/JavaAPISuite.java | 2 +- 618 files changed, 21265 insertions(+), 18339 deletions(-) delete mode 100644 R/pkg/tests/fulltests/test_sparkSQL_eager.R delete mode 100644 core/benchmarks/KryoBenchmark-results.txt create mode 100644 core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java create mode 100644 core/src/main/scala/org/apache/spark/Accumulable.scala create mode 100644 core/src/main/scala/org/apache/spark/Accumulator.scala delete mode 100644 core/src/main/scala/org/apache/spark/status/AppStatusSource.scala delete mode 100644 docs/_data/menu-sql.yaml delete mode 100644 docs/_includes/nav-left-wrapper-sql.html rename docs/{sql-data-sources-avro.md => avro-data-source-guide.md} (94%) delete mode 100644 docs/ml-datasource.md delete mode 100644 docs/sql-data-sources-hive-tables.md delete mode 100644 docs/sql-data-sources-jdbc.md delete mode 100644 docs/sql-data-sources-json.md delete mode 100644 docs/sql-data-sources-load-save-functions.md delete mode 100644 docs/sql-data-sources-orc.md delete mode 100644 docs/sql-data-sources-parquet.md delete mode 100644 docs/sql-data-sources-troubleshooting.md delete mode 100644 docs/sql-data-sources.md delete mode 100644 docs/sql-distributed-sql-engine.md delete mode 100644 docs/sql-getting-started.md delete mode 100644 docs/sql-migration-guide-hive-compatibility.md delete mode 100644 docs/sql-migration-guide-upgrade.md delete mode 100644 docs/sql-migration-guide.md delete mode 100644 docs/sql-performance-tuning.md delete mode 100644 docs/sql-pyspark-pandas-with-arrow.md delete mode 100644 docs/sql-reference.md create mode 100644 docs/streaming-flume-integration.md create mode 100644 docs/streaming-kafka-0-8-integration.md delete mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java delete mode 100644 examples/src/main/python/ml/prefixspan_example.py create mode 100644 examples/src/main/python/streaming/direct_kafka_wordcount.py create mode 100644 examples/src/main/python/streaming/flume_wordcount.py create mode 100644 examples/src/main/python/streaming/kafka_wordcount.py delete mode 100644 examples/src/main/r/ml/prefixSpan.R delete mode 100644 examples/src/main/resources/users.orc delete mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala delete mode 100644 external/avro/benchmarks/AvroWriteBenchmark-results.txt create mode 100644 external/flume-assembly/pom.xml create mode 100644 external/flume-sink/pom.xml create mode 100644 external/flume-sink/src/main/avro/sparkflume.avdl create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala => external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala (57%) rename mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala => external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala (63%) create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala create mode 100644 external/flume-sink/src/test/resources/log4j.properties create mode 100644 external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala create mode 100644 external/flume/pom.xml create mode 100644 external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java create mode 100644 external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala => external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala (70%) create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java create mode 100644 external/flume/src/test/resources/log4j.properties create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala create mode 100644 external/kafka-0-8-assembly/pom.xml create mode 100644 external/kafka-0-8/pom.xml create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala create mode 100644 external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala => external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala (54%) create mode 100644 external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java create mode 100644 external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java create mode 100644 external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java create mode 100644 external/kafka-0-8/src/test/resources/log4j.properties create mode 100644 external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala create mode 100644 external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala create mode 100644 external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala create mode 100644 external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala create mode 100644 external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala create mode 100644 python/lib/py4j-0.10.7-src.zip delete mode 100644 python/lib/py4j-0.10.8.1-src.zip create mode 100644 python/pyspark/streaming/flume.py create mode 100644 python/pyspark/streaming/kafka.py delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala delete mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala delete mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala delete mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala delete mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala delete mode 100644 sql/catalyst/benchmarks/HashBenchmark-results.txt delete mode 100644 sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseSuite.scala delete mode 100644 sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt delete mode 100644 sql/core/benchmarks/CSVBenchmark-results.txt delete mode 100644 sql/core/benchmarks/DataSourceReadBenchmark-results.txt delete mode 100644 sql/core/benchmarks/JSONBenchmark-results.txt delete mode 100644 sql/core/benchmarks/JoinBenchmark-results.txt delete mode 100644 sql/core/benchmarks/MiscBenchmark-results.txt delete mode 100644 sql/core/benchmarks/RangeBenchmark-results.txt rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst/util => core/src/main/scala/org/apache/spark/sql/execution/datasources}/FailureSafeParser.scala (93%) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst => core/src/main/scala/org/apache/spark/sql/execution/datasources}/csv/CSVInferSchema.scala (80%) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst => core/src/main/scala/org/apache/spark/sql/execution/datasources}/csv/CSVOptions.scala (98%) rename sql/{catalyst/src/main/scala/org/apache/spark/sql/catalyst => core/src/main/scala/org/apache/spark/sql/execution/datasources}/csv/UnivocityParser.scala (92%) delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/count.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/count.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out delete mode 100644 sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out delete mode 100644 sql/core/src/test/resources/test-data/with-array-fields.json delete mode 100644 sql/core/src/test/resources/test-data/with-map-fields.json delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseEndToEndSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/{CSVBenchmark.scala => CSVBenchmarks.scala} (61%) rename sql/{catalyst/src/test/scala/org/apache/spark/sql/catalyst => core/src/test/scala/org/apache/spark/sql/execution/datasources}/csv/CSVInferSchemaSuite.scala (99%) rename sql/{catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala => core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala} (55%) rename sql/{catalyst/src/test/scala/org/apache/spark/sql/catalyst => core/src/test/scala/org/apache/spark/sql/execution/datasources}/csv/UnivocityParserSuite.scala (99%) rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/{JsonBenchmark.scala => JsonBenchmarks.scala} (61%) delete mode 100644 sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt rename sql/{core => hive}/src/test/resources/sample.json (100%) delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala rename sql/{core/src/test/scala/org/apache/spark/sql => hive/src/test/scala/org/apache/spark/sql/hive}/ShowCreateTableSuite.scala (56%) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index f9f556e69a1fc..96ff389faf4a0 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -28,8 +28,9 @@ importFrom("utils", "download.file", "object.size", "packageVersion", "tail", "u # S3 methods exported export("sparkR.session") -export("sparkR.session.stop") +export("sparkR.init") export("sparkR.stop") +export("sparkR.session.stop") export("sparkR.conf") export("sparkR.version") export("sparkR.uiWebUrl") @@ -41,6 +42,9 @@ export("sparkR.callJStatic") export("install.spark") +export("sparkRSQL.init", + "sparkRHive.init") + # MLlib integration exportMethods("glm", "spark.glm", @@ -66,8 +70,7 @@ exportMethods("glm", "spark.svmLinear", "spark.fpGrowth", "spark.freqItemsets", - "spark.associationRules", - "spark.findFrequentSequentialPatterns") + "spark.associationRules") # Job group lifecycle management methods export("setJobGroup", @@ -147,6 +150,7 @@ exportMethods("arrange", "printSchema", "randomSplit", "rbind", + "registerTempTable", "rename", "repartition", "repartitionByRange", @@ -154,6 +158,7 @@ exportMethods("arrange", "sample", "sample_frac", "sampleBy", + "saveAsParquetFile", "saveAsTable", "saveDF", "schema", @@ -169,6 +174,7 @@ exportMethods("arrange", "toJSON", "transform", "union", + "unionAll", "unionByName", "unique", "unpersist", @@ -268,7 +274,6 @@ exportMethods("%<=>%", "floor", "format_number", "format_string", - "from_csv", "from_json", "from_unixtime", "from_utc_timestamp", @@ -408,14 +413,18 @@ export("as.DataFrame", "cacheTable", "clearCache", "createDataFrame", + "createExternalTable", "createTable", "currentDatabase", + "dropTempTable", "dropTempView", + "jsonFile", "listColumns", "listDatabases", "listFunctions", "listTables", "loadDF", + "parquetFile", "read.df", "read.jdbc", "read.json", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index c99ad76f7643c..34691883bc5a9 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -226,9 +226,7 @@ setMethod("showDF", #' show #' -#' If eager evaluation is enabled and the Spark object is a SparkDataFrame, evaluate the -#' SparkDataFrame and print top rows of the SparkDataFrame, otherwise, print the class -#' and type information of the Spark object. +#' Print class and type information of a Spark object. #' #' @param object a Spark object. Can be a SparkDataFrame, Column, GroupedData, WindowSpec. #' @@ -246,33 +244,11 @@ setMethod("showDF", #' @note show(SparkDataFrame) since 1.4.0 setMethod("show", "SparkDataFrame", function(object) { - allConf <- sparkR.conf() - prop <- allConf[["spark.sql.repl.eagerEval.enabled"]] - if (!is.null(prop) && identical(prop, "true")) { - argsList <- list() - argsList$x <- object - prop <- allConf[["spark.sql.repl.eagerEval.maxNumRows"]] - if (!is.null(prop)) { - numRows <- as.integer(prop) - if (numRows > 0) { - argsList$numRows <- numRows - } - } - prop <- allConf[["spark.sql.repl.eagerEval.truncate"]] - if (!is.null(prop)) { - truncate <- as.integer(prop) - if (truncate > 0) { - argsList$truncate <- truncate - } - } - do.call(showDF, argsList) - } else { - cols <- lapply(dtypes(object), function(l) { - paste(l, collapse = ":") - }) - s <- paste(cols, collapse = ", ") - cat(paste(class(object), "[", s, "]\n", sep = "")) - } + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste(class(object), "[", s, "]\n", sep = "")) }) #' DataTypes @@ -521,6 +497,32 @@ setMethod("createOrReplaceTempView", invisible(callJMethod(x@sdf, "createOrReplaceTempView", viewName)) }) +#' (Deprecated) Register Temporary Table +#' +#' Registers a SparkDataFrame as a Temporary Table in the SparkSession +#' @param x A SparkDataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @seealso \link{createOrReplaceTempView} +#' @rdname registerTempTable-deprecated +#' @name registerTempTable +#' @aliases registerTempTable,SparkDataFrame,character-method +#' @examples +#'\dontrun{ +#' sparkR.session() +#' path <- "path/to/file.json" +#' df <- read.json(path) +#' registerTempTable(df, "json_df") +#' new_df <- sql("SELECT * FROM json_df") +#'} +#' @note registerTempTable since 1.4.0 +setMethod("registerTempTable", + signature(x = "SparkDataFrame", tableName = "character"), + function(x, tableName) { + .Deprecated("createOrReplaceTempView") + invisible(callJMethod(x@sdf, "createOrReplaceTempView", tableName)) + }) + #' insertInto #' #' Insert the contents of a SparkDataFrame into a table registered in the current SparkSession. @@ -930,6 +932,7 @@ setMethod("write.orc", #' path <- "path/to/file.json" #' df <- read.json(path) #' write.parquet(df, "/tmp/sparkr-tmp1/") +#' saveAsParquetFile(df, "/tmp/sparkr-tmp2/") #'} #' @note write.parquet since 1.6.0 setMethod("write.parquet", @@ -940,6 +943,17 @@ setMethod("write.parquet", invisible(handledCallJMethod(write, "parquet", path)) }) +#' @rdname write.parquet +#' @name saveAsParquetFile +#' @aliases saveAsParquetFile,SparkDataFrame,character-method +#' @note saveAsParquetFile since 1.4.0 +setMethod("saveAsParquetFile", + signature(x = "SparkDataFrame", path = "character"), + function(x, path) { + .Deprecated("write.parquet") + write.parquet(x, path) + }) + #' Save the content of SparkDataFrame in a text file at the specified path. #' #' Save the content of the SparkDataFrame in a text file at the specified path. @@ -2724,6 +2738,18 @@ setMethod("union", dataFrame(unioned) }) +#' unionAll is deprecated - use union instead +#' @rdname union +#' @name unionAll +#' @aliases unionAll,SparkDataFrame,SparkDataFrame-method +#' @note unionAll since 1.4.0 +setMethod("unionAll", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + .Deprecated("union") + union(x, y) + }) + #' Return a new SparkDataFrame containing the union of rows, matched by column names #' #' Return a new SparkDataFrame containing the union of rows in this SparkDataFrame diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index afcdd6faa849d..c819a7d14ae98 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -37,6 +37,37 @@ getInternalType <- function(x) { stop(paste("Unsupported type for SparkDataFrame:", class(x)))) } +#' Temporary function to reroute old S3 Method call to new +#' This function is specifically implemented to remove SQLContext from the parameter list. +#' It determines the target to route the call by checking the parent of this callsite (say 'func'). +#' The target should be called 'func.default'. +#' We need to check the class of x to ensure it is SQLContext/HiveContext before dispatching. +#' @param newFuncSig name of the function the user should call instead in the deprecation message +#' @param x the first parameter of the original call +#' @param ... the rest of parameter to pass along +#' @return whatever the target returns +#' @noRd +dispatchFunc <- function(newFuncSig, x, ...) { + # When called with SparkR::createDataFrame, sys.call()[[1]] returns c(::, SparkR, createDataFrame) + callsite <- as.character(sys.call(sys.parent())[[1]]) + funcName <- callsite[[length(callsite)]] + f <- get(paste0(funcName, ".default")) + # Strip sqlContext from list of parameters and then pass the rest along. + contextNames <- c("org.apache.spark.sql.SQLContext", + "org.apache.spark.sql.hive.HiveContext", + "org.apache.spark.sql.hive.test.TestHiveContext", + "org.apache.spark.sql.SparkSession") + if (missing(x) && length(list(...)) == 0) { + f() + } else if (class(x) == "jobj" && + any(grepl(paste(contextNames, collapse = "|"), getClassName.jobj(x)))) { + .Deprecated(newFuncSig, old = paste0(funcName, "(sqlContext...)")) + f(...) + } else { + f(x, ...) + } +} + #' return the SparkSession #' @noRd getSparkSession <- function() { @@ -167,10 +198,11 @@ getDefaultSqlSource <- function() { #' df4 <- createDataFrame(cars, numPartitions = 2) #' } #' @name createDataFrame +#' @method createDataFrame default #' @note createDataFrame since 1.4.0 # TODO(davies): support sampling and infer type from NA -createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0, - numPartitions = NULL) { +createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, + numPartitions = NULL) { sparkSession <- getSparkSession() if (is.data.frame(data)) { @@ -253,18 +285,31 @@ createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0, dataFrame(sdf) } +createDataFrame <- function(x, ...) { + dispatchFunc("createDataFrame(data, schema = NULL)", x, ...) +} + #' @rdname createDataFrame #' @aliases createDataFrame +#' @method as.DataFrame default #' @note as.DataFrame since 1.6.0 -as.DataFrame <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { +as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { createDataFrame(data, schema, samplingRatio, numPartitions) } +#' @param ... additional argument(s). +#' @rdname createDataFrame +#' @aliases as.DataFrame +as.DataFrame <- function(data, ...) { + dispatchFunc("as.DataFrame(data, schema = NULL)", data, ...) +} + #' toDF #' #' Converts an RDD to a SparkDataFrame by infer the types. #' #' @param x An RDD +#' #' @rdname SparkDataFrame #' @noRd #' @examples @@ -298,10 +343,12 @@ setMethod("toDF", signature(x = "RDD"), #' path <- "path/to/file.json" #' df <- read.json(path) #' df <- read.json(path, multiLine = TRUE) +#' df <- jsonFile(path) #' } #' @name read.json +#' @method read.json default #' @note read.json since 1.6.0 -read.json <- function(path, ...) { +read.json.default <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the text file path @@ -312,6 +359,55 @@ read.json <- function(path, ...) { dataFrame(sdf) } +read.json <- function(x, ...) { + dispatchFunc("read.json(path)", x, ...) +} + +#' @rdname read.json +#' @name jsonFile +#' @method jsonFile default +#' @note jsonFile since 1.4.0 +jsonFile.default <- function(path) { + .Deprecated("read.json") + read.json(path) +} + +jsonFile <- function(x, ...) { + dispatchFunc("jsonFile(path)", x, ...) +} + +#' JSON RDD +#' +#' Loads an RDD storing one JSON object per string as a SparkDataFrame. +#' +#' @param sqlContext SQLContext to use +#' @param rdd An RDD of JSON string +#' @param schema A StructType object to use as schema +#' @param samplingRatio The ratio of simpling used to infer the schema +#' @return A SparkDataFrame +#' @noRd +#' @examples +#'\dontrun{ +#' sparkR.session() +#' rdd <- texFile(sc, "path/to/json") +#' df <- jsonRDD(sqlContext, rdd) +#'} + +# TODO: remove - this method is no longer exported +# TODO: support schema +jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { + .Deprecated("read.json") + rdd <- serializeToString(rdd) + if (is.null(schema)) { + read <- callJMethod(sqlContext, "read") + # samplingRatio is deprecated + sdf <- callJMethod(read, "json", callJMethod(getJRDD(rdd), "rdd")) + dataFrame(sdf) + } else { + stop("not implemented") + } +} + #' Create a SparkDataFrame from an ORC file. #' #' Loads an ORC file, returning the result as a SparkDataFrame. @@ -338,12 +434,12 @@ read.orc <- function(path, ...) { #' Loads a Parquet file, returning the result as a SparkDataFrame. #' #' @param path path of file to read. A vector of multiple paths is allowed. -#' @param ... additional data source specific named properties. #' @return SparkDataFrame #' @rdname read.parquet #' @name read.parquet +#' @method read.parquet default #' @note read.parquet since 1.6.0 -read.parquet <- function(path, ...) { +read.parquet.default <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the Parquet file path @@ -354,6 +450,24 @@ read.parquet <- function(path, ...) { dataFrame(sdf) } +read.parquet <- function(x, ...) { + dispatchFunc("read.parquet(...)", x, ...) +} + +#' @param ... argument(s) passed to the method. +#' @rdname read.parquet +#' @name parquetFile +#' @method parquetFile default +#' @note parquetFile since 1.4.0 +parquetFile.default <- function(...) { + .Deprecated("read.parquet") + read.parquet(unlist(list(...))) +} + +parquetFile <- function(x, ...) { + dispatchFunc("parquetFile(...)", x, ...) +} + #' Create a SparkDataFrame from a text file. #' #' Loads text files and returns a SparkDataFrame whose schema starts with @@ -373,8 +487,9 @@ read.parquet <- function(path, ...) { #' df <- read.text(path) #' } #' @name read.text +#' @method read.text default #' @note read.text since 1.6.1 -read.text <- function(path, ...) { +read.text.default <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the text file path @@ -385,6 +500,10 @@ read.text <- function(path, ...) { dataFrame(sdf) } +read.text <- function(x, ...) { + dispatchFunc("read.text(path)", x, ...) +} + #' SQL Query #' #' Executes a SQL query using Spark, returning the result as a SparkDataFrame. @@ -401,13 +520,18 @@ read.text <- function(path, ...) { #' new_df <- sql("SELECT * FROM table") #' } #' @name sql +#' @method sql default #' @note sql since 1.4.0 -sql <- function(sqlQuery) { +sql.default <- function(sqlQuery) { sparkSession <- getSparkSession() sdf <- callJMethod(sparkSession, "sql", sqlQuery) dataFrame(sdf) } +sql <- function(x, ...) { + dispatchFunc("sql(sqlQuery)", x, ...) +} + #' Create a SparkDataFrame from a SparkSQL table or view #' #' Returns the specified table or view as a SparkDataFrame. The table or view must already exist or @@ -466,8 +590,9 @@ tableToDF <- function(tableName) { #' df4 <- read.df(mapTypeJsonPath, "json", stringSchema, multiLine = TRUE) #' } #' @name read.df +#' @method read.df default #' @note read.df since 1.4.0 -read.df <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { +read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { if (!is.null(path) && !is.character(path)) { stop("path should be character, NULL or omitted.") } @@ -502,13 +627,22 @@ read.df <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA" dataFrame(sdf) } +read.df <- function(x = NULL, ...) { + dispatchFunc("read.df(path = NULL, source = NULL, schema = NULL, ...)", x, ...) +} + #' @rdname read.df #' @name loadDF +#' @method loadDF default #' @note loadDF since 1.6.0 -loadDF <- function(path = NULL, source = NULL, schema = NULL, ...) { +loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) { read.df(path, source, schema, ...) } +loadDF <- function(x = NULL, ...) { + dispatchFunc("loadDF(path = NULL, source = NULL, schema = NULL, ...)", x, ...) +} + #' Create a SparkDataFrame representing the database table accessible via JDBC URL #' #' Additional JDBC database connection properties can be set (...) diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R index 7641f8a7a0432..c2d0fc38786be 100644 --- a/R/pkg/R/catalog.R +++ b/R/pkg/R/catalog.R @@ -17,6 +17,40 @@ # catalog.R: SparkSession catalog functions +#' (Deprecated) Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns a SparkDataFrame associated with the external table. +#' +#' The data source is specified by the \code{source} and a set of options(...). +#' If \code{source} is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param tableName a name of the table. +#' @param path the path of files to load. +#' @param source the name of external data source. +#' @param schema the schema of the data required for some data sources. +#' @param ... additional argument(s) passed to the method. +#' @return A SparkDataFrame. +#' @rdname createExternalTable-deprecated +#' @seealso \link{createTable} +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df <- createExternalTable("myjson", path="path/to/json", source="json", schema) +#' } +#' @name createExternalTable +#' @method createExternalTable default +#' @note createExternalTable since 1.4.0 +createExternalTable.default <- function(tableName, path = NULL, source = NULL, schema = NULL, ...) { + .Deprecated("createTable", old = "createExternalTable") + createTable(tableName, path, source, schema, ...) +} + +createExternalTable <- function(x, ...) { + dispatchFunc("createExternalTable(tableName, path = NULL, source = NULL, ...)", x, ...) +} + #' Creates a table based on the dataset in a data source #' #' Creates a table based on the dataset in a data source. Returns a SparkDataFrame associated with @@ -82,13 +116,18 @@ createTable <- function(tableName, path = NULL, source = NULL, schema = NULL, .. #' cacheTable("table") #' } #' @name cacheTable +#' @method cacheTable default #' @note cacheTable since 1.4.0 -cacheTable <- function(tableName) { +cacheTable.default <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(handledCallJMethod(catalog, "cacheTable", tableName)) } +cacheTable <- function(x, ...) { + dispatchFunc("cacheTable(tableName)", x, ...) +} + #' Uncache Table #' #' Removes the specified table from the in-memory cache. @@ -106,13 +145,18 @@ cacheTable <- function(tableName) { #' uncacheTable("table") #' } #' @name uncacheTable +#' @method uncacheTable default #' @note uncacheTable since 1.4.0 -uncacheTable <- function(tableName) { +uncacheTable.default <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(handledCallJMethod(catalog, "uncacheTable", tableName)) } +uncacheTable <- function(x, ...) { + dispatchFunc("uncacheTable(tableName)", x, ...) +} + #' Clear Cache #' #' Removes all cached tables from the in-memory cache. @@ -123,13 +167,48 @@ uncacheTable <- function(tableName) { #' clearCache() #' } #' @name clearCache +#' @method clearCache default #' @note clearCache since 1.4.0 -clearCache <- function() { +clearCache.default <- function() { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(callJMethod(catalog, "clearCache")) } +clearCache <- function() { + dispatchFunc("clearCache()") +} + +#' (Deprecated) Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param tableName The name of the SparkSQL table to be dropped. +#' @seealso \link{dropTempView} +#' @rdname dropTempTable-deprecated +#' @examples +#' \dontrun{ +#' sparkR.session() +#' df <- read.df(path, "parquet") +#' createOrReplaceTempView(df, "table") +#' dropTempTable("table") +#' } +#' @name dropTempTable +#' @method dropTempTable default +#' @note dropTempTable since 1.4.0 +dropTempTable.default <- function(tableName) { + .Deprecated("dropTempView", old = "dropTempTable") + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + dropTempView(tableName) +} + +dropTempTable <- function(x, ...) { + dispatchFunc("dropTempView(viewName)", x, ...) +} + #' Drops the temporary view with the given view name in the catalog. #' #' Drops the temporary view with the given view name in the catalog. @@ -170,12 +249,17 @@ dropTempView <- function(viewName) { #' tables("hive") #' } #' @name tables +#' @method tables default #' @note tables since 1.4.0 -tables <- function(databaseName = NULL) { +tables.default <- function(databaseName = NULL) { # rename column to match previous output schema withColumnRenamed(listTables(databaseName), "name", "tableName") } +tables <- function(x, ...) { + dispatchFunc("tables(databaseName = NULL)", x, ...) +} + #' Table Names #' #' Returns the names of tables in the given database as an array. @@ -189,8 +273,9 @@ tables <- function(databaseName = NULL) { #' tableNames("hive") #' } #' @name tableNames +#' @method tableNames default #' @note tableNames since 1.4.0 -tableNames <- function(databaseName = NULL) { +tableNames.default <- function(databaseName = NULL) { sparkSession <- getSparkSession() callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getTableNames", @@ -198,6 +283,10 @@ tableNames <- function(databaseName = NULL) { databaseName) } +tableNames <- function(x, ...) { + dispatchFunc("tableNames(databaseName = NULL)", x, ...) +} + #' Returns the current default database #' #' Returns the current default database. diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index d2ca1d6c00bb4..2cb4cb8d531e1 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -188,7 +188,6 @@ NULL #' \item \code{to_json}: it is the column containing the struct, array of the structs, #' the map or array of maps. #' \item \code{from_json}: it is the column containing the JSON string. -#' \item \code{from_csv}: it is the column containing the CSV string. #' } #' @param y Column to compute on. #' @param value A value to compute on. @@ -197,13 +196,6 @@ NULL #' \item \code{array_position}: a value to locate in the given array. #' \item \code{array_remove}: a value to remove in the given array. #' } -#' @param schema -#' \itemize{ -#' \item \code{from_json}: a structType object to use as the schema to use -#' when parsing the JSON string. Since Spark 2.3, the DDL-formatted string is -#' also supported for the schema. -#' \item \code{from_csv}: a DDL-formatted string -#' } #' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains #' additional named properties to control how it is converted, accepts the same #' options as the JSON data source. Additionally \code{to_json} supports the "pretty" @@ -2173,6 +2165,8 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' to \code{TRUE}. If the string is unparseable, the Column will contain the value NA. #' #' @rdname column_collection_functions +#' @param schema a structType object to use as the schema to use when parsing the JSON string. +#' Since Spark 2.3, the DDL-formatted string is also supported for the schema. #' @param as.json.array indicating if input string is JSON array of objects or a single object. #' @aliases from_json from_json,Column,characterOrstructType-method #' @examples @@ -2209,36 +2203,6 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") column(jc) }) -#' @details -#' \code{from_csv}: Parses a column containing a CSV string into a Column of \code{structType} -#' with the specified \code{schema}. -#' If the string is unparseable, the Column will contain the value NA. -#' -#' @rdname column_collection_functions -#' @aliases from_csv from_csv,Column,character-method -#' @examples -#' -#' \dontrun{ -#' df <- sql("SELECT 'Amsterdam,2018' as csv") -#' schema <- "city STRING, year INT" -#' head(select(df, from_csv(df$csv, schema)))} -#' @note from_csv since 3.0.0 -setMethod("from_csv", signature(x = "Column", schema = "characterOrColumn"), - function(x, schema, ...) { - if (class(schema) == "Column") { - jschema <- schema@jc - } else if (is.character(schema)) { - jschema <- callJStatic("org.apache.spark.sql.functions", "lit", schema) - } else { - stop("schema argument should be a column or character") - } - options <- varargsToStrEnv(...) - jc <- callJStatic("org.apache.spark.sql.functions", - "from_csv", - x@jc, jschema, options) - column(jc) - }) - #' @details #' \code{from_utc_timestamp}: This is a common function for databases supporting TIMESTAMP WITHOUT #' TIMEZONE. This function takes a timestamp which is timezone-agnostic, and interprets it as a @@ -3509,21 +3473,13 @@ setMethod("collect_set", #' @details #' \code{split_string}: Splits string on regular expression. -#' Equivalent to \code{split} SQL function. Optionally a -#' \code{limit} can be specified +#' Equivalent to \code{split} SQL function. #' #' @rdname column_string_functions -#' @param limit determines the length of the returned array. -#' \itemize{ -#' \item \code{limit > 0}: length of the array will be at most \code{limit} -#' \item \code{limit <= 0}: the returned array can have any length -#' } -#' #' @aliases split_string split_string,Column-method #' @examples #' #' \dontrun{ -#' head(select(df, split_string(df$Class, "\\d", 2))) #' head(select(df, split_string(df$Sex, "a"))) #' head(select(df, split_string(df$Class, "\\d"))) #' # This is equivalent to the following SQL expression @@ -3531,9 +3487,8 @@ setMethod("collect_set", #' @note split_string 2.3.0 setMethod("split_string", signature(x = "Column", pattern = "character"), - function(x, pattern, limit = -1) { - jc <- callJStatic("org.apache.spark.sql.functions", - "split", x@jc, pattern, as.integer(limit)) + function(x, pattern) { + jc <- callJStatic("org.apache.spark.sql.functions", "split", x@jc, pattern) column(jc) }) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 76e17c10843d2..27c1b312d645c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -528,6 +528,9 @@ setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) #' @rdname printSchema setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) +#' @rdname registerTempTable-deprecated +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + #' @rdname rename setGeneric("rename", function(x, ...) { standardGeneric("rename") }) @@ -592,6 +595,9 @@ setGeneric("write.parquet", function(x, path, ...) { standardGeneric("write.parquet") }) +#' @rdname write.parquet +setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) + #' @rdname write.stream setGeneric("write.stream", function(df, source = NULL, outputMode = NULL, ...) { standardGeneric("write.stream") @@ -631,6 +637,9 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) #' @rdname union setGeneric("union", function(x, y) { standardGeneric("union") }) +#' @rdname union +setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) + #' @rdname unionByName setGeneric("unionByName", function(x, y) { standardGeneric("unionByName") }) @@ -975,10 +984,6 @@ setGeneric("format_string", function(format, x, ...) { standardGeneric("format_s #' @name NULL setGeneric("from_json", function(x, schema, ...) { standardGeneric("from_json") }) -#' @rdname column_collection_functions -#' @name NULL -setGeneric("from_csv", function(x, schema, ...) { standardGeneric("from_csv") }) - #' @rdname column_datetime_functions #' @name NULL setGeneric("from_unixtime", function(x, ...) { standardGeneric("from_unixtime") }) @@ -1253,7 +1258,7 @@ setGeneric("sort_array", function(x, asc = TRUE) { standardGeneric("sort_array") #' @rdname column_string_functions #' @name NULL -setGeneric("split_string", function(x, pattern, ...) { standardGeneric("split_string") }) +setGeneric("split_string", function(x, pattern) { standardGeneric("split_string") }) #' @rdname column_string_functions #' @name NULL @@ -1448,10 +1453,6 @@ setGeneric("spark.freqItemsets", function(object) { standardGeneric("spark.freqI #' @rdname spark.fpGrowth setGeneric("spark.associationRules", function(object) { standardGeneric("spark.associationRules") }) -#' @rdname spark.prefixSpan -setGeneric("spark.findFrequentSequentialPatterns", - function(data, ...) { standardGeneric("spark.findFrequentSequentialPatterns") }) - #' @param object a fitted ML model object. #' @param path the directory where the model is saved. #' @param ... additional argument(s) passed to the method. diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index ac37580c6b373..4ad34fe82328f 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -23,12 +23,6 @@ #' @note FPGrowthModel since 2.2.0 setClass("FPGrowthModel", slots = list(jobj = "jobj")) -#' S4 class that represents a PrefixSpan -#' -#' @param jobj a Java object reference to the backing Scala PrefixSpan -#' @note PrefixSpan since 3.0.0 -setClass("PrefixSpan", slots = list(jobj = "jobj")) - #' FP-growth #' #' A parallel FP-growth algorithm to mine frequent itemsets. @@ -161,61 +155,3 @@ setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) - -#' PrefixSpan -#' -#' A parallel PrefixSpan algorithm to mine frequent sequential patterns. -#' \code{spark.findFrequentSequentialPatterns} returns a complete set of frequent sequential -#' patterns. -#' For more details, see -#' \href{https://spark.apache.org/docs/latest/mllib-frequent-pattern-mining.html#prefixspan}{ -#' PrefixSpan}. -#' -# Find frequent sequential patterns. -#' @param data A SparkDataFrame. -#' @param minSupport Minimal support level. -#' @param maxPatternLength Maximal pattern length. -#' @param maxLocalProjDBSize Maximum number of items (including delimiters used in the internal -#' storage format) allowed in a projected database before local -#' processing. -#' @param sequenceCol name of the sequence column in dataset. -#' @param ... additional argument(s) passed to the method. -#' @return A complete set of frequent sequential patterns in the input sequences of itemsets. -#' The returned \code{SparkDataFrame} contains columns of sequence and corresponding -#' frequency. The schema of it will be: -#' \code{sequence: ArrayType(ArrayType(T))} (T is the item type) -#' \code{freq: Long} -#' @rdname spark.prefixSpan -#' @aliases findFrequentSequentialPatterns,PrefixSpan,SparkDataFrame-method -#' @examples -#' \dontrun{ -#' df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), -#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), -#' list(list(list(1L, 2L), list(5L))), -#' list(list(list(6L)))), schema = c("sequence")) -#' frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, -#' maxLocalProjDBSize = 32000000L) -#' showDF(frequency) -#' } -#' @note spark.findFrequentSequentialPatterns(SparkDataFrame) since 3.0.0 -setMethod("spark.findFrequentSequentialPatterns", - signature(data = "SparkDataFrame"), - function(data, minSupport = 0.1, maxPatternLength = 10L, - maxLocalProjDBSize = 32000000L, sequenceCol = "sequence") { - if (!is.numeric(minSupport) || minSupport < 0) { - stop("minSupport should be a number with value >= 0.") - } - if (!is.integer(maxPatternLength) || maxPatternLength <= 0) { - stop("maxPatternLength should be a number with value > 0.") - } - if (!is.numeric(maxLocalProjDBSize) || maxLocalProjDBSize <= 0) { - stop("maxLocalProjDBSize should be a number with value > 0.") - } - - jobj <- callJStatic("org.apache.spark.ml.r.PrefixSpanWrapper", "getPrefixSpan", - as.numeric(minSupport), as.integer(maxPatternLength), - as.numeric(maxLocalProjDBSize), as.character(sequenceCol)) - object <- new("PrefixSpan", jobj = jobj) - dataFrame(callJMethod(object@jobj, "findFrequentSequentialPatterns", data@sdf)) - } - ) diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index ac289d38d01bd..038fefadaaeff 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -88,6 +88,49 @@ sparkR.stop <- function() { sparkR.session.stop() } +#' (Deprecated) Initialize a new Spark Context +#' +#' This function initializes a new SparkContext. +#' +#' @param master The Spark master URL +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkEnvir Named list of environment variables to set on worker nodes +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors +#' @param sparkJars Character vector of jar files to pass to the worker nodes +#' @param sparkPackages Character vector of package coordinates +#' @seealso \link{sparkR.session} +#' @rdname sparkR.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark") +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g")) +#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="4g"), +#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), +#' c("one.jar", "two.jar", "three.jar"), +#' c("com.databricks:spark-avro_2.11:2.0.1")) +#'} +#' @note sparkR.init since 1.4.0 +sparkR.init <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvir = list(), + sparkExecutorEnv = list(), + sparkJars = "", + sparkPackages = "") { + .Deprecated("sparkR.session") + sparkR.sparkContext(master, + appName, + sparkHome, + convertNamedListToEnv(sparkEnvir), + convertNamedListToEnv(sparkExecutorEnv), + sparkJars, + sparkPackages) +} + # Internal function to handle creating the SparkContext. sparkR.sparkContext <- function( master = "", @@ -229,6 +272,61 @@ sparkR.sparkContext <- function( sc } +#' (Deprecated) Initialize a new SQLContext +#' +#' This function creates a SparkContext from an existing JavaSparkContext and +#' then uses it to initialize a new SQLContext +#' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRSQL.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRSQL.init(sc) +#'} +#' @note sparkRSQL.init since 1.4.0 +sparkRSQL.init <- function(jsc = NULL) { + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) + } + + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = FALSE) +} + +#' (Deprecated) Initialize a new HiveContext +#' +#' This function creates a HiveContext from an existing JavaSparkContext +#' +#' Starting SparkR 2.0, a SparkSession is initialized and returned instead. +#' This API is deprecated and kept for backward compatibility only. +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @seealso \link{sparkR.session} +#' @rdname sparkRHive.init-deprecated +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlContext <- sparkRHive.init(sc) +#'} +#' @note sparkRHive.init since 1.4.0 +sparkRHive.init <- function(jsc = NULL) { + .Deprecated("sparkR.session") + + if (exists(".sparkRsession", envir = .sparkREnv)) { + return(get(".sparkRsession", envir = .sparkREnv)) + } + + # Default to without Hive support for backward compatibility. + sparkR.session(enableHiveSupport = TRUE) +} + #' Get the existing SparkSession or initialize a new SparkSession. #' #' SparkSession is the entry point into SparkR. \code{sparkR.session} gets the existing @@ -384,11 +482,26 @@ sparkR.uiWebUrl <- function() { #' setJobGroup("myJobGroup", "My job group description", TRUE) #'} #' @note setJobGroup since 1.5.0 -setJobGroup <- function(groupId, description, interruptOnCancel) { +#' @method setJobGroup default +setJobGroup.default <- function(groupId, description, interruptOnCancel) { sc <- getSparkContext() invisible(callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel)) } +setJobGroup <- function(sc, groupId, description, interruptOnCancel) { + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("setJobGroup(groupId, description, interruptOnCancel)", + old = "setJobGroup(sc, groupId, description, interruptOnCancel)") + setJobGroup.default(groupId, description, interruptOnCancel) + } else { + # Parameter order is shifted + groupIdToUse <- sc + descriptionToUse <- groupId + interruptOnCancelToUse <- description + setJobGroup.default(groupIdToUse, descriptionToUse, interruptOnCancelToUse) + } +} + #' Clear current job group ID and its description #' #' @rdname clearJobGroup @@ -399,11 +512,22 @@ setJobGroup <- function(groupId, description, interruptOnCancel) { #' clearJobGroup() #'} #' @note clearJobGroup since 1.5.0 -clearJobGroup <- function() { +#' @method clearJobGroup default +clearJobGroup.default <- function() { sc <- getSparkContext() invisible(callJMethod(sc, "clearJobGroup")) } +clearJobGroup <- function(sc) { + if (!missing(sc) && + class(sc) == "jobj" && + any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("clearJobGroup()", old = "clearJobGroup(sc)") + } + clearJobGroup.default() +} + + #' Cancel active jobs for the specified group #' #' @param groupId the ID of job group to be cancelled @@ -415,11 +539,23 @@ clearJobGroup <- function() { #' cancelJobGroup("myJobGroup") #'} #' @note cancelJobGroup since 1.5.0 -cancelJobGroup <- function(groupId) { +#' @method cancelJobGroup default +cancelJobGroup.default <- function(groupId) { sc <- getSparkContext() invisible(callJMethod(sc, "cancelJobGroup", groupId)) } +cancelJobGroup <- function(sc, groupId) { + if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { + .Deprecated("cancelJobGroup(groupId)", old = "cancelJobGroup(sc, groupId)") + cancelJobGroup.default(groupId) + } else { + # Parameter order is shifted + groupIdToUse <- sc + cancelJobGroup.default(groupIdToUse) + } +} + #' Set a human readable description of the current job. #' #' Set a description that is shown as a job description in UI. diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index eb8d2a700e1ea..288a2714a554e 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -54,6 +54,15 @@ test_that("Check masked functions", { sort(namesOfMaskedCompletely, na.last = TRUE)) }) +test_that("repeatedly starting and stopping SparkR", { + for (i in 1:4) { + sc <- suppressWarnings(sparkR.init(master = sparkRTestMaster)) + rdd <- parallelize(sc, 1:20, 2L) + expect_equal(countRDD(rdd), 20) + suppressWarnings(sparkR.stop()) + } +}) + test_that("repeatedly starting and stopping SparkSession", { for (i in 1:4) { sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) @@ -92,6 +101,9 @@ test_that("job group functions can be called", { cancelJobGroup("groupId") clearJobGroup() + suppressWarnings(setJobGroup(sc, "groupId", "job description", TRUE)) + suppressWarnings(cancelJobGroup(sc, "groupId")) + suppressWarnings(clearJobGroup(sc)) sparkR.session.stop() }) diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R index daf9ff97a8216..d80f66a25de1c 100644 --- a/R/pkg/tests/fulltests/test_mllib_fpm.R +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -83,20 +83,4 @@ test_that("spark.fpGrowth", { }) -test_that("spark.prefixSpan", { - df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) - result1 <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, - maxLocalProjDBSize = 32000000L) - - expected_result <- createDataFrame(list(list(list(list(1L)), 3L), - list(list(list(3L)), 2L), - list(list(list(2L)), 3L), - list(list(list(1L, 2L)), 3L), - list(list(list(1L), list(3L)), 2L)), - schema = c("sequence", "freq")) - }) - sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 58e0a54d2aacc..50eff3755edf8 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -106,6 +106,15 @@ if (is_windows()) { Sys.setenv(TZ = "GMT") } +test_that("calling sparkRSQL.init returns existing SQL context", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sqlContext) +}) + +test_that("calling sparkRSQL.init returns existing SparkSession", { + expect_equal(suppressWarnings(sparkRSQL.init(sc)), sparkSession) +}) + test_that("calling sparkR.session returns existing SparkSession", { expect_equal(sparkR.session(), sparkSession) }) @@ -212,7 +221,7 @@ test_that("structField type strings", { test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) - df <- SparkR::createDataFrame(rdd, list("a", "b")) + df <- createDataFrame(rdd, list("a", "b")) dfAsDF <- as.DataFrame(rdd, list("a", "b")) expect_is(df, "SparkDataFrame") expect_is(dfAsDF, "SparkDataFrame") @@ -278,7 +287,7 @@ test_that("create DataFrame from RDD", { df <- as.DataFrame(cars, numPartitions = 2) expect_equal(getNumPartitions(df), 2) - df <- SparkR::createDataFrame(cars, numPartitions = 3) + df <- createDataFrame(cars, numPartitions = 3) expect_equal(getNumPartitions(df), 3) # validate limit by num of rows df <- createDataFrame(cars, numPartitions = 60) @@ -299,7 +308,7 @@ test_that("create DataFrame from RDD", { sql("CREATE TABLE people (name string, age double, height float)") df <- read.df(jsonPathNa, "json", schema) insertInto(df, "people") - expect_equal(collect(SparkR::sql("SELECT age from people WHERE name = 'Bob'"))$age, + expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age, c(16)) expect_equal(collect(sql("SELECT height from people WHERE name ='Bob'"))$height, c(176.5)) @@ -619,10 +628,14 @@ test_that("read/write json files", { jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") write.json(df, jsonPath3) - # Test read.json() works with multiple input paths + # Test read.json()/jsonFile() works with multiple input paths jsonDF1 <- read.json(c(jsonPath2, jsonPath3)) expect_is(jsonDF1, "SparkDataFrame") expect_equal(count(jsonDF1), 6) + # Suppress warnings because jsonFile is deprecated + jsonDF2 <- suppressWarnings(jsonFile(c(jsonPath2, jsonPath3))) + expect_is(jsonDF2, "SparkDataFrame") + expect_equal(count(jsonDF2), 6) unlink(jsonPath2) unlink(jsonPath3) @@ -642,6 +655,20 @@ test_that("read/write json files - compression option", { unlink(jsonPath) }) +test_that("jsonRDD() on a RDD with json string", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + rdd <- parallelize(sc, mockLines) + expect_equal(countRDD(rdd), 3) + df <- suppressWarnings(jsonRDD(sqlContext, rdd)) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 3) + + rdd2 <- flatMap(rdd, function(x) c(x, x)) + df <- suppressWarnings(jsonRDD(sqlContext, rdd2)) + expect_is(df, "SparkDataFrame") + expect_equal(count(df), 6) +}) + test_that("test tableNames and tables", { count <- count(listTables()) @@ -656,10 +683,10 @@ test_that("test tableNames and tables", { expect_true("tableName" %in% colnames(tables())) expect_true(all(c("tableName", "database", "isTemporary") %in% colnames(tables()))) - createOrReplaceTempView(df, "table2") + suppressWarnings(registerTempTable(df, "table2")) tables <- listTables() expect_equal(count(tables), count + 2) - dropTempView("table1") + suppressWarnings(dropTempTable("table1")) expect_true(dropTempView("table2")) tables <- listTables() @@ -1620,13 +1647,6 @@ test_that("column functions", { expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) - # Test from_csv() - df <- as.DataFrame(list(list("col" = "1"))) - c <- collect(select(df, alias(from_csv(df$col, "a INT"), "csv"))) - expect_equal(c[[1]][[1]]$a, 1) - c <- collect(select(df, alias(from_csv(df$col, lit("a INT")), "csv"))) - expect_equal(c[[1]][[1]]$a, 1) - # Test to_json(), from_json() df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") j <- collect(select(df, alias(to_json(df$people), "json"))) @@ -1667,7 +1687,7 @@ test_that("column functions", { df <- as.DataFrame(list(list("col" = "{\"date\":\"21/10/2014\"}"))) schema2 <- structType(structField("date", "date")) s <- collect(select(df, from_json(df$col, schema2))) - expect_equal(s[[1]][[1]]$date, NA) + expect_equal(s[[1]][[1]], NA) s <- collect(select(df, from_json(df$col, schema2, dateFormat = "dd/MM/yyyy"))) expect_is(s[[1]][[1]]$date, "Date") expect_equal(as.character(s[[1]][[1]]$date), "2014-10-21") @@ -1799,14 +1819,6 @@ test_that("string operators", { collect(select(df4, split_string(df4$a, "\\\\")))[1, 1], list(list("a.b@c.d 1", "b")) ) - expect_equal( - collect(select(df4, split_string(df4$a, "\\.", 2)))[1, 1], - list(list("a", "b@c.d 1\\b")) - ) - expect_equal( - collect(select(df4, split_string(df4$a, "b", 0)))[1, 1], - list(list("a.", "@c.d 1\\", "")) - ) l5 <- list(list(a = "abc")) df5 <- createDataFrame(l5) @@ -2452,7 +2464,7 @@ test_that("union(), unionByName(), rbind(), except(), and intersect() on a DataF expect_is(unioned, "SparkDataFrame") expect_equal(count(unioned), 6) expect_equal(first(unioned)$name, "Michael") - expect_equal(count(arrange(suppressWarnings(union(df, df2)), df$age)), 6) + expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6) df1 <- select(df2, "age", "name") unioned1 <- arrange(unionByName(df1, df), df1$age) @@ -2631,14 +2643,17 @@ test_that("read/write Parquet files", { expect_is(df2, "SparkDataFrame") expect_equal(count(df2), 3) - # Test write.parquet and read.parquet + # Test write.parquet/saveAsParquetFile and read.parquet/parquetFile parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") write.parquet(df, parquetPath2) parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") - write.parquet(df, parquetPath3) + suppressWarnings(saveAsParquetFile(df, parquetPath3)) parquetDF <- read.parquet(c(parquetPath2, parquetPath3)) expect_is(parquetDF, "SparkDataFrame") expect_equal(count(parquetDF), count(df) * 2) + parquetDF2 <- suppressWarnings(parquetFile(parquetPath2, parquetPath3)) + expect_is(parquetDF2, "SparkDataFrame") + expect_equal(count(parquetDF2), count(df) * 2) # Test if varargs works with variables saveMode <- "overwrite" @@ -3447,6 +3462,39 @@ test_that("Window functions on a DataFrame", { expect_equal(result, expected) }) +test_that("createDataFrame sqlContext parameter backward compatibility", { + sqlContext <- suppressWarnings(sparkRSQL.init(sc)) + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + # Call function with namespace :: operator - SPARK-16538 + df <- suppressWarnings(SparkR::createDataFrame(sqlContext, ldf)) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) + + df2 <- suppressWarnings(createDataFrame(sqlContext, iris)) + expect_equal(count(df2), 150) + expect_equal(ncol(df2), 5) + + df3 <- suppressWarnings(read.df(sqlContext, jsonPath, "json")) + expect_is(df3, "SparkDataFrame") + expect_equal(count(df3), 3) + + before <- suppressWarnings(createDataFrame(sqlContext, iris)) + after <- suppressWarnings(createDataFrame(iris)) + expect_equal(collect(before), collect(after)) + + # more tests for SPARK-16538 + createOrReplaceTempView(df, "table") + SparkR::listTables() + SparkR::sql("SELECT 1") + suppressWarnings(SparkR::sql(sqlContext, "SELECT * FROM table")) + suppressWarnings(SparkR::dropTempTable(sqlContext, "table")) +}) + test_that("randomSplit", { num <- 4000 df <- createDataFrame(data.frame(id = 1:num)) @@ -3633,7 +3681,7 @@ test_that("catalog APIs, listTables, listColumns, listFunctions", { createOrReplaceTempView(as.DataFrame(cars), "cars") - tb <- SparkR::listTables() + tb <- listTables() expect_equal(nrow(tb), count + 1) tbs <- collect(tb) expect_true(nrow(tbs[tbs$name == "cars", ]) > 0) diff --git a/R/pkg/tests/fulltests/test_sparkSQL_eager.R b/R/pkg/tests/fulltests/test_sparkSQL_eager.R deleted file mode 100644 index 9b4489a47b655..0000000000000 --- a/R/pkg/tests/fulltests/test_sparkSQL_eager.R +++ /dev/null @@ -1,72 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -library(testthat) - -context("test show SparkDataFrame when eager execution is enabled.") - -test_that("eager execution is not enabled", { - # Start Spark session without eager execution enabled - sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) - - df <- createDataFrame(faithful) - expect_is(df, "SparkDataFrame") - expected <- "eruptions:double, waiting:double" - expect_output(show(df), expected) - - # Stop Spark session - sparkR.session.stop() -}) - -test_that("eager execution is enabled", { - # Start Spark session with eager execution enabled - sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true") - - sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) - - df <- createDataFrame(faithful) - expect_is(df, "SparkDataFrame") - expected <- paste0("(+---------+-------+\n", - "|eruptions|waiting|\n", - "+---------+-------+\n)*", - "(only showing top 20 rows)") - expect_output(show(df), expected) - - # Stop Spark session - sparkR.session.stop() -}) - -test_that("eager execution is enabled with maxNumRows and truncate set", { - # Start Spark session with eager execution enabled - sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true", - spark.sql.repl.eagerEval.maxNumRows = as.integer(5), - spark.sql.repl.eagerEval.truncate = as.integer(2)) - - sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) - - df <- arrange(createDataFrame(faithful), "waiting") - expect_is(df, "SparkDataFrame") - expected <- paste0("(+---------+-------+\n", - "|eruptions|waiting|\n", - "+---------+-------+\n", - "| 1.| 43|\n)*", - "(only showing top 5 rows)") - expect_output(show(df), expected) - - # Stop Spark session - sparkR.session.stop() -}) diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 7d924efa9d4bb..ad934947437bc 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -542,7 +542,6 @@ SparkR supports the following machine learning models and algorithms. #### Frequent Pattern Mining * FP-growth -* PrefixSpan #### Statistics @@ -999,18 +998,6 @@ We can make predictions based on the `antecedent`. head(predict(fpm, df)) ``` -#### PrefixSpan - -`spark.findFrequentSequentialPatterns` method can be used to find the complete set of frequent sequential patterns in the input sequences of itemsets. - -```{r} -df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) -head(spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L)) -``` - #### Kolmogorov-Smirnov Test `spark.kstest` runs a two-sided, one-sample [Kolmogorov-Smirnov (KS) test](https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test). diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 61959ca2a3041..d6371051ef7fb 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -44,48 +44,26 @@ function image_ref { function build { local BUILD_ARGS local IMG_PATH - local JARS if [ ! -f "$SPARK_HOME/RELEASE" ]; then # Set image build arguments accordingly if this is a source repo and not a distribution archive. - # - # Note that this will copy all of the example jars directory into the image, and that will - # contain a lot of duplicated jars with the main Spark directory. In a proper distribution, - # the examples directory is cleaned up before generating the distribution tarball, so this - # issue does not occur. IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles - JARS=assembly/target/scala-$SPARK_SCALA_VERSION/jars BUILD_ARGS=( ${BUILD_PARAMS} --build-arg img_path=$IMG_PATH --build-arg - spark_jars=$JARS - --build-arg - example_jars=examples/target/scala-$SPARK_SCALA_VERSION/jars - --build-arg - k8s_tests=resource-managers/kubernetes/integration-tests/tests + spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars ) else - # Not passed as arguments to docker, but used to validate the Spark directory. + # Not passed as an argument to docker, but used to validate the Spark directory. IMG_PATH="kubernetes/dockerfiles" - JARS=jars BUILD_ARGS=(${BUILD_PARAMS}) fi - # Verify that the Docker image content directory is present if [ ! -d "$IMG_PATH" ]; then error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark." fi - - # Verify that Spark has actually been built/is a runnable distribution - # i.e. the Spark JARs that the Docker files will place into the image are present - local TOTAL_JARS=$(ls $JARS/spark-* | wc -l) - TOTAL_JARS=$(( $TOTAL_JARS )) - if [ "${TOTAL_JARS}" -eq 0 ]; then - error "Cannot find Spark JARs. This script assumes that Apache Spark has first been built locally or this is a runnable distribution." - fi - local BINDING_BUILD_ARGS=( ${BUILD_PARAMS} --build-arg @@ -98,37 +76,20 @@ function build { docker build $NOCACHEARG "${BUILD_ARGS[@]}" \ -t $(image_ref spark) \ -f "$BASEDOCKERFILE" . - if [ $? -ne 0 ]; then - error "Failed to build Spark JVM Docker image, please refer to Docker build output for details." - fi docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ -t $(image_ref spark-py) \ -f "$PYDOCKERFILE" . - if [ $? -ne 0 ]; then - error "Failed to build PySpark Docker image, please refer to Docker build output for details." - fi + docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ -t $(image_ref spark-r) \ -f "$RDOCKERFILE" . - if [ $? -ne 0 ]; then - error "Failed to build SparkR Docker image, please refer to Docker build output for details." - fi } function push { docker push "$(image_ref spark)" - if [ $? -ne 0 ]; then - error "Failed to push Spark JVM Docker image." - fi docker push "$(image_ref spark-py)" - if [ $? -ne 0 ]; then - error "Failed to push PySpark Docker image." - fi docker push "$(image_ref spark-r)" - if [ $? -ne 0 ]; then - error "Failed to push SparkR Docker image." - fi } function usage { @@ -182,7 +143,7 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:nb: option +while getopts f:p:R:mr:t:n:b: option do case "${option}" in diff --git a/bin/pyspark b/bin/pyspark index 1dcddcc6196b8..5d5affb1f97c3 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -57,7 +57,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.10.8.1-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-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 479fd464c7d3e..15fa910c277b3 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.10.8.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.7-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/build/mvn b/build/mvn index 5b2b3c8351114..36ef414bdc032 100755 --- a/build/mvn +++ b/build/mvn @@ -149,7 +149,6 @@ if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}` export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} "${ZINC_BIN}" -start -port ${ZINC_PORT} \ - -server 127.0.0.1 -idle-timeout 3h \ -scala-compiler "${SCALA_COMPILER}" \ -scala-library "${SCALA_LIBRARY}" &>/dev/null fi @@ -159,12 +158,5 @@ export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} echo "Using \`mvn\` from path: $MVN_BIN" 1>&2 -# call the `mvn` command as usual -# SPARK-25854 +# Last, call the `mvn` command as usual "${MVN_BIN}" -DzincPort=${ZINC_PORT} -e "$@" -MVN_RETCODE=$? - -# Try to shut down zinc explicitly if the server is still running. -"${ZINC_BIN}" -shutdown -port ${ZINC_PORT} - -exit $MVN_RETCODE diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java index 771a9541bb349..bd8d9486acde5 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java @@ -54,8 +54,11 @@ public final byte[] serialize(Object o) throws Exception { return ((String) o).getBytes(UTF_8); } else { ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (GZIPOutputStream out = new GZIPOutputStream(bytes)) { + GZIPOutputStream out = new GZIPOutputStream(bytes); + try { mapper.writeValue(out, o); + } finally { + out.close(); } return bytes.toByteArray(); } @@ -66,8 +69,11 @@ public final T deserialize(byte[] data, Class klass) throws Exception { if (klass.equals(String.class)) { return (T) new String(data, UTF_8); } else { - try (GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data))) { + GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(data)); + try { return mapper.readValue(in, klass); + } finally { + in.close(); } } } diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 0ff0ceb20d5a5..b7a89e5462505 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -222,7 +222,7 @@ public void testSkip() throws Exception { public void testNegativeIndexValues() throws Exception { List expected = Arrays.asList(-100, -50, 0, 50, 100); - expected.forEach(i -> { + expected.stream().forEach(i -> { try { db.write(createCustomType1(i)); } catch (Exception e) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java index 64fdb32a67ada..056505ef53356 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java @@ -159,21 +159,15 @@ public void close() throws IOException { // accurately report the errors when they happen. RuntimeException error = null; byte[] dummy = new byte[8]; - if (encryptor != null) { - try { - doCipherOp(Cipher.ENCRYPT_MODE, dummy, true); - } catch (Exception e) { - error = new RuntimeException(e); - } - encryptor = null; + try { + doCipherOp(encryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); } - if (decryptor != null) { - try { - doCipherOp(Cipher.DECRYPT_MODE, dummy, true); - } catch (Exception e) { - error = new RuntimeException(e); - } - decryptor = null; + try { + doCipherOp(decryptor, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); } random.close(); @@ -195,11 +189,11 @@ byte[] rawResponse(byte[] challenge) { } private byte[] decrypt(byte[] in) throws GeneralSecurityException { - return doCipherOp(Cipher.DECRYPT_MODE, in, false); + return doCipherOp(decryptor, in, false); } private byte[] encrypt(byte[] in) throws GeneralSecurityException { - return doCipherOp(Cipher.ENCRYPT_MODE, in, false); + return doCipherOp(encryptor, in, false); } private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) @@ -211,13 +205,11 @@ private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) byte[] iv = new byte[conf.ivLength()]; System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); - CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); - _encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); - this.encryptor = _encryptor; + encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); - CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); - _decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); - this.decryptor = _decryptor; + decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); } /** @@ -249,52 +241,29 @@ private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int k return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); } - private byte[] doCipherOp(int mode, byte[] in, boolean isFinal) + private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal) throws GeneralSecurityException { - CryptoCipher cipher; - switch (mode) { - case Cipher.ENCRYPT_MODE: - cipher = encryptor; - break; - case Cipher.DECRYPT_MODE: - cipher = decryptor; - break; - default: - throw new IllegalArgumentException(String.valueOf(mode)); - } - - Preconditions.checkState(cipher != null, "Cipher is invalid because of previous error."); + Preconditions.checkState(cipher != null); - try { - int scale = 1; - while (true) { - int size = in.length * scale; - byte[] buffer = new byte[size]; - try { - int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) - : cipher.update(in, 0, in.length, buffer, 0); - if (outSize != buffer.length) { - byte[] output = new byte[outSize]; - System.arraycopy(buffer, 0, output, 0, output.length); - return output; - } else { - return buffer; - } - } catch (ShortBufferException e) { - // Try again with a bigger buffer. - scale *= 2; + int scale = 1; + while (true) { + int size = in.length * scale; + byte[] buffer = new byte[size]; + try { + int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) + : cipher.update(in, 0, in.length, buffer, 0); + if (outSize != buffer.length) { + byte[] output = new byte[outSize]; + System.arraycopy(buffer, 0, output, 0, output.length); + return output; + } else { + return buffer; } + } catch (ShortBufferException e) { + // Try again with a bigger buffer. + scale *= 2; } - } catch (InternalError ie) { - // SPARK-25535. The commons-cryto library will throw InternalError if something goes wrong, - // and leave bad state behind in the Java wrappers, so it's not safe to use them afterwards. - if (mode == Cipher.ENCRYPT_MODE) { - this.encryptor = null; - } else { - this.decryptor = null; - } - throw ie; } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 2745052265f7f..b64e4b7a970b5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -107,72 +107,45 @@ public void addToChannel(Channel ch) throws IOException { private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { private final ByteArrayWritableChannel byteChannel; private final CryptoOutputStream cos; - private boolean isCipherValid; EncryptionHandler(TransportCipher cipher) throws IOException { byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); cos = cipher.createOutputStream(byteChannel); - isCipherValid = true; } @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - ctx.write(new EncryptedMessage(this, cos, msg, byteChannel), promise); + ctx.write(new EncryptedMessage(cos, msg, byteChannel), promise); } @Override public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { try { - if (isCipherValid) { - cos.close(); - } + cos.close(); } finally { super.close(ctx, promise); } } - - /** - * SPARK-25535. Workaround for CRYPTO-141. Avoid further interaction with the underlying cipher - * after an error occurs. - */ - void reportError() { - this.isCipherValid = false; - } - - boolean isCipherValid() { - return isCipherValid; - } } private static class DecryptionHandler extends ChannelInboundHandlerAdapter { private final CryptoInputStream cis; private final ByteArrayReadableChannel byteChannel; - private boolean isCipherValid; DecryptionHandler(TransportCipher cipher) throws IOException { byteChannel = new ByteArrayReadableChannel(); cis = cipher.createInputStream(byteChannel); - isCipherValid = true; } @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { - if (!isCipherValid) { - throw new IOException("Cipher is in invalid state."); - } byteChannel.feedData((ByteBuf) data); byte[] decryptedData = new byte[byteChannel.readableBytes()]; int offset = 0; while (offset < decryptedData.length) { - // SPARK-25535: workaround for CRYPTO-141. - try { - offset += cis.read(decryptedData, offset, decryptedData.length - offset); - } catch (InternalError ie) { - isCipherValid = false; - throw ie; - } + offset += cis.read(decryptedData, offset, decryptedData.length - offset); } ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); @@ -181,9 +154,7 @@ public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { try { - if (isCipherValid) { - cis.close(); - } + cis.close(); } finally { super.channelInactive(ctx); } @@ -194,9 +165,8 @@ private static class EncryptedMessage extends AbstractFileRegion { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; - private final CryptoOutputStream cos; - private final EncryptionHandler handler; private long transferred; + private CryptoOutputStream cos; // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has // to utilize two helper ByteArrayWritableChannel for streaming. One is used to receive raw data @@ -206,14 +176,9 @@ private static class EncryptedMessage extends AbstractFileRegion { private ByteBuffer currentEncrypted; - EncryptedMessage( - EncryptionHandler handler, - CryptoOutputStream cos, - Object msg, - ByteArrayWritableChannel ch) { + EncryptedMessage(CryptoOutputStream cos, Object msg, ByteArrayWritableChannel ch) { Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, "Unrecognized message type: %s", msg.getClass().getName()); - this.handler = handler; this.isByteBuf = msg instanceof ByteBuf; this.buf = isByteBuf ? (ByteBuf) msg : null; this.region = isByteBuf ? null : (FileRegion) msg; @@ -296,9 +261,6 @@ public long transferTo(WritableByteChannel target, long position) throws IOExcep } private void encryptMore() throws IOException { - if (!handler.isCipherValid()) { - throw new IOException("Cipher is in invalid state."); - } byteRawChannel.reset(); if (isByteBuf) { @@ -307,14 +269,8 @@ private void encryptMore() throws IOException { } else { region.transferTo(byteRawChannel, region.transferred()); } - - try { - cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); - cos.flush(); - } catch (InternalError ie) { - handler.reportError(); - throw ie; - } + cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); + cos.flush(); currentEncrypted = ByteBuffer.wrap(byteEncChannel.getData(), 0, byteEncChannel.length()); diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 43a6bc7dc3d06..6d5cccd20b333 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -296,21 +296,17 @@ public long maxChunksBeingTransferred() { * and could take long time to process due to disk contentions. By configuring a slightly * higher number of shuffler server threads, we are able to reserve some threads for * handling other RPC messages, thus making the Client less likely to experience timeout - * when sending RPC messages to the shuffle server. The number of threads used for handling - * chunked fetch requests are percentage of io.serverThreads (if defined) else it is a percentage - * of 2 * #cores. However, a percentage of 0 means netty default number of threads which - * is 2 * #cores ignoring io.serverThreads. The percentage here is configured via - * spark.shuffle.server.chunkFetchHandlerThreadsPercent. The returned value is rounded off to - * ceiling of the nearest integer. + * when sending RPC messages to the shuffle server. Default to 0, which is 2*#cores + * or io.serverThreads. 90 would mean 90% of 2*#cores or 90% of io.serverThreads + * which equals 0.9 * 2*#cores or 0.9 * io.serverThreads. */ public int chunkFetchHandlerThreads() { if (!this.getModuleName().equalsIgnoreCase("shuffle")) { return 0; } int chunkFetchHandlerThreadsPercent = - conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); - return (int)Math.ceil( - (this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) * - chunkFetchHandlerThreadsPercent/(double)100); + conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0); + return this.serverThreads() > 0 ? (this.serverThreads() * chunkFetchHandlerThreadsPercent)/100: + (2 * NettyRuntime.availableProcessors() * chunkFetchHandlerThreadsPercent)/100; } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 37a8664a52661..824482af08dd4 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -143,39 +143,37 @@ public void releaseBuffers() { } private FetchResult fetchChunks(List chunkIndices) throws Exception { - final FetchResult res = new FetchResult(); - - try (TransportClient client = - clientFactory.createClient(TestUtils.getLocalHost(), server.getPort())) { - final Semaphore sem = new Semaphore(0); - - res.successChunks = Collections.synchronizedSet(new HashSet()); - res.failedChunks = Collections.synchronizedSet(new HashSet()); - res.buffers = Collections.synchronizedList(new LinkedList()); - - ChunkReceivedCallback callback = new ChunkReceivedCallback() { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - buffer.retain(); - res.successChunks.add(chunkIndex); - res.buffers.add(buffer); - sem.release(); - } + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); - @Override - public void onFailure(int chunkIndex, Throwable e) { - res.failedChunks.add(chunkIndex); - sem.release(); - } - }; + final FetchResult res = new FetchResult(); + res.successChunks = Collections.synchronizedSet(new HashSet()); + res.failedChunks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); - for (int chunkIndex : chunkIndices) { - client.fetchChunk(STREAM_ID, chunkIndex, callback); + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + buffer.retain(); + res.successChunks.add(chunkIndex); + res.buffers.add(buffer); + sem.release(); } - if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server"); + + @Override + public void onFailure(int chunkIndex, Throwable e) { + res.failedChunks.add(chunkIndex); + sem.release(); } + }; + + for (int chunkIndex : chunkIndices) { + client.fetchChunk(STREAM_ID, chunkIndex, callback); + } + if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); } + client.close(); return res; } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java index c0aa298a4017c..a3519fe4a423e 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -18,11 +18,8 @@ package org.apache.spark.network.crypto; import java.util.Arrays; -import java.util.Map; -import java.security.InvalidKeyException; import static java.nio.charset.StandardCharsets.UTF_8; -import com.google.common.collect.ImmutableMap; import org.junit.BeforeClass; import org.junit.Test; import static org.junit.Assert.*; @@ -107,18 +104,4 @@ public void testBadChallenge() throws Exception { challenge.cipher, challenge.keyLength, challenge.nonce, badChallenge)); } - @Test(expected = InvalidKeyException.class) - public void testBadKeySize() throws Exception { - Map mconf = ImmutableMap.of("spark.network.crypto.keyLength", "42"); - TransportConf conf = new TransportConf("rpc", new MapConfigProvider(mconf)); - - try (AuthEngine engine = new AuthEngine("appId", "secret", conf)) { - engine.challenge(); - fail("Should have failed to create challenge message."); - - // Call close explicitly to make sure it's idempotent. - engine.close(); - } - } - } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java index 371149bef3974..386738ece51a6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleIndexInformation.java @@ -37,8 +37,14 @@ public ShuffleIndexInformation(File indexFile) throws IOException { size = (int)indexFile.length(); ByteBuffer buffer = ByteBuffer.allocate(size); offsets = buffer.asLongBuffer(); - try (DataInputStream dis = new DataInputStream(Files.newInputStream(indexFile.toPath()))) { + DataInputStream dis = null; + try { + dis = new DataInputStream(Files.newInputStream(indexFile.toPath())); dis.readFully(buffer.array()); + } finally { + if (dis != null) { + dis.close(); + } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 459629c5f05fe..d2072a54fa415 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -98,19 +98,19 @@ public void testSortShuffleBlocks() throws IOException { resolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - try (InputStream block0Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 0).createInputStream()) { - String block0 = - CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); - assertEquals(sortBlock0, block0); - } - - try (InputStream block1Stream = resolver.getBlockData( - "app0", "exec0", 0, 0, 1).createInputStream()) { - String block1 = - CharStreams.toString(new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); - assertEquals(sortBlock1, block1); - } + InputStream block0Stream = + resolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream(); + String block0 = CharStreams.toString( + new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); + block0Stream.close(); + assertEquals(sortBlock0, block0); + + InputStream block1Stream = + resolver.getBlockData("app0", "exec0", 0, 0, 1).createInputStream(); + String block1 = CharStreams.toString( + new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); + block1Stream.close(); + assertEquals(sortBlock1, block1); } @Test @@ -149,7 +149,7 @@ public void testNormalizeAndInternPathname() { private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) { String normPathname = - ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3); + ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3); assertEquals(expectedPathname, normPathname); File file = new File(normPathname); String returnedPath = file.getPath(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 526b96b364473..a6a1b8d0ac3f1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -133,37 +133,37 @@ private FetchResult fetchBlocks( final Semaphore requestsRemaining = new Semaphore(0); - try (ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, 5000)) { - client.init(APP_ID); - client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, - new BlockFetchingListener() { - @Override - public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - data.retain(); - res.successBlocks.add(blockId); - res.buffers.add(data); - requestsRemaining.release(); - } + ExternalShuffleClient client = new ExternalShuffleClient(clientConf, null, false, 5000); + client.init(APP_ID); + client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, + new BlockFetchingListener() { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + data.retain(); + res.successBlocks.add(blockId); + res.buffers.add(data); + requestsRemaining.release(); } } - - @Override - public void onBlockFetchFailure(String blockId, Throwable exception) { - synchronized (this) { - if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { - res.failedBlocks.add(blockId); - requestsRemaining.release(); - } + } + + @Override + public void onBlockFetchFailure(String blockId, Throwable exception) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + res.failedBlocks.add(blockId); + requestsRemaining.release(); } } - }, null); + } + }, null); - if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server"); - } + if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); } + client.close(); return res; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 82caf392b821b..16bad9f1b319d 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -96,16 +96,14 @@ private void validate(String appId, String secretKey, boolean encrypt) ImmutableMap.of("spark.authenticate.enableSaslEncryption", "true"))); } - try (ExternalShuffleClient client = - new ExternalShuffleClient( - testConf, new TestSecretKeyHolder(appId, secretKey), true, 5000)) { - client.init(appId); - // Registration either succeeds or throws an exception. - client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", - new ExecutorShuffleInfo( - new String[0], 0, "org.apache.spark.shuffle.sort.SortShuffleManager") - ); - } + ExternalShuffleClient client = + new ExternalShuffleClient(testConf, new TestSecretKeyHolder(appId, secretKey), true, 5000); + client.init(appId); + // Registration either succeeds or throws an exception. + client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", + new ExecutorShuffleInfo(new String[0], 0, + "org.apache.spark.shuffle.sort.SortShuffleManager")); + client.close(); } /** Provides a secret key holder which always returns the given secret key, for a single appId. */ diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java index 06a248c9a27c2..f7c22dddb8cc0 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketch.java @@ -191,9 +191,10 @@ public static CountMinSketch readFrom(InputStream in) throws IOException { * Reads in a {@link CountMinSketch} from a byte array. */ public static CountMinSketch readFrom(byte[] bytes) throws IOException { - try (InputStream in = new ByteArrayInputStream(bytes)) { - return readFrom(in); - } + InputStream in = new ByteArrayInputStream(bytes); + CountMinSketch cms = readFrom(in); + in.close(); + return cms; } /** diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java index b78c1677a1213..fd1906d2e5ae9 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java @@ -322,10 +322,10 @@ public void writeTo(OutputStream out) throws IOException { @Override public byte[] toByteArray() throws IOException { - try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { - writeTo(out); - return out.toByteArray(); - } + ByteArrayOutputStream out = new ByteArrayOutputStream(); + writeTo(out); + out.close(); + return out.toByteArray(); } public static CountMinSketchImpl readFrom(InputStream in) throws IOException { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 3a3bfc4a94bb3..dff4a73f3e9da 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -958,12 +958,6 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { } public UTF8String[] split(UTF8String pattern, int limit) { - // Java String's split method supports "ignore empty string" behavior when the limit is 0 - // whereas other languages do not. To avoid this java specific behavior, we fall back to - // -1 when the limit is 0. - if (limit == 0) { - limit = -1; - } String[] splits = toString().split(pattern.toString(), limit); UTF8String[] res = new UTF8String[splits.length]; for (int i = 0; i < res.length; i++) { diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index cf9cc6b1800a9..dae13f03b02ff 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -393,14 +393,12 @@ public void substringSQL() { @Test public void split() { - UTF8String[] negativeAndZeroLimitCase = - new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi"), fromString("")}; - assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), 0), - negativeAndZeroLimitCase)); - assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), -1), - negativeAndZeroLimitCase)); - assertTrue(Arrays.equals(fromString("ab,def,ghi,").split(fromString(","), 2), - new UTF8String[]{fromString("ab"), fromString("def,ghi,")})); + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), -1), + new UTF8String[]{fromString("ab"), fromString("def"), fromString("ghi")})); + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), + new UTF8String[]{fromString("ab"), fromString("def,ghi")})); + assertTrue(Arrays.equals(fromString("ab,def,ghi").split(fromString(","), 2), + new UTF8String[]{fromString("ab"), fromString("def,ghi")})); } @Test diff --git a/core/benchmarks/KryoBenchmark-results.txt b/core/benchmarks/KryoBenchmark-results.txt deleted file mode 100644 index 91e22f3afc14f..0000000000000 --- a/core/benchmarks/KryoBenchmark-results.txt +++ /dev/null @@ -1,29 +0,0 @@ -================================================================================================ -Benchmark Kryo Unsafe vs safe Serialization -================================================================================================ - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - -Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -basicTypes: Int with unsafe:true 138 / 149 7.2 138.0 1.0X -basicTypes: Long with unsafe:true 168 / 173 6.0 167.7 0.8X -basicTypes: Float with unsafe:true 153 / 174 6.5 153.1 0.9X -basicTypes: Double with unsafe:true 161 / 185 6.2 161.1 0.9X -Array: Int with unsafe:true 2 / 3 409.7 2.4 56.5X -Array: Long with unsafe:true 4 / 5 232.5 4.3 32.1X -Array: Float with unsafe:true 3 / 4 367.3 2.7 50.7X -Array: Double with unsafe:true 4 / 5 228.5 4.4 31.5X -Map of string->Double with unsafe:true 38 / 45 26.5 37.8 3.7X -basicTypes: Int with unsafe:false 176 / 187 5.7 175.9 0.8X -basicTypes: Long with unsafe:false 191 / 203 5.2 191.2 0.7X -basicTypes: Float with unsafe:false 166 / 176 6.0 166.2 0.8X -basicTypes: Double with unsafe:false 174 / 190 5.7 174.3 0.8X -Array: Int with unsafe:false 19 / 26 52.9 18.9 7.3X -Array: Long with unsafe:false 27 / 31 37.7 26.5 5.2X -Array: Float with unsafe:false 8 / 10 124.3 8.0 17.2X -Array: Double with unsafe:false 12 / 13 83.6 12.0 11.5X -Map of string->Double with unsafe:false 38 / 42 26.1 38.3 3.6X - - diff --git a/core/pom.xml b/core/pom.xml index 9b4c731772041..a658ecd8bee2b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -358,7 +358,7 @@ net.sf.py4j py4j - 0.10.8.1 + 0.10.7 org.apache.spark diff --git a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java new file mode 100644 index 0000000000000..0dd8fafbf2c82 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java; + +import java.util.ArrayList; +import java.util.List; + +// See +// http://scala-programming-language.1934581.n4.nabble.com/Workaround-for-implementing-java-varargs-in-2-7-2-final-tp1944767p1944772.html +abstract class JavaSparkContextVarargsWorkaround { + + @SafeVarargs + public final JavaRDD union(JavaRDD... rdds) { + if (rdds.length == 0) { + throw new IllegalArgumentException("Union called on empty list"); + } + List> rest = new ArrayList<>(rdds.length - 1); + for (int i = 1; i < rdds.length; i++) { + rest.add(rdds[i]); + } + return union(rdds[0], rest); + } + + public JavaDoubleRDD union(JavaDoubleRDD... rdds) { + if (rdds.length == 0) { + throw new IllegalArgumentException("Union called on empty list"); + } + List rest = new ArrayList<>(rdds.length - 1); + for (int i = 1; i < rdds.length; i++) { + rest.add(rdds[i]); + } + return union(rdds[0], rest); + } + + @SafeVarargs + public final JavaPairRDD union(JavaPairRDD... rdds) { + if (rdds.length == 0) { + throw new IllegalArgumentException("Union called on empty list"); + } + List> rest = new ArrayList<>(rdds.length - 1); + for (int i = 1; i < rdds.length; i++) { + rest.add(rdds[i]); + } + return union(rdds[0], rest); + } + + // These methods take separate "first" and "rest" elements to avoid having the same type erasure + public abstract JavaRDD union(JavaRDD first, List> rest); + public abstract JavaDoubleRDD union(JavaDoubleRDD first, List rest); + public abstract JavaPairRDD union(JavaPairRDD first, List> + rest); +} diff --git a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java index 2e18715b600e0..0cced9e222952 100644 --- a/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java +++ b/core/src/main/java/org/apache/spark/io/ReadAheadInputStream.java @@ -135,58 +135,62 @@ private void readAsync() throws IOException { } finally { stateChangeLock.unlock(); } - executorService.execute(() -> { - stateChangeLock.lock(); - try { - if (isClosed) { - readInProgress = false; - return; - } - // Flip this so that the close method will not close the underlying input stream when we - // are reading. - isReading = true; - } finally { - stateChangeLock.unlock(); - } + executorService.execute(new Runnable() { - // Please note that it is safe to release the lock and read into the read ahead buffer - // because either of following two conditions will hold - 1. The active buffer has - // data available to read so the reader will not read from the read ahead buffer. - // 2. This is the first time read is called or the active buffer is exhausted, - // in that case the reader waits for this async read to complete. - // So there is no race condition in both the situations. - int read = 0; - int off = 0, len = arr.length; - Throwable exception = null; - try { - // try to fill the read ahead buffer. - // if a reader is waiting, possibly return early. - do { - read = underlyingInputStream.read(arr, off, len); - if (read <= 0) break; - off += read; - len -= read; - } while (len > 0 && !isWaiting.get()); - } catch (Throwable ex) { - exception = ex; - if (ex instanceof Error) { - // `readException` may not be reported to the user. Rethrow Error to make sure at least - // The user can see Error in UncaughtExceptionHandler. - throw (Error) ex; - } - } finally { + @Override + public void run() { stateChangeLock.lock(); - readAheadBuffer.limit(off); - if (read < 0 || (exception instanceof EOFException)) { - endOfStream = true; - } else if (exception != null) { - readAborted = true; - readException = exception; + try { + if (isClosed) { + readInProgress = false; + return; + } + // Flip this so that the close method will not close the underlying input stream when we + // are reading. + isReading = true; + } finally { + stateChangeLock.unlock(); + } + + // Please note that it is safe to release the lock and read into the read ahead buffer + // because either of following two conditions will hold - 1. The active buffer has + // data available to read so the reader will not read from the read ahead buffer. + // 2. This is the first time read is called or the active buffer is exhausted, + // in that case the reader waits for this async read to complete. + // So there is no race condition in both the situations. + int read = 0; + int off = 0, len = arr.length; + Throwable exception = null; + try { + // try to fill the read ahead buffer. + // if a reader is waiting, possibly return early. + do { + read = underlyingInputStream.read(arr, off, len); + if (read <= 0) break; + off += read; + len -= read; + } while (len > 0 && !isWaiting.get()); + } catch (Throwable ex) { + exception = ex; + if (ex instanceof Error) { + // `readException` may not be reported to the user. Rethrow Error to make sure at least + // The user can see Error in UncaughtExceptionHandler. + throw (Error) ex; + } + } finally { + stateChangeLock.lock(); + readAheadBuffer.limit(off); + if (read < 0 || (exception instanceof EOFException)) { + endOfStream = true; + } else if (exception != null) { + readAborted = true; + readException = exception; + } + readInProgress = false; + signalAsyncReadComplete(); + stateChangeLock.unlock(); + closeUnderlyingInputStreamIfNecessary(); } - readInProgress = false; - signalAsyncReadComplete(); - stateChangeLock.unlock(); - closeUnderlyingInputStreamIfNecessary(); } }); } diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java index a029931f9e4c0..4426c7afcebdd 100644 --- a/core/src/main/java/org/apache/spark/package-info.java +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -16,8 +16,8 @@ */ /** - * Core Spark classes in Scala. A few classes here, such as - * {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} + * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the * {@link org.apache.spark.api.java} package contains the main Java API. */ package 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 b020a6d99247b..323a5d3c52831 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 @@ -152,9 +152,9 @@ public void write(Iterator> records) throws IOException { } for (int i = 0; i < numPartitions; i++) { - try (DiskBlockObjectWriter writer = partitionWriters[i]) { - partitionWriterSegments[i] = writer.commitAndGet(); - } + final DiskBlockObjectWriter writer = partitionWriters[i]; + partitionWriterSegments[i] = writer.commitAndGet(); + writer.close(); } File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 1c0d664afb138..c7d2db4217d96 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -181,43 +181,42 @@ private void writeSortedFile(boolean isLastFile) { // around this, we pass a dummy no-op serializer. final SerializerInstance ser = DummySerializerInstance.INSTANCE; - int currentPartition = -1; - final FileSegment committedSegment; - try (DiskBlockObjectWriter writer = - blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse)) { - - final int uaoSize = UnsafeAlignedOffset.getUaoSize(); - while (sortedRecords.hasNext()) { - sortedRecords.loadNext(); - final int partition = sortedRecords.packedRecordPointer.getPartitionId(); - assert (partition >= currentPartition); - if (partition != currentPartition) { - // Switch to the new partition - if (currentPartition != -1) { - final FileSegment fileSegment = writer.commitAndGet(); - spillInfo.partitionLengths[currentPartition] = fileSegment.length(); - } - currentPartition = partition; - } + final DiskBlockObjectWriter writer = + blockManager.getDiskWriter(blockId, file, ser, fileBufferSizeBytes, writeMetricsToUse); - final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); - final Object recordPage = taskMemoryManager.getPage(recordPointer); - final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer); - int dataRemaining = UnsafeAlignedOffset.getSize(recordPage, recordOffsetInPage); - long recordReadPosition = recordOffsetInPage + uaoSize; // skip over record length - while (dataRemaining > 0) { - final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); - Platform.copyMemory( - recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); - writer.write(writeBuffer, 0, toTransfer); - recordReadPosition += toTransfer; - dataRemaining -= toTransfer; + int currentPartition = -1; + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); + while (sortedRecords.hasNext()) { + sortedRecords.loadNext(); + final int partition = sortedRecords.packedRecordPointer.getPartitionId(); + assert (partition >= currentPartition); + if (partition != currentPartition) { + // Switch to the new partition + if (currentPartition != -1) { + final FileSegment fileSegment = writer.commitAndGet(); + spillInfo.partitionLengths[currentPartition] = fileSegment.length(); } - writer.recordWritten(); + currentPartition = partition; } - committedSegment = writer.commitAndGet(); + final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); + final Object recordPage = taskMemoryManager.getPage(recordPointer); + final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer); + int dataRemaining = UnsafeAlignedOffset.getSize(recordPage, recordOffsetInPage); + long recordReadPosition = recordOffsetInPage + uaoSize; // skip over record length + while (dataRemaining > 0) { + final int toTransfer = Math.min(diskWriteBufferSize, dataRemaining); + Platform.copyMemory( + recordPage, recordReadPosition, writeBuffer, Platform.BYTE_ARRAY_OFFSET, toTransfer); + writer.write(writeBuffer, 0, toTransfer); + recordReadPosition += toTransfer; + dataRemaining -= toTransfer; + } + writer.recordWritten(); } + + final FileSegment committedSegment = writer.commitAndGet(); + writer.close(); // If `writeSortedFile()` was called from `closeAndGetSpills()` and no records were inserted, // then the file might be empty. Note that it might be better to avoid calling // writeSortedFile() in that case. diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 266eeec55576e..4b060b0f4e53e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -88,10 +88,6 @@ a.kill-link { float: right; } -a.name-link { - word-wrap: break-word; -} - span.expand-details { font-size: 10pt; cursor: pointer; diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index b1254e08fa504..12c056af9a51a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -83,7 +83,7 @@ $(function() { collapseTablePageLoad('collapse-aggregated-rdds','aggregated-rdds'); collapseTablePageLoad('collapse-aggregated-activeBatches','aggregated-activeBatches'); collapseTablePageLoad('collapse-aggregated-completedBatches','aggregated-completedBatches'); - collapseTablePageLoad('collapse-aggregated-runningExecutions','aggregated-runningExecutions'); - collapseTablePageLoad('collapse-aggregated-completedExecutions','aggregated-completedExecutions'); - collapseTablePageLoad('collapse-aggregated-failedExecutions','aggregated-failedExecutions'); + collapseTablePageLoad('collapse-aggregated-runningExecutions','runningExecutions'); + collapseTablePageLoad('collapse-aggregated-completedExecutions','completedExecutions'); + collapseTablePageLoad('collapse-aggregated-failedExecutions','failedExecutions'); }); \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala new file mode 100644 index 0000000000000..3092074232d18 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/Accumulable.scala @@ -0,0 +1,226 @@ +/* + * 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 + +import java.io.Serializable + +import scala.collection.generic.Growable +import scala.reflect.ClassTag + +import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LegacyAccumulatorWrapper} + + +/** + * A data type that can be accumulated, i.e. has a commutative and associative "add" operation, + * but where the result type, `R`, may be different from the element type being added, `T`. + * + * You must define how to add data, and how to merge two of these together. For some data types, + * such as a counter, these might be the same operation. In that case, you can use the simpler + * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are + * accumulating a set. You will add items to the set, and you will union two sets together. + * + * Operations are not thread-safe. + * + * @param id ID of this accumulator; for internal use only. + * @param initialValue initial value of accumulator + * @param param helper object defining how to add elements of type `R` and `T` + * @param name human-readable name for use in Spark's web UI + * @param countFailedValues whether to accumulate values from failed tasks. This is set to true + * for system and time metrics like serialization time or bytes spilled, + * and false for things with absolute values like number of input rows. + * This should be used for internal metrics only. + * @tparam R the full accumulated data (result type) + * @tparam T partial data that can be added in + */ +@deprecated("use AccumulatorV2", "2.0.0") +class Accumulable[R, T] private ( + val id: Long, + // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile + @transient private val initialValue: R, + param: AccumulableParam[R, T], + val name: Option[String], + private[spark] val countFailedValues: Boolean) + extends Serializable { + + private[spark] def this( + initialValue: R, + param: AccumulableParam[R, T], + name: Option[String], + countFailedValues: Boolean) = { + this(AccumulatorContext.newId(), initialValue, param, name, countFailedValues) + } + + private[spark] def this(initialValue: R, param: AccumulableParam[R, T], name: Option[String]) = { + this(initialValue, param, name, false /* countFailedValues */) + } + + def this(initialValue: R, param: AccumulableParam[R, T]) = this(initialValue, param, None) + + val zero = param.zero(initialValue) + private[spark] val newAcc = new LegacyAccumulatorWrapper(initialValue, param) + newAcc.metadata = AccumulatorMetadata(id, name, countFailedValues) + // Register the new accumulator in ctor, to follow the previous behaviour. + AccumulatorContext.register(newAcc) + + /** + * Add more data to this accumulator / accumulable + * @param term the data to add + */ + def += (term: T) { newAcc.add(term) } + + /** + * Add more data to this accumulator / accumulable + * @param term the data to add + */ + def add(term: T) { newAcc.add(term) } + + /** + * Merge two accumulable objects together + * + * Normally, a user will not want to use this version, but will instead call `+=`. + * @param term the other `R` that will get merged with this + */ + def ++= (term: R) { newAcc._value = param.addInPlace(newAcc._value, term) } + + /** + * Merge two accumulable objects together + * + * Normally, a user will not want to use this version, but will instead call `add`. + * @param term the other `R` that will get merged with this + */ + def merge(term: R) { newAcc._value = param.addInPlace(newAcc._value, term) } + + /** + * Access the accumulator's current value; only allowed on driver. + */ + def value: R = { + if (newAcc.isAtDriverSide) { + newAcc.value + } else { + throw new UnsupportedOperationException("Can't read accumulator value in task") + } + } + + /** + * Get the current value of this accumulator from within a task. + * + * This is NOT the global value of the accumulator. To get the global value after a + * completed operation on the dataset, call `value`. + * + * The typical use of this method is to directly mutate the local value, eg., to add + * an element to a Set. + */ + def localValue: R = newAcc.value + + /** + * Set the accumulator's value; only allowed on driver. + */ + def value_= (newValue: R) { + if (newAcc.isAtDriverSide) { + newAcc._value = newValue + } else { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } + } + + /** + * Set the accumulator's value. For internal use only. + */ + def setValue(newValue: R): Unit = { newAcc._value = newValue } + + /** + * Set the accumulator's value. For internal use only. + */ + private[spark] def setValueAny(newValue: Any): Unit = { setValue(newValue.asInstanceOf[R]) } + + /** + * Create an [[AccumulableInfo]] representation of this [[Accumulable]] with the provided values. + */ + private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { + val isInternal = name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) + new AccumulableInfo(id, name, update, value, isInternal, countFailedValues) + } + + override def toString: String = if (newAcc._value == null) "null" else newAcc._value.toString +} + + +/** + * Helper object defining how to accumulate values of a particular type. An implicit + * AccumulableParam needs to be available when you create [[Accumulable]]s of a specific type. + * + * @tparam R the full accumulated data (result type) + * @tparam T partial data that can be added in + */ +@deprecated("use AccumulatorV2", "2.0.0") +trait AccumulableParam[R, T] extends Serializable { + /** + * Add additional data to the accumulator value. Is allowed to modify and return `r` + * for efficiency (to avoid allocating objects). + * + * @param r the current value of the accumulator + * @param t the data to be added to the accumulator + * @return the new value of the accumulator + */ + def addAccumulator(r: R, t: T): R + + /** + * Merge two accumulated values together. Is allowed to modify and return the first value + * for efficiency (to avoid allocating objects). + * + * @param r1 one set of accumulated data + * @param r2 another set of accumulated data + * @return both data sets merged together + */ + def addInPlace(r1: R, r2: R): R + + /** + * Return the "zero" (identity) value for an accumulator type, given its initial value. For + * example, if R was a vector of N dimensions, this would return a vector of N zeroes. + */ + def zero(initialValue: R): R +} + + +@deprecated("use AccumulatorV2", "2.0.0") +private[spark] class +GrowableAccumulableParam[R : ClassTag, T] + (implicit rg: R => Growable[T] with TraversableOnce[T] with Serializable) + extends AccumulableParam[R, T] { + + def addAccumulator(growable: R, elem: T): R = { + growable += elem + growable + } + + def addInPlace(t1: R, t2: R): R = { + t1 ++= t2 + t1 + } + + def zero(initialValue: R): R = { + // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. + // Instead we'll serialize it to a buffer and load it back. + val ser = new JavaSerializer(new SparkConf(false)).newInstance() + val copy = ser.deserialize[R](ser.serialize(initialValue)) + copy.clear() // In case it contained stuff + copy + } +} diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala new file mode 100644 index 0000000000000..9d5fbefc824ad --- /dev/null +++ b/core/src/main/scala/org/apache/spark/Accumulator.scala @@ -0,0 +1,117 @@ +/* + * 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 + +/** + * A simpler value of [[Accumulable]] where the result type being accumulated is the same + * as the types of elements being merged, i.e. variables that are only "added" to through an + * associative and commutative operation and can therefore be efficiently supported in parallel. + * They can be used to implement counters (as in MapReduce) or sums. Spark natively supports + * accumulators of numeric value types, and programmers can add support for new types. + * + * An accumulator is created from an initial value `v` by calling `SparkContext.accumulator`. + * Tasks running on the cluster can then add to it using the `+=` operator. + * However, they cannot read its value. Only the driver program can read the accumulator's value, + * using its [[#value]] method. + * + * The interpreter session below shows an accumulator being used to add up the elements of an array: + * + * {{{ + * scala> val accum = sc.accumulator(0) + * accum: org.apache.spark.Accumulator[Int] = 0 + * + * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) + * ... + * 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s + * + * scala> accum.value + * res2: Int = 10 + * }}} + * + * @param initialValue initial value of accumulator + * @param param helper object defining how to add elements of type `T` + * @param name human-readable name associated with this accumulator + * @param countFailedValues whether to accumulate values from failed tasks + * @tparam T result type +*/ +@deprecated("use AccumulatorV2", "2.0.0") +class Accumulator[T] private[spark] ( + // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile + @transient private val initialValue: T, + param: AccumulatorParam[T], + name: Option[String] = None, + countFailedValues: Boolean = false) + extends Accumulable[T, T](initialValue, param, name, countFailedValues) + + +/** + * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add + * in is the same type as the accumulated value. An implicit AccumulatorParam object needs to be + * available when you create Accumulators of a specific type. + * + * @tparam T type of value to accumulate + */ +@deprecated("use AccumulatorV2", "2.0.0") +trait AccumulatorParam[T] extends AccumulableParam[T, T] { + def addAccumulator(t1: T, t2: T): T = { + addInPlace(t1, t2) + } +} + + +@deprecated("use AccumulatorV2", "2.0.0") +object AccumulatorParam { + + // The following implicit objects were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, as there are duplicate codes in SparkContext for backward + // compatibility, please update them accordingly if you modify the following implicit objects. + + @deprecated("use AccumulatorV2", "2.0.0") + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 + def zero(initialValue: Double): Double = 0.0 + } + + @deprecated("use AccumulatorV2", "2.0.0") + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 + def zero(initialValue: Int): Int = 0 + } + + @deprecated("use AccumulatorV2", "2.0.0") + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long): Long = t1 + t2 + def zero(initialValue: Long): Long = 0L + } + + @deprecated("use AccumulatorV2", "2.0.0") + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float): Float = t1 + t2 + def zero(initialValue: Float): Float = 0f + } + + // Note: when merging values, this param just adopts the newer value. This is used only + // internally for things that shouldn't really be accumulated across tasks, like input + // read method, which should be the same across all tasks in the same stage. + @deprecated("use AccumulatorV2", "2.0.0") + private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { + def addInPlace(t1: String, t2: String): String = t2 + def zero(initialValue: String): String = "" + } +} diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 84091eef04306..5ba1b9b2d828e 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,7 +61,7 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by ExecutorMetricType.values + * determined by MetricGetter.values */ def getCurrentMetrics(): ExecutorMetrics = { val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 8537c536887e6..81aa31d79ba82 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -729,11 +729,9 @@ private[spark] object SparkConf extends Logging { EXECUTOR_MEMORY_OVERHEAD.key -> Seq( AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3")), KEYTAB.key -> Seq( - AlternateConfig("spark.yarn.keytab", "3.0")), + AlternateConfig("spark.yarn.keytab", "2.5")), PRINCIPAL.key -> Seq( - AlternateConfig("spark.yarn.principal", "3.0")), - KERBEROS_RELOGIN_PERIOD.key -> Seq( - AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0")) + AlternateConfig("spark.yarn.principal", "2.5")) ) /** diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 31e687ab77cce..bc6ca969a012c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReferenc import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} @@ -52,9 +53,9 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} import org.apache.spark.scheduler.local.LocalSchedulerBackend -import org.apache.spark.status.{AppStatusSource, AppStatusStore} +import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump @@ -422,8 +423,7 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. - val appStatusSource = AppStatusSource.createSource(conf) - _statusStore = AppStatusStore.createLiveStore(conf, appStatusSource) + _statusStore = AppStatusStore.createLiveStore(conf) listenerBus.addToStatusQueue(_statusStore.listener.get) // Create the Spark execution environment (cache, map output tracker, etc) @@ -574,7 +574,7 @@ class SparkContext(config: SparkConf) extends Logging { _executorAllocationManager.foreach { e => _env.metricsSystem.registerSource(e.executorAllocationManagerSource) } - appStatusSource.foreach(_env.metricsSystem.registerSource(_)) + // Make sure the context is stopped if the user forgets about it. This avoids leaving // unfinished event logs around after the JVM exits cleanly. It doesn't help if the JVM // is killed, though. @@ -1341,6 +1341,76 @@ class SparkContext(config: SparkConf) extends Logging { // Methods for creating shared variables + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `+=` method. Only the driver can access the accumulator's `value`. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { + val acc = new Accumulator(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) + acc + } + + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display + * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the + * driver can access the accumulator's `value`. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) + : Accumulator[T] = { + val acc = new Accumulator(initialValue, param, Option(name)) + cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) + acc + } + + /** + * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values + * with `+=`. Only the driver can access the accumulable's `value`. + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { + val acc = new Accumulable(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) + acc + } + + /** + * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the + * Spark UI. Tasks can add values to the accumulable using the `+=` operator. Only the driver can + * access the accumulable's `value`. + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) + : Accumulable[R, T] = { + val acc = new Accumulable(initialValue, param, Option(name)) + cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) + acc + } + + /** + * Create an accumulator from a "mutable collection" type. + * + * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by + * standard mutable collections. So you can use this with mutable Map, Set, etc. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] + (initialValue: R): Accumulable[R, T] = { + // TODO the context bound (<%) above should be replaced with simple type bound and implicit + // conversion but is a breaking change. This should be fixed in Spark 3.x. + val param = new GrowableAccumulableParam[R, T] + val acc = new Accumulable(initialValue, param) + cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) + acc + } + /** * Register the given accumulator. * diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 80a4f84087466..9544475ff0428 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import java.{lang => jl} import java.lang.{Iterable => JIterable} -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, List => JList} import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -34,8 +34,7 @@ import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap -import org.apache.spark.api.java.function.{FlatMapFunction, Function => JFunction, - Function2 => JFunction2, PairFunction} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.rdd.RDD.rddToPairRDDFunctions @@ -675,8 +674,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: FlatMapFunction[V, U]): JavaPairRDD[K, U] = { - def fn: (V) => Iterator[U] = (x: V) => f.call(x).asScala + def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { + def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index f1e7ce83c05a1..fbfd28422cd3f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -21,7 +21,6 @@ import java.io.Closeable import java.util import java.util.{Map => JMap} -import scala.annotation.varargs import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -31,10 +30,11 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ +import org.apache.spark.AccumulatorParam._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -43,7 +43,8 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. */ -class JavaSparkContext(val sc: SparkContext) extends Closeable { +class JavaSparkContext(val sc: SparkContext) + extends JavaSparkContextVarargsWorkaround with Closeable { /** * Create a JavaSparkContext that loads settings from system properties (for instance, when @@ -506,31 +507,141 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } - /** Build the union of JavaRDDs. */ - @varargs - def union[T](rdds: JavaRDD[T]*): JavaRDD[T] = { - require(rdds.nonEmpty, "Union called on no RDDs") - implicit val ctag: ClassTag[T] = rdds.head.classTag - sc.union(rdds.map(_.rdd)) + /** Build the union of two or more RDDs. */ + override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = { + val rdds: Seq[RDD[T]] = (Seq(first) ++ rest.asScala).map(_.rdd) + implicit val ctag: ClassTag[T] = first.classTag + sc.union(rdds) } - /** Build the union of JavaPairRDDs. */ - @varargs - def union[K, V](rdds: JavaPairRDD[K, V]*): JavaPairRDD[K, V] = { - require(rdds.nonEmpty, "Union called on no RDDs") - implicit val ctag: ClassTag[(K, V)] = rdds.head.classTag - implicit val ctagK: ClassTag[K] = rdds.head.kClassTag - implicit val ctagV: ClassTag[V] = rdds.head.vClassTag - new JavaPairRDD(sc.union(rdds.map(_.rdd))) + /** Build the union of two or more RDDs. */ + override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]]) + : JavaPairRDD[K, V] = { + val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ rest.asScala).map(_.rdd) + implicit val ctag: ClassTag[(K, V)] = first.classTag + implicit val ctagK: ClassTag[K] = first.kClassTag + implicit val ctagV: ClassTag[V] = first.vClassTag + new JavaPairRDD(sc.union(rdds)) } - /** Build the union of JavaDoubleRDDs. */ - @varargs - def union(rdds: JavaDoubleRDD*): JavaDoubleRDD = { - require(rdds.nonEmpty, "Union called on no RDDs") - new JavaDoubleRDD(sc.union(rdds.map(_.srdd))) + /** Build the union of two or more RDDs. */ + override def union(first: JavaDoubleRDD, rest: java.util.List[JavaDoubleRDD]): JavaDoubleRDD = { + val rdds: Seq[RDD[Double]] = (Seq(first) ++ rest.asScala).map(_.srdd) + new JavaDoubleRDD(sc.union(rdds)) } + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + @deprecated("use sc().longAccumulator()", "2.0.0") + def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = + sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] + + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use sc().longAccumulator(String)", "2.0.0") + def intAccumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + sc.accumulator(initialValue, name)(IntAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Integer]] + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + @deprecated("use sc().doubleAccumulator()", "2.0.0") + def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = + sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use sc().doubleAccumulator(String)", "2.0.0") + def doubleAccumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + sc.accumulator(initialValue, name)(DoubleAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Double]] + + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + @deprecated("use sc().longAccumulator()", "2.0.0") + def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) + + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use sc().longAccumulator(String)", "2.0.0") + def accumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + intAccumulator(initialValue, name) + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + @deprecated("use sc().doubleAccumulator()", "2.0.0") + def accumulator(initialValue: Double): Accumulator[java.lang.Double] = + doubleAccumulator(initialValue) + + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use sc().doubleAccumulator(String)", "2.0.0") + def accumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + doubleAccumulator(initialValue, name) + + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = + sc.accumulator(initialValue)(accumulatorParam) + + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) + : Accumulator[T] = + sc.accumulator(initialValue, name)(accumulatorParam) + + /** + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumulable's `value`. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = + sc.accumulable(initialValue)(param) + + /** + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumulable's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + @deprecated("use AccumulatorV2", "2.0.0") + def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) + : Accumulable[T, R] = + sc.accumulable(initialValue, name)(param) + /** * Broadcast a read-only variable to the cluster, returning a * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 463e4d76e1b55..a7b283b789104 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -106,19 +106,17 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (memoryMb.isDefined) { envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString) } - val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap, - condaInstructions) - // Whether is the worker released into idle pool or closed. When any codes try to release or - // close a worker, they should use `releasedOrClosed.compareAndSet` to flip the state to make - // sure there is only one winner that is going to release or close the worker. - val releasedOrClosed = new AtomicBoolean(false) + val worker: Socket = env.createPythonWorker( + pythonExec, envVars.asScala.toMap, condaInstructions) + // Whether is the worker released into idle pool + val released = new AtomicBoolean(false) // Start a thread to feed the process input from our parent's iterator val writerThread = newWriterThread(env, worker, inputIterator, partitionIndex, context) context.addTaskCompletionListener[Unit] { _ => writerThread.shutdownOnTaskCompletion() - if (!reuseWorker || releasedOrClosed.compareAndSet(false, true)) { + if (!reuseWorker || !released.get) { try { worker.close() } catch { @@ -135,7 +133,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = newReaderIterator( - stream, writerThread, startTime, env, worker, releasedOrClosed, context) + stream, writerThread, startTime, env, worker, released, context) new InterruptibleIterator(context, stdoutIterator) } @@ -152,7 +150,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - releasedOrClosed: AtomicBoolean, + released: AtomicBoolean, context: TaskContext): Iterator[OUT] /** @@ -341,7 +339,6 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } } dataOut.flush() - dataOut.writeInt(evalType) writeCommand(dataOut) writeIteratorToStream(dataOut) @@ -396,7 +393,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - releasedOrClosed: AtomicBoolean, + released: AtomicBoolean, context: TaskContext) extends Iterator[OUT] { @@ -467,8 +464,9 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } // Check whether the worker is ready to be re-used. if (stream.readInt() == SpecialLengths.END_OF_STREAM) { - if (reuseWorker && releasedOrClosed.compareAndSet(false, true)) { + if (reuseWorker) { env.releasePythonWorker(pythonExec, envVars.asScala.toMap, condaInstructions, worker) + released.set(true) } } eos = true @@ -568,9 +566,9 @@ private[spark] class PythonRunner(funcs: Seq[ChainedPythonFunctions]) startTime: Long, env: SparkEnv, worker: Socket, - releasedOrClosed: AtomicBoolean, + released: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { 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 b6b0cac910d69..cdce371dfcbfa 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,8 +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.10.8.1-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.7-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index c6f75ad513327..f74f09f412845 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -19,8 +19,8 @@ package org.apache.spark.api.python import java.io.{DataInputStream, DataOutputStream, EOFException, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} +import java.nio.charset.StandardCharsets import java.util.Arrays -import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable @@ -35,7 +35,7 @@ import org.apache.spark.util.{RedirectThread, Utils} private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], requestedEnvVars: Map[String, String], condaInstructions: Option[CondaSetupInstructions]) - extends Logging { self => + extends Logging { import PythonWorkerFactory._ @@ -43,7 +43,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], // pyspark/daemon.py (by default) and tell it to fork new workers for our tasks. This daemon // currently only works on UNIX-based systems now because it uses signals for child management, // so we can also fall back to launching workers, pyspark/worker.py (by default) directly. - private val useDaemon = { + val useDaemon = { val useDaemonEnabled = SparkEnv.get.conf.getBoolean("spark.python.use.daemon", true) // This flag is ignored on Windows as it's unable to fork. @@ -55,42 +55,34 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], // as expert-only option, and shouldn't be used before knowing what it means exactly. // This configuration indicates the module to run the daemon to execute its Python workers. - private val daemonModule = - SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value => - logInfo( - s"Python daemon module in PySpark is set to [$value] in 'spark.python.daemon.module', " + - "using this to start the daemon up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is enabled and the platform is not Windows.") - value - }.getOrElse("pyspark.daemon") + val daemonModule = SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value => + logInfo( + s"Python daemon module in PySpark is set to [$value] in 'spark.python.daemon.module', " + + "using this to start the daemon up. Note that this configuration only has an effect when " + + "'spark.python.use.daemon' is enabled and the platform is not Windows.") + value + }.getOrElse("pyspark.daemon") // This configuration indicates the module to run each Python worker. - private val workerModule = - SparkEnv.get.conf.getOption("spark.python.worker.module").map { value => - logInfo( - s"Python worker module in PySpark is set to [$value] in 'spark.python.worker.module', " + - "using this to start the worker up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is disabled or the platform is Windows.") - value - }.getOrElse("pyspark.worker") + val workerModule = SparkEnv.get.conf.getOption("spark.python.worker.module").map { value => + logInfo( + s"Python worker module in PySpark is set to [$value] in 'spark.python.worker.module', " + + "using this to start the worker up. Note that this configuration only has an effect when " + + "'spark.python.use.daemon' is disabled or the platform is Windows.") + value + }.getOrElse("pyspark.worker") private val authHelper = new SocketAuthHelper(SparkEnv.get.conf) - @GuardedBy("self") - private var daemon: Process = null + var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) - @GuardedBy("self") - private var daemonPort: Int = 0 - @GuardedBy("self") - private val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() - @GuardedBy("self") - private val idleWorkers = new mutable.Queue[Socket]() - @GuardedBy("self") - private var lastActivity = 0L + var daemonPort: Int = 0 + val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + val idleWorkers = new mutable.Queue[Socket]() + var lastActivity = 0L new MonitorThread().start() - @GuardedBy("self") - private val simpleWorkers = new mutable.WeakHashMap[Socket, Process]() + var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() private[this] val condaEnv = { // Set up conda environment if there are any conda packages requested @@ -118,8 +110,8 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], def create(): Socket = { if (useDaemon) { - self.synchronized { - if (idleWorkers.nonEmpty) { + synchronized { + if (idleWorkers.size > 0) { return idleWorkers.dequeue() } } @@ -148,7 +140,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], socket } - self.synchronized { + synchronized { // Start the daemon if it hasn't been started startDaemon() @@ -194,9 +186,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], try { val socket = serverSocket.accept() authHelper.authClient(socket) - self.synchronized { - simpleWorkers.put(socket, worker) - } + simpleWorkers.put(socket, worker) return socket } catch { case e: Exception => @@ -211,7 +201,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], } private def startDaemon() { - self.synchronized { + synchronized { // Is it already running? if (daemon != null) { return @@ -311,7 +301,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], override def run() { while (true) { - self.synchronized { + synchronized { if (lastActivity + IDLE_WORKER_TIMEOUT_MS < System.currentTimeMillis()) { cleanupIdleWorkers() lastActivity = System.currentTimeMillis() @@ -336,7 +326,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], } private def stopDaemon() { - self.synchronized { + synchronized { if (useDaemon) { cleanupIdleWorkers() @@ -358,7 +348,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], } def stopWorker(worker: Socket) { - self.synchronized { + synchronized { if (useDaemon) { if (daemon != null) { daemonWorkers.get(worker).foreach { pid => @@ -378,7 +368,7 @@ private[spark] class PythonWorkerFactory(requestedPythonExec: Option[String], def releaseWorker(worker: Socket) { if (useDaemon) { - self.synchronized { + synchronized { lastActivity = System.currentTimeMillis() idleWorkers.enqueue(worker) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 5979151345415..4cc0063d010ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream, File, IOException} -import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import java.text.DateFormat import java.util.{Arrays, Comparator, Date, Locale} @@ -31,7 +30,7 @@ import scala.util.control.NonFatal import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} @@ -413,6 +412,20 @@ object SparkHadoopUtil { def get: SparkHadoopUtil = instance + /** + * Given an expiration date for the current set of credentials, calculate the time when new + * credentials should be created. + * + * @param expirationDate Drop-dead expiration date + * @param conf Spark configuration + * @return Timestamp when new credentials should be created. + */ + private[spark] def nextCredentialRenewalTime(expirationDate: Long, conf: SparkConf): Long = { + val ct = System.currentTimeMillis + val ratio = conf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) + (ct + (ratio * (expirationDate - ct))).toLong + } + /** * Returns a Configuration object with Spark configuration applied on top. Unlike * the instance method, this will always return a Configuration instance, and not a @@ -458,33 +471,4 @@ object SparkHadoopUtil { hadoopConf.set(key.substring("spark.hadoop.".length), value) } } - - // scalastyle:off line.size.limit - /** - * Create a path that uses replication instead of erasure coding (ec), regardless of the default - * configuration in hdfs for the given path. This can be helpful as hdfs ec doesn't support - * hflush(), hsync(), or append() - * https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations - */ - // scalastyle:on line.size.limit - def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = { - try { - // Use reflection as this uses apis only avialable in hadoop 3 - val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) - val builder = builderMethod.invoke(fs, path) - val builderCls = builder.getClass() - // this may throw a NoSuchMethodException if the path is not on hdfs - val replicateMethod = builderCls.getMethod("replicate") - val buildMethod = builderCls.getMethod("build") - val b2 = replicateMethod.invoke(builder) - buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] - } catch { - case _: NoSuchMethodException => - // No createFile() method, we're using an older hdfs client, which doesn't give us control - // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with - // old apis. - fs.create(path) - } - } - } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 88df7324a354a..d5f2865f87281 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -335,7 +335,7 @@ private[spark] class SparkSubmit extends Logging { val targetDir = Utils.createTempDir() // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) { + if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient) { if (args.principal != null) { if (args.keytab != null) { require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") @@ -646,8 +646,7 @@ private[spark] class SparkSubmit extends Logging { } } - if ((clusterManager == MESOS || clusterManager == KUBERNETES) - && UserGroupInformation.isSecurityEnabled) { + if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } @@ -763,8 +762,8 @@ private[spark] class SparkSubmit extends Logging { } // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with - // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos or Kubernetes - // mode, we must trick it into thinking we're YARN. + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we + // must trick it into thinking we're YARN. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" @@ -928,6 +927,8 @@ object SparkSubmit extends CommandLineUtils with Logging { } catch { case e: SparkUserAppException => exitFn(e.exitCode) + case e: SparkException => + printErrorAndExit(e.getMessage()) } } @@ -992,9 +993,9 @@ private[spark] object SparkSubmitUtils { // Exposed for testing. // These components are used to make the default exclusion rules for Spark dependencies. - // We need to specify each component explicitly, otherwise we miss - // spark-streaming utility components. Underscore is there to differentiate between - // spark-streaming_2.1x and spark-streaming-kafka-0-10-assembly_2.1x + // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka-0-8 and + // other spark-streaming utility components. Underscore is there to differentiate between + // spark-streaming_2.1x and spark-streaming-kafka-0-8-assembly_2.1x val IVY_DEFAULT_EXCLUDES = Seq("catalyst_", "core_", "graphx_", "kvstore_", "launcher_", "mllib_", "mllib-local_", "network-common_", "network-shuffle_", "repl_", "sketch_", "sql_", "streaming_", "tags_", "unsafe_") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index c4517d3dfd931..c23a659e76df1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -34,7 +34,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} +import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException import org.fusesource.leveldbjni.internal.NativeDB @@ -449,7 +449,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) } - if (shouldReloadLog(info, entry)) { + if (info.fileSize < entry.getLen()) { if (info.appId.isDefined && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. @@ -541,24 +541,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } - private[history] def shouldReloadLog(info: LogInfo, entry: FileStatus): Boolean = { - var result = info.fileSize < entry.getLen - if (!result && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { - try { - result = Utils.tryWithResource(fs.open(entry.getPath)) { in => - in.getWrappedStream match { - case dfsIn: DFSInputStream => info.fileSize < dfsIn.getFileLength - case _ => false - } - } - } catch { - case e: Exception => - logDebug(s"Failed to check the length for the file : ${info.logPath}", e) - } - } - result - } - private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { try { val app = load(appId) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 49f00cb10179e..080ba12c2f0d1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -34,21 +34,35 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin @tailrec private def parse(args: List[String]): Unit = { - args match { - case ("--help" | "-h") :: tail => - printUsageAndExit(0) + if (args.length == 1) { + setLogDirectory(args.head) + } else { + args match { + case ("--dir" | "-d") :: value :: tail => + setLogDirectory(value) + parse(tail) - case ("--properties-file") :: value :: tail => - propertiesFile = value - parse(tail) + case ("--help" | "-h") :: tail => + printUsageAndExit(0) - case Nil => + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) - case _ => - printUsageAndExit(1) + case Nil => + + case _ => + printUsageAndExit(1) + } } } + private def setLogDirectory(value: String): Unit = { + logWarning("Setting log directory through the command line is deprecated as of " + + "Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.") + conf.set("spark.history.fs.logDirectory", value) + } + // This mutates the SparkConf, so all accesses to it must be made after this line Utils.loadDefaultSparkProperties(conf, propertiesFile) @@ -59,6 +73,8 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin |Usage: HistoryServer [options] | |Options: + | DIR Deprecated; set spark.history.fs.logDirectory directly + | --dir DIR (-d DIR) Deprecated; set spark.history.fs.logDirectory directly | --properties-file FILE Path to a custom Spark properties file. | Default is conf/spark-defaults.conf. | diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index afa1a5fbba792..22b65abce611a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -138,16 +138,6 @@ private[rest] class StandaloneSubmitRequestServlet( val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") val superviseDriver = sparkProperties.get("spark.driver.supervise") - // The semantics of "spark.master" and the masterUrl are different. While the - // property "spark.master" could contain all registered masters, masterUrl - // contains only the active master. To make sure a Spark driver can recover - // in a multi-master setup, we use the "spark.master" property while submitting - // the driver. - val masters = sparkProperties.get("spark.master") - val (_, masterPort) = Utils.extractHostPortFromSparkUrl(masterUrl) - val masterRestPort = this.conf.getInt("spark.master.rest.port", 6066) - val updatedMasters = masters.map( - _.replace(s":$masterRestPort", s":$masterPort")).getOrElse(masterUrl) val appArgs = request.appArgs // Filter SPARK_LOCAL_(IP|HOSTNAME) environment variables from being set on the remote system. val environmentVariables = @@ -156,7 +146,7 @@ private[rest] class StandaloneSubmitRequestServlet( // Construct driver description val conf = new SparkConf(false) .setAll(sparkProperties) - .set("spark.master", updatedMasters) + .set("spark.master", masterUrl) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 10cd8742f2b49..ab8d8d96a9b08 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -17,158 +17,76 @@ package org.apache.spark.deploy.security -import java.io.File -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicReference - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.ThreadUtils /** - * Manager for delegation tokens in a Spark application. - * - * This manager has two modes of operation: - * - * 1. When configured with a principal and a keytab, it will make sure long-running apps can run - * without interruption while accessing secured services. It periodically logs in to the KDC with - * user-provided credentials, and contacts all the configured secure services to obtain delegation - * tokens to be distributed to the rest of the application. - * - * Because the Hadoop UGI API does not expose the TTL of the TGT, a configuration controls how often - * to check that a relogin is necessary. This is done reasonably often since the check is a no-op - * when the relogin is not yet needed. The check period can be overridden in the configuration. + * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to + * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], + * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not + * explicitly disabled. * - * New delegation tokens are created once 75% of the renewal interval of the original tokens has - * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. - * The driver is tasked with distributing the tokens to other processes that might need them. + * Also, each HadoopDelegationTokenProvider is controlled by + * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to + * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be + * enabled/disabled by the configuration spark.security.credentials.hive.enabled. * - * 2. When operating without an explicit principal and keytab, token renewal will not be available. - * Starting the manager will distribute an initial set of delegation tokens to the provided Spark - * driver, but the app will not get new tokens when those expire. - * - * It can also be used just to create delegation tokens, by calling the `obtainDelegationTokens` - * method. This option does not require calling the `start` method, but leaves it up to the - * caller to distribute the tokens that were generated. + * @param sparkConf Spark configuration + * @param hadoopConf Hadoop configuration + * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. */ private[spark] class HadoopDelegationTokenManager( - protected val sparkConf: SparkConf, - protected val hadoopConf: Configuration) extends Logging { + sparkConf: SparkConf, + hadoopConf: Configuration, + fileSystems: Configuration => Set[FileSystem]) + extends Logging { private val deprecatedProviderEnabledConfigs = List( "spark.yarn.security.tokens.%s.enabled", "spark.yarn.security.credentials.%s.enabled") private val providerEnabledConfig = "spark.security.credentials.%s.enabled" - private val principal = sparkConf.get(PRINCIPAL).orNull - private val keytab = sparkConf.get(KEYTAB).orNull - - require((principal == null) == (keytab == null), - "Both principal and keytab must be defined, or neither.") - require(keytab == null || new File(keytab).isFile(), s"Cannot find keytab at $keytab.") - - private val delegationTokenProviders = loadProviders() + // Maintain all the registered delegation token providers + private val delegationTokenProviders = getDelegationTokenProviders logDebug("Using the following builtin delegation token providers: " + s"${delegationTokenProviders.keys.mkString(", ")}.") - private var renewalExecutor: ScheduledExecutorService = _ - private val driverRef = new AtomicReference[RpcEndpointRef]() - - /** Set the endpoint used to send tokens to the driver. */ - def setDriverRef(ref: RpcEndpointRef): Unit = { - driverRef.set(ref) + /** Construct a [[HadoopDelegationTokenManager]] for the default Hadoop filesystem */ + def this(sparkConf: SparkConf, hadoopConf: Configuration) = { + this( + sparkConf, + hadoopConf, + hadoopConf => Set(FileSystem.get(hadoopConf).getHomeDirectory.getFileSystem(hadoopConf))) } - /** @return Whether delegation token renewal is enabled. */ - def renewalEnabled: Boolean = principal != null - - /** - * Start the token renewer. Requires a principal and keytab. Upon start, the renewer will: - * - * - log in the configured principal, and set up a task to keep that user's ticket renewed - * - obtain delegation tokens from all available providers - * - send the tokens to the driver, if it's already registered - * - schedule a periodic task to update the tokens when needed. - * - * @return The newly logged in user. - */ - def start(): UserGroupInformation = { - require(renewalEnabled, "Token renewal must be enabled to start the renewer.") - renewalExecutor = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") - - val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() - val ugi = doLogin() - - val tgtRenewalTask = new Runnable() { - override def run(): Unit = { - ugi.checkTGTAndReloginFromKeytab() - } - } - val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) - renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, - TimeUnit.SECONDS) - - val creds = obtainTokensAndScheduleRenewal(ugi) - ugi.addCredentials(creds) - - val driver = driverRef.get() - if (driver != null) { - val tokens = SparkHadoopUtil.get.serialize(creds) - driver.send(UpdateDelegationTokens(tokens)) - } - - // Transfer the original user's tokens to the new user, since it may contain needed tokens - // (such as those user to connect to YARN). Explicitly avoid overwriting tokens that already - // exist in the current user's credentials, since those were freshly obtained above - // (see SPARK-23361). - val existing = ugi.getCredentials() - existing.mergeAll(originalCreds) - ugi.addCredentials(existing) - ugi - } + private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { + val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++ + safeCreateProvider(new HiveDelegationTokenProvider) ++ + safeCreateProvider(new HBaseDelegationTokenProvider) - def stop(): Unit = { - if (renewalExecutor != null) { - renewalExecutor.shutdown() - } + // Filter out providers for which spark.security.credentials.{service}.enabled is false. + providers + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap } - /** - * Fetch new delegation tokens for configured services, storing them in the given credentials. - * Tokens are fetched for the current logged in user. - * - * @param creds Credentials object where to store the delegation tokens. - * @return The time by which the tokens must be renewed. - */ - def obtainDelegationTokens(creds: Credentials): Long = { - delegationTokenProviders.values.flatMap { provider => - if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { - provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) - } else { - logDebug(s"Service ${provider.serviceName} does not require a token." + - s" Check your configuration to see if security is disabled or not.") + private def safeCreateProvider( + createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { + try { + Some(createFn) + } catch { + case t: Throwable => + logDebug(s"Failed to load built in provider.", t) None - } - }.foldLeft(Long.MaxValue)(math.min) - } - - // Visible for testing. - def isProviderLoaded(serviceName: String): Boolean = { - delegationTokenProviders.contains(serviceName) + } } - protected def isServiceEnabled(serviceName: String): Boolean = { + def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) deprecatedProviderEnabledConfigs.foreach { pattern => @@ -192,104 +110,32 @@ private[spark] class HadoopDelegationTokenManager( } /** - * List of file systems for which to obtain delegation tokens. The base implementation - * returns just the default file system in the given Hadoop configuration. + * Get delegation token provider for the specified service. */ - protected def fileSystemsToAccess(): Set[FileSystem] = { - Set(FileSystem.get(hadoopConf)) - } - - private def scheduleRenewal(delay: Long): Unit = { - val _delay = math.max(0, delay) - logInfo(s"Scheduling login from keytab in ${UIUtils.formatDuration(delay)}.") - - val renewalTask = new Runnable() { - override def run(): Unit = { - updateTokensTask() - } - } - renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) + def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { + delegationTokenProviders.get(service) } /** - * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself - * to fetch the next set of tokens when needed. - */ - private def updateTokensTask(): Unit = { - try { - val freshUGI = doLogin() - val creds = obtainTokensAndScheduleRenewal(freshUGI) - val tokens = SparkHadoopUtil.get.serialize(creds) - - val driver = driverRef.get() - if (driver != null) { - logInfo("Updating delegation tokens.") - driver.send(UpdateDelegationTokens(tokens)) - } else { - // This shouldn't really happen, since the driver should register way before tokens expire. - logWarning("Delegation tokens close to expiration but no driver has registered yet.") - SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) - } - } catch { - case e: Exception => - val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) - logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + - " If this happens too often tasks will fail.", e) - scheduleRenewal(delay) - } - } - - /** - * Obtain new delegation tokens from the available providers. Schedules a new task to fetch - * new tokens before the new set expires. + * Writes delegation tokens to creds. Delegation tokens are fetched from all registered + * providers. * - * @return Credentials containing the new tokens. + * @param hadoopConf hadoop Configuration + * @param creds Credentials that will be updated in place (overwritten) + * @return Time after which the fetched delegation tokens should be renewed. */ - private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { - ugi.doAs(new PrivilegedExceptionAction[Credentials]() { - override def run(): Credentials = { - val creds = new Credentials() - val nextRenewal = obtainDelegationTokens(creds) - - // Calculate the time when new credentials should be created, based on the configured - // ratio. - val now = System.currentTimeMillis - val ratio = sparkConf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) - val delay = (ratio * (nextRenewal - now)).toLong - scheduleRenewal(delay) - creds - } - }) - } - - private def doLogin(): UserGroupInformation = { - logInfo(s"Attempting to login to KDC using principal: $principal") - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - logInfo("Successfully logged into KDC.") - ugi - } - - private def loadProviders(): Map[String, HadoopDelegationTokenProvider] = { - val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystemsToAccess)) ++ - safeCreateProvider(new HiveDelegationTokenProvider) ++ - safeCreateProvider(new HBaseDelegationTokenProvider) - - // Filter out providers for which spark.security.credentials.{service}.enabled is false. - providers - .filter { p => isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap - } - - private def safeCreateProvider( - createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { - try { - Some(createFn) - } catch { - case t: Throwable => - logDebug(s"Failed to load built in provider.", t) + def obtainDelegationTokens( + hadoopConf: Configuration, + creds: Credentials): Long = { + delegationTokenProviders.values.flatMap { provider => + if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { + provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") None - } + } + }.foldLeft(Long.MaxValue)(math.min) } - } + diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 767b5521e8d7b..21ca669ea98f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -30,7 +30,7 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => Set[FileSystem]) +private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration => Set[FileSystem]) extends HadoopDelegationTokenProvider with Logging { // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. @@ -44,7 +44,8 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => Set[Fil hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] = { - val fsToGetTokens = fileSystems() + + val fsToGetTokens = fileSystems(hadoopConf) val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) // Get the token renewal interval if it is not set. It will only be called once. diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 1befd27de1cba..2933f3ba6d3b5 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -28,7 +28,7 @@ import org.apache.spark.metrics.ExecutorMetricType @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - // Metrics are indexed by ExecutorMetricType.values + // Metrics are indexed by MetricGetter.values private val metrics = new Array[Long](ExecutorMetricType.values.length) // the first element is initialized to -1, indicating that the values for the array diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index 5b33c110154d6..ab020aaf6fa4f 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -52,18 +52,6 @@ private[spark] abstract class StreamFileInputFormat[T] val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum val bytesPerCore = totalBytes / defaultParallelism val maxSplitSize = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) - - // For small files we need to ensure the min split size per node & rack <= maxSplitSize - val jobConfig = context.getConfiguration - val minSplitSizePerNode = jobConfig.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 0L) - val minSplitSizePerRack = jobConfig.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 0L) - - if (maxSplitSize < minSplitSizePerNode) { - super.setMinSplitSizeNode(maxSplitSize) - } - if (maxSplitSize < minSplitSizePerRack) { - super.setMinSplitSizeRack(maxSplitSize) - } super.setMaxSplitSize(maxSplitSize) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index ba617433928e4..458df0ba97e24 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -58,11 +58,6 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val EVENT_LOG_ALLOW_EC = - ConfigBuilder("spark.eventLog.allowErasureCoding") - .booleanConf - .createWithDefault(false) - private[spark] val EVENT_LOG_TESTING = ConfigBuilder("spark.eventLog.testing") .internal() @@ -179,10 +174,6 @@ package object config { .doc("Name of the Kerberos principal.") .stringConf.createOptional - private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.kerberos.relogin.period") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("1m") - private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") .intConf .createOptional diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0664c5ac752c1..7722db56ee297 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -154,19 +154,72 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { */ @DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { + val version = SnappyCompressionCodec.version - try { + override def compressedOutputStream(s: OutputStream): OutputStream = { + val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt + new SnappyOutputStreamWrapper(new SnappyOutputStream(s, blockSize)) + } + + override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) +} + +/** + * Object guards against memory leak bug in snappy-java library: + * (https://github.com/xerial/snappy-java/issues/131). + * Before a new version of the library, we only call the method once and cache the result. + */ +private final object SnappyCompressionCodec { + private lazy val version: String = try { Snappy.getNativeLibraryVersion } catch { case e: Error => throw new IllegalArgumentException(e) } +} - override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt - new SnappyOutputStream(s, blockSize) +/** + * Wrapper over `SnappyOutputStream` which guards against write-after-close and double-close + * issues. See SPARK-7660 for more details. This wrapping can be removed if we upgrade to a version + * of snappy-java that contains the fix for https://github.com/xerial/snappy-java/issues/107. + */ +private final class SnappyOutputStreamWrapper(os: SnappyOutputStream) extends OutputStream { + + private[this] var closed: Boolean = false + + override def write(b: Int): Unit = { + if (closed) { + throw new IOException("Stream is closed") + } + os.write(b) } - override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s) + override def write(b: Array[Byte]): Unit = { + if (closed) { + throw new IOException("Stream is closed") + } + os.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + if (closed) { + throw new IOException("Stream is closed") + } + os.write(b, off, len) + } + + override def flush(): Unit = { + if (closed) { + throw new IOException("Stream is closed") + } + os.flush() + } + + override def close(): Unit = { + if (!closed) { + closed = true + os.close() + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index a58c8fa2e763f..eef8c31e05ab1 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} -import org.apache.spark.storage.{BlockId, EncryptedManagedBuffer, StorageLevel} +import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.ThreadUtils private[spark] @@ -104,8 +104,6 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo data match { case f: FileSegmentManagedBuffer => result.success(f) - case e: EncryptedManagedBuffer => - result.success(e) case _ => val ret = ByteBuffer.allocate(data.size.toInt) ret.put(data.nioByteBuffer()) diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index d745345f4e0d2..0a5fe5a1d3ee1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Information about an [[org.apache.spark.util.AccumulatorV2]] modified during a task or stage. + * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. * * @param id accumulator ID * @param name accumulator name diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala index 7cd2b862216ee..e2b6df4600590 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -169,8 +169,7 @@ private class AsyncEventQueue( val prevLastReportTimestamp = lastReportTimestamp lastReportTimestamp = System.currentTimeMillis() val previous = new java.util.Date(prevLastReportTimestamp) - logWarning(s"Dropped $droppedCount events from $name since " + - s"${if (prevLastReportTimestamp == 0) "the application started" else s"$previous"}.") + logWarning(s"Dropped $droppedCount events from $name since $previous.") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 06966e77db81e..f93d8a8d5de55 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -265,7 +265,7 @@ private[spark] class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, - // executor metrics indexed by ExecutorMetricType.values + // executor metrics indexed by MetricGetter.values executorUpdates: ExecutorMetrics): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, Some(executorUpdates))) @@ -1295,27 +1295,6 @@ private[spark] class DAGScheduler( Utils.getFormattedClassName(event.task), event.reason, event.taskInfo, taskMetrics)) } - /** - * Check [[SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL]] in job properties to see if we should - * interrupt running tasks. Returns `false` if the property value is not a boolean value - */ - private def shouldInterruptTaskThread(job: ActiveJob): Boolean = { - if (job.properties == null) { - false - } else { - val shouldInterruptThread = - job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false") - try { - shouldInterruptThread.toBoolean - } catch { - case e: IllegalArgumentException => - logWarning(s"${SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL} in Job ${job.jobId} " + - s"is invalid: $shouldInterruptThread. Using 'false' instead", e) - false - } - } - } - /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -1385,21 +1364,6 @@ private[spark] class DAGScheduler( if (job.numFinished == job.numPartitions) { markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) - try { - // killAllTaskAttempts will fail if a SchedulerBackend does not implement - // killTask. - logInfo(s"Job ${job.jobId} is finished. Cancelling potential speculative " + - "or zombie tasks for this job") - // ResultStage is only used by this job. It's safe to kill speculative or - // zombie tasks in this stage. - taskScheduler.killAllTaskAttempts( - stageId, - shouldInterruptTaskThread(job), - reason = "Stage finished") - } catch { - case e: UnsupportedOperationException => - logWarning(s"Could not cancel tasks for stage $stageId", e) - } listenerBus.post( SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) } @@ -1409,7 +1373,7 @@ private[spark] class DAGScheduler( try { job.listener.taskSucceeded(rt.outputId, event.result) } catch { - case e: Throwable if !Utils.isFatalError(e) => + case e: Exception => // TODO: Perhaps we want to mark the resultStage as failed? job.listener.jobFailed(new SparkDriverExecutionException(e)) } @@ -1926,6 +1890,10 @@ private[spark] class DAGScheduler( val error = new SparkException(failureReason, exception.getOrElse(null)) var ableToCancelStages = true + val shouldInterruptThread = + if (job.properties == null) false + else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean + // Cancel all independent, running stages. val stages = jobIdToStageIds(job.jobId) if (stages.isEmpty) { @@ -1945,12 +1913,12 @@ private[spark] class DAGScheduler( val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask - taskScheduler.cancelTasks(stageId, shouldInterruptTaskThread(job)) + taskScheduler.cancelTasks(stageId, shouldInterruptThread) markStageAsFinished(stage, Some(failureReason)) } catch { case e: UnsupportedOperationException => - logWarning(s"Could not cancel tasks for stage $stageId", e) - ableToCancelStages = false + logInfo(s"Could not cancel tasks for stage $stageId", e) + ableToCancelStages = false } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 788b23d1bfb03..1629e1797977f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI import java.nio.charset.StandardCharsets +import java.util.EnumSet import java.util.Locale import scala.collection.mutable.{ArrayBuffer, Map} @@ -27,6 +28,8 @@ import scala.collection.mutable.{ArrayBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.hdfs.DFSOutputStream +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ @@ -67,7 +70,6 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) - private val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt @@ -120,11 +122,7 @@ private[spark] class EventLoggingListener( if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { new FileOutputStream(uri.getPath) } else { - hadoopDataStream = Some(if (shouldAllowECLogs) { - fileSystem.create(path) - } else { - SparkHadoopUtil.createNonECFile(fileSystem, path) - }) + hadoopDataStream = Some(fileSystem.create(path)) hadoopDataStream.get } @@ -151,7 +149,10 @@ private[spark] class EventLoggingListener( // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) - hadoopDataStream.foreach(_.hflush()) + hadoopDataStream.foreach(ds => ds.getWrappedStream match { + case wrapped: DFSOutputStream => wrapped.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)) + case _ => ds.hflush() + }) } if (testing) { loggedEvents += eventJson diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 4c6b0c1227b18..226c23733c870 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -118,8 +118,6 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { case e: HaltReplayException => // Just stop replay. case _: EOFException if maybeTruncated => - case _: IOException if maybeTruncated => - logWarning(s"Failed to read Spark event log: $sourceName") case ioe: IOException => throw ioe case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index e92b8a2718df0..293e8369677f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -175,7 +175,7 @@ case class SparkListenerExecutorMetricsUpdate( * @param execId executor id * @param stageId stage id * @param stageAttemptId stage attempt - * @param executorMetrics executor level metrics, indexed by ExecutorMetricType.values + * @param executorMetrics executor level metrics, indexed by MetricGetter.values */ @DeveloperApi case class SparkListenerStageExecutorMetrics( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index ad2c50a76a7f9..cedeca1a20904 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -18,17 +18,13 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} +import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future -import org.apache.hadoop.security.UserGroupInformation - import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ @@ -99,12 +95,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 - // Current set of delegation tokens to send to executors. - private val delegationTokens = new AtomicReference[Array[Byte]]() - - // The token manager used to create security tokens. - private var delegationTokenManager: Option[HadoopDelegationTokenManager] = None - private val reviveThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") @@ -162,8 +152,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } case UpdateDelegationTokens(newDelegationTokens) => - SparkHadoopUtil.get.addDelegationTokens(newDelegationTokens, conf) - delegationTokens.set(newDelegationTokens) executorDataMap.values.foreach { ed => ed.executorEndpoint.send(UpdateDelegationTokens(newDelegationTokens)) } @@ -242,7 +230,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), - Option(delegationTokens.get())) + fetchHadoopDelegationTokens()) context.reply(reply) } @@ -402,21 +390,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // TODO (prashant) send conf instead of properties driverEndpoint = createDriverEndpointRef(properties) - - if (UserGroupInformation.isSecurityEnabled()) { - delegationTokenManager = createTokenManager() - delegationTokenManager.foreach { dtm => - dtm.setDriverRef(driverEndpoint) - val creds = if (dtm.renewalEnabled) { - dtm.start().getCredentials() - } else { - val creds = UserGroupInformation.getCurrentUser().getCredentials() - dtm.obtainDelegationTokens(creds) - creds - } - delegationTokens.set(SparkHadoopUtil.get.serialize(creds)) - } - } } protected def createDriverEndpointRef( @@ -443,7 +416,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def stop() { reviveThread.shutdownNow() stopExecutors() - delegationTokenManager.foreach(_.stop()) try { if (driverEndpoint != null) { driverEndpoint.askSync[Boolean](StopDriver) @@ -712,13 +684,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp true } - /** - * Create the delegation token manager to be used for the application. This method is called - * once during the start of the scheduler backend (so after the object has already been - * fully constructed), only if security is enabled in the Hadoop configuration. - */ - protected def createTokenManager(): Option[HadoopDelegationTokenManager] = None - + protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala index 18b735b8035ab..00621976b77f4 100644 --- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.security -import java.io.{Closeable, InputStream, IOException, OutputStream} +import java.io.{InputStream, OutputStream} import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, WritableByteChannel} import java.util.Properties @@ -54,10 +54,8 @@ private[spark] object CryptoStreamUtils extends Logging { val params = new CryptoParams(key, sparkConf) val iv = createInitializationVector(params.conf) os.write(iv) - new ErrorHandlingOutputStream( - new CryptoOutputStream(params.transformation, params.conf, os, params.keySpec, - new IvParameterSpec(iv)), - os) + new CryptoOutputStream(params.transformation, params.conf, os, params.keySpec, + new IvParameterSpec(iv)) } /** @@ -72,10 +70,8 @@ private[spark] object CryptoStreamUtils extends Logging { val helper = new CryptoHelperChannel(channel) helper.write(ByteBuffer.wrap(iv)) - new ErrorHandlingWritableChannel( - new CryptoOutputStream(params.transformation, params.conf, helper, params.keySpec, - new IvParameterSpec(iv)), - helper) + new CryptoOutputStream(params.transformation, params.conf, helper, params.keySpec, + new IvParameterSpec(iv)) } /** @@ -88,10 +84,8 @@ private[spark] object CryptoStreamUtils extends Logging { val iv = new Array[Byte](IV_LENGTH_IN_BYTES) ByteStreams.readFully(is, iv) val params = new CryptoParams(key, sparkConf) - new ErrorHandlingInputStream( - new CryptoInputStream(params.transformation, params.conf, is, params.keySpec, - new IvParameterSpec(iv)), - is) + new CryptoInputStream(params.transformation, params.conf, is, params.keySpec, + new IvParameterSpec(iv)) } /** @@ -106,10 +100,8 @@ private[spark] object CryptoStreamUtils extends Logging { JavaUtils.readFully(channel, buf) val params = new CryptoParams(key, sparkConf) - new ErrorHandlingReadableChannel( - new CryptoInputStream(params.transformation, params.conf, channel, params.keySpec, - new IvParameterSpec(iv)), - channel) + new CryptoInputStream(params.transformation, params.conf, channel, params.keySpec, + new IvParameterSpec(iv)) } def toCryptoConf(conf: SparkConf): Properties = { @@ -165,117 +157,6 @@ private[spark] object CryptoStreamUtils extends Logging { } - /** - * SPARK-25535. The commons-cryto library will throw InternalError if something goes - * wrong, and leave bad state behind in the Java wrappers, so it's not safe to use them - * afterwards. This wrapper detects that situation and avoids further calls into the - * commons-crypto code, while still allowing the underlying streams to be closed. - * - * This should be removed once CRYPTO-141 is fixed (and Spark upgrades its commons-crypto - * dependency). - */ - trait BaseErrorHandler extends Closeable { - - private var closed = false - - /** The encrypted stream that may get into an unhealthy state. */ - protected def cipherStream: Closeable - - /** - * The underlying stream that is being wrapped by the encrypted stream, so that it can be - * closed even if there's an error in the crypto layer. - */ - protected def original: Closeable - - protected def safeCall[T](fn: => T): T = { - if (closed) { - throw new IOException("Cipher stream is closed.") - } - try { - fn - } catch { - case ie: InternalError => - closed = true - original.close() - throw ie - } - } - - override def close(): Unit = { - if (!closed) { - cipherStream.close() - } - } - - } - - // Visible for testing. - class ErrorHandlingReadableChannel( - protected val cipherStream: ReadableByteChannel, - protected val original: ReadableByteChannel) - extends ReadableByteChannel with BaseErrorHandler { - - override def read(src: ByteBuffer): Int = safeCall { - cipherStream.read(src) - } - - override def isOpen(): Boolean = cipherStream.isOpen() - - } - - private class ErrorHandlingInputStream( - protected val cipherStream: InputStream, - protected val original: InputStream) - extends InputStream with BaseErrorHandler { - - override def read(b: Array[Byte]): Int = safeCall { - cipherStream.read(b) - } - - override def read(b: Array[Byte], off: Int, len: Int): Int = safeCall { - cipherStream.read(b, off, len) - } - - override def read(): Int = safeCall { - cipherStream.read() - } - } - - private class ErrorHandlingWritableChannel( - protected val cipherStream: WritableByteChannel, - protected val original: WritableByteChannel) - extends WritableByteChannel with BaseErrorHandler { - - override def write(src: ByteBuffer): Int = safeCall { - cipherStream.write(src) - } - - override def isOpen(): Boolean = cipherStream.isOpen() - - } - - private class ErrorHandlingOutputStream( - protected val cipherStream: OutputStream, - protected val original: OutputStream) - extends OutputStream with BaseErrorHandler { - - override def flush(): Unit = safeCall { - cipherStream.flush() - } - - override def write(b: Array[Byte]): Unit = safeCall { - cipherStream.write(b) - } - - override def write(b: Array[Byte], off: Int, len: Int): Unit = safeCall { - cipherStream.write(b, off, len) - } - - override def write(b: Int): Unit = safeCall { - cipherStream.write(b) - } - } - private class CryptoParams(key: Array[Byte], sparkConf: SparkConf) { val keySpec = new SecretKeySpec(key, "AES") diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index e2c190ea198e0..36aaf67b57298 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -44,7 +44,6 @@ private[spark] class AppStatusListener( kvstore: ElementTrackingStore, conf: SparkConf, live: Boolean, - appStatusSource: Option[AppStatusSource] = None, lastUpdateTime: Option[Long] = None) extends SparkListener with Logging { import config._ @@ -281,11 +280,6 @@ private[spark] class AppStatusListener( private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = { liveExecutors.get(execId).foreach { exec => exec.isBlacklisted = blacklisted - if (blacklisted) { - appStatusSource.foreach(_.BLACKLISTED_EXECUTORS.inc()) - } else { - appStatusSource.foreach(_.UNBLACKLISTED_EXECUTORS.inc()) - } liveUpdate(exec, System.nanoTime()) } } @@ -388,34 +382,11 @@ private[spark] class AppStatusListener( } job.status = event.jobResult match { - case JobSucceeded => - appStatusSource.foreach{_.SUCCEEDED_JOBS.inc()} - JobExecutionStatus.SUCCEEDED - case JobFailed(_) => - appStatusSource.foreach{_.FAILED_JOBS.inc()} - JobExecutionStatus.FAILED + case JobSucceeded => JobExecutionStatus.SUCCEEDED + case JobFailed(_) => JobExecutionStatus.FAILED } job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None - - for { - source <- appStatusSource - submissionTime <- job.submissionTime - completionTime <- job.completionTime - } { - source.JOB_DURATION.value.set(completionTime.getTime() - submissionTime.getTime()) - } - - // update global app status counters - appStatusSource.foreach { source => - source.COMPLETED_STAGES.inc(job.completedStages.size) - source.FAILED_STAGES.inc(job.failedStages) - source.COMPLETED_TASKS.inc(job.completedTasks) - source.FAILED_TASKS.inc(job.failedTasks) - source.KILLED_TASKS.inc(job.killedTasks) - source.SKIPPED_TASKS.inc(job.skippedTasks) - source.SKIPPED_STAGES.inc(job.skippedStages.size) - } update(job, now, last = true) if (job.status == JobExecutionStatus.SUCCEEDED) { appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages) @@ -1073,6 +1044,16 @@ private[spark] class AppStatusListener( kvstore.delete(e.getClass(), e.id) } + val tasks = kvstore.view(classOf[TaskDataWrapper]) + .index("stage") + .first(key) + .last(key) + .asScala + + tasks.foreach { t => + kvstore.delete(t.getClass(), t.taskId) + } + // Check whether there are remaining attempts for the same stage. If there aren't, then // also delete the RDD graph data. val remainingAttempts = kvstore.view(classOf[StageDataWrapper]) @@ -1095,15 +1076,6 @@ private[spark] class AppStatusListener( cleanupCachedQuantiles(key) } - - // Delete tasks for all stages in one pass, as deleting them for each stage individually is slow - val tasks = kvstore.view(classOf[TaskDataWrapper]).asScala - val keys = stages.map { s => (s.info.stageId, s.info.attemptId) }.toSet - tasks.foreach { t => - if (keys.contains((t.stageId, t.stageAttemptId))) { - kvstore.delete(t.getClass(), t.taskId) - } - } } private def cleanupTasks(stage: LiveStage): Unit = { diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala deleted file mode 100644 index 3ab293dd648b5..0000000000000 --- a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.status - -import java.util.concurrent.atomic.AtomicLong - -import AppStatusSource.getCounter -import com.codahale.metrics.{Counter, Gauge, MetricRegistry} - -import org.apache.spark.SparkConf -import org.apache.spark.internal.config.ConfigBuilder -import org.apache.spark.metrics.source.Source - -private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] { - override def getValue: Long = value.get() -} - -private[spark] class AppStatusSource extends Source { - - override implicit val metricRegistry = new MetricRegistry() - - override val sourceName = "appStatus" - - val jobDuration = new JobDuration(new AtomicLong(0L)) - - // Duration of each job in milliseconds - val JOB_DURATION = metricRegistry - .register(MetricRegistry.name("jobDuration"), jobDuration) - - val FAILED_STAGES = getCounter("stages", "failedStages") - - val SKIPPED_STAGES = getCounter("stages", "skippedStages") - - val COMPLETED_STAGES = getCounter("stages", "completedStages") - - val SUCCEEDED_JOBS = getCounter("jobs", "succeededJobs") - - val FAILED_JOBS = getCounter("jobs", "failedJobs") - - val COMPLETED_TASKS = getCounter("tasks", "completedTasks") - - val FAILED_TASKS = getCounter("tasks", "failedTasks") - - val KILLED_TASKS = getCounter("tasks", "killedTasks") - - val SKIPPED_TASKS = getCounter("tasks", "skippedTasks") - - val BLACKLISTED_EXECUTORS = getCounter("tasks", "blackListedExecutors") - - val UNBLACKLISTED_EXECUTORS = getCounter("tasks", "unblackListedExecutors") -} - -private[spark] object AppStatusSource { - - def getCounter(prefix: String, name: String)(implicit metricRegistry: MetricRegistry): Counter = { - metricRegistry.counter(MetricRegistry.name(prefix, name)) - } - - def createSource(conf: SparkConf): Option[AppStatusSource] = { - Option(conf.get(AppStatusSource.APP_STATUS_METRICS_ENABLED)) - .filter(identity) - .map { _ => new AppStatusSource() } - } - - val APP_STATUS_METRICS_ENABLED = - ConfigBuilder("spark.app.status.metrics.enabled") - .doc("Whether Dropwizard/Codahale metrics " + - "will be reported for the status of the running spark app.") - .booleanConf - .createWithDefault(false) -} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 63b9d8988499d..e237281c552b1 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -112,12 +112,10 @@ private[spark] class AppStatusStore( } } - def stageAttempt(stageId: Int, stageAttemptId: Int, - details: Boolean = false): (v1.StageData, Seq[Int]) = { + def stageAttempt(stageId: Int, stageAttemptId: Int, details: Boolean = false): v1.StageData = { val stageKey = Array(stageId, stageAttemptId) - val stageDataWrapper = store.read(classOf[StageDataWrapper], stageKey) - val stage = if (details) stageWithDetails(stageDataWrapper.info) else stageDataWrapper.info - (stage, stageDataWrapper.jobIds.toSeq) + val stage = store.read(classOf[StageDataWrapper], stageKey).info + if (details) stageWithDetails(stage) else stage } def taskCount(stageId: Int, stageAttemptId: Int): Long = { @@ -505,11 +503,10 @@ private[spark] object AppStatusStore { /** * Create an in-memory store for a live application. */ - def createLiveStore( - conf: SparkConf, - appStatusSource: Option[AppStatusSource] = None): AppStatusStore = { + def createLiveStore(conf: SparkConf): AppStatusStore = { val store = new ElementTrackingStore(new InMemoryStore(), conf) - val listener = new AppStatusListener(store, conf, true, appStatusSource) + val listener = new AppStatusListener(store, conf, true) new AppStatusStore(store, listener = Some(listener)) } + } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 80663318c1ba1..8708e64db3c17 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -61,7 +61,7 @@ private[spark] abstract class LiveEntity { private class LiveJob( val jobId: Int, name: String, - val submissionTime: Option[Date], + submissionTime: Option[Date], val stageIds: Seq[Int], jobGroup: Option[String], numTasks: Int) extends LiveEntity { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 30d52b97833e6..96249e4bfd5fa 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -56,7 +56,7 @@ private[v1] class StagesResource extends BaseAppResource { @PathParam("stageAttemptId") stageAttemptId: Int, @QueryParam("details") @DefaultValue("true") details: Boolean): StageData = withUI { ui => try { - ui.store.stageAttempt(stageId, stageAttemptId, details = details)._1 + ui.store.stageAttempt(stageId, stageAttemptId, details = details) } catch { case _: NoSuchElementException => // Change the message depending on whether there are any attempts for the requested stage. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 2361dd9aafea7..5ee6e8dfd4fd6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -133,6 +133,8 @@ private[spark] class BlockManager( private[spark] val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED) + private val chunkSize = + conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt private val remoteReadNioBufferConversion = conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) @@ -455,7 +457,7 @@ private[spark] class BlockManager( new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) case None => - ChunkedByteBuffer.fromFile(tmpFile) + ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) } putBytes(blockId, buffer, level)(classTag) tmpFile.delete() @@ -727,7 +729,7 @@ private[spark] class BlockManager( * Get block from remote block managers as serialized bytes. */ def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { - // TODO SPARK-25905 if we change this method to return the ManagedBuffer, then getRemoteValues + // TODO if we change this method to return the ManagedBuffer, then getRemoteValues // could just use the inputStream on the temp file, rather than reading the file into memory. // Until then, replication can cause the process to use too much memory and get killed // even though we've read the data to disk. @@ -801,7 +803,7 @@ private[spark] class BlockManager( if (remoteReadNioBufferConversion) { return Some(new ChunkedByteBuffer(data.nioByteBuffer())) } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data)) + return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) } } logDebug(s"The value of block $blockId is null") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 841e16afc7549..d88bd710d1ead 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -201,7 +201,7 @@ private class DiskBlockData( private def open() = new FileInputStream(file).getChannel } -private[spark] class EncryptedBlockData( +private class EncryptedBlockData( file: File, blockSize: Long, conf: SparkConf, @@ -263,8 +263,7 @@ private[spark] class EncryptedBlockData( } } -private[spark] class EncryptedManagedBuffer( - val blockData: EncryptedBlockData) extends ManagedBuffer { +private class EncryptedManagedBuffer(val blockData: EncryptedBlockData) extends ManagedBuffer { // This is the size of the decrypted data override def size(): Long = blockData.size diff --git a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala index 0bbb10a995bcb..65fa38387b9ee 100644 --- a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils * * @param pageSize the number of rows in a page */ -private[spark] abstract class PagedDataSource[T](val pageSize: Int) { +private[ui] abstract class PagedDataSource[T](val pageSize: Int) { if (pageSize <= 0) { throw new IllegalArgumentException("Page size must be positive") @@ -72,7 +72,7 @@ private[ui] case class PageData[T](totalPage: Int, data: Seq[T]) /** * A paged table that will generate a HTML table for a specified page and also the page navigation. */ -private[spark] trait PagedTable[T] { +private[ui] trait PagedTable[T] { def tableId: String @@ -122,9 +122,13 @@ private[spark] trait PagedTable[T] { /** * Return a page navigation. - * - * It will create a page navigation including a group of page numbers and a form - * to submit the page number. + *
    + *
  • If the totalPages is 1, the page navigation will be empty
  • + *
  • + * If the totalPages is more than 1, it will create a page navigation including a group of + * page numbers and a form to submit the page number. + *
  • + *
* * Here are some examples of the page navigation: * {{{ @@ -150,116 +154,120 @@ private[spark] trait PagedTable[T] { * }}} */ private[ui] def pageNavigation(page: Int, pageSize: Int, totalPages: Int): Seq[Node] = { - // A group includes all page numbers will be shown in the page navigation. - // The size of group is 10 means there are 10 page numbers will be shown. - // The first group is 1 to 10, the second is 2 to 20, and so on - val groupSize = 10 - val firstGroup = 0 - val lastGroup = (totalPages - 1) / groupSize - val currentGroup = (page - 1) / groupSize - val startPage = currentGroup * groupSize + 1 - val endPage = totalPages.min(startPage + groupSize - 1) - val pageTags = (startPage to endPage).map { p => - if (p == page) { - // The current page should be disabled so that it cannot be clicked. -
  • {p}
  • - } else { -
  • {p}
  • + if (totalPages == 1) { + Nil + } else { + // A group includes all page numbers will be shown in the page navigation. + // The size of group is 10 means there are 10 page numbers will be shown. + // The first group is 1 to 10, the second is 2 to 20, and so on + val groupSize = 10 + val firstGroup = 0 + val lastGroup = (totalPages - 1) / groupSize + val currentGroup = (page - 1) / groupSize + val startPage = currentGroup * groupSize + 1 + val endPage = totalPages.min(startPage + groupSize - 1) + val pageTags = (startPage to endPage).map { p => + if (p == page) { + // The current page should be disabled so that it cannot be clicked. +
  • {p}
  • + } else { +
  • {p}
  • + } } - } - val hiddenFormFields = { - if (goButtonFormPath.contains('?')) { - val queryString = goButtonFormPath.split("\\?", 2)(1) - val search = queryString.split("#")(0) - Splitter - .on('&') - .trimResults() - .omitEmptyStrings() - .withKeyValueSeparator("=") - .split(search) - .asScala - .filterKeys(_ != pageSizeFormField) - .filterKeys(_ != prevPageSizeFormField) - .filterKeys(_ != pageNumberFormField) - .mapValues(URLDecoder.decode(_, "UTF-8")) - .map { case (k, v) => - - } - } else { - Seq.empty + val hiddenFormFields = { + if (goButtonFormPath.contains('?')) { + val queryString = goButtonFormPath.split("\\?", 2)(1) + val search = queryString.split("#")(0) + Splitter + .on('&') + .trimResults() + .omitEmptyStrings() + .withKeyValueSeparator("=") + .split(search) + .asScala + .filterKeys(_ != pageSizeFormField) + .filterKeys(_ != prevPageSizeFormField) + .filterKeys(_ != pageNumberFormField) + .mapValues(URLDecoder.decode(_, "UTF-8")) + .map { case (k, v) => + + } + } else { + Seq.empty + } } - } -
    -
    - - {hiddenFormFields} - - +
    + + + {hiddenFormFields} + + - - - + + + - - -
    - +
    -
    + } } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0f74b07a6265c..7428bbe6c5592 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -105,7 +105,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val stageAttemptId = parameterAttempt.toInt val stageHeader = s"Details for Stage $stageId (Attempt $stageAttemptId)" - val (stageData, stageJobIds) = parent.store + val stageData = parent.store .asOption(parent.store.stageAttempt(stageId, stageAttemptId, details = false)) .getOrElse { val content = @@ -183,15 +183,6 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We {Utils.bytesToString(stageData.diskBytesSpilled)} }} - {if (!stageJobIds.isEmpty) { -
  • - Associated Job Ids: - {stageJobIds.map(jobId => {val detailUrl = "%s/jobs/job/?id=%s".format( - UIUtils.prependBaseUri(request, parent.basePath), jobId) - {s"${jobId}"}    - })} -
  • - }} @@ -1057,7 +1048,7 @@ private[ui] object ApiHelper { } def lastStageNameAndDescription(store: AppStatusStore, job: JobData): (String, String) = { - val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)._1) + val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)) (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name)) } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index d5b3ce36e742a..bf618b4afbce0 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -485,3 +485,34 @@ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { _list.addAll(newValue) } } + + +class LegacyAccumulatorWrapper[R, T]( + initialValue: R, + param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { + private[spark] var _value = initialValue // Current value on driver + + @transient private lazy val _zero = param.zero(initialValue) + + override def isZero: Boolean = _value.asInstanceOf[AnyRef].eq(_zero.asInstanceOf[AnyRef]) + + override def copy(): LegacyAccumulatorWrapper[R, T] = { + val acc = new LegacyAccumulatorWrapper(initialValue, param) + acc._value = _value + acc + } + + override def reset(): Unit = { + _value = _zero + } + + override def add(v: T): Unit = _value = param.addAccumulator(_value, v) + + override def merge(other: AccumulatorV2[T, R]): Unit = other match { + case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } + + override def value: R = _value +} diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index 2e517707ff774..a8f10684d5a2c 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -60,14 +60,6 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } } - /** - * Remove all listeners and they won't receive any events. This method is thread-safe and can be - * called in any thread. - */ - final def removeAllListeners(): Unit = { - listenersPlusTimers.clear() - } - /** * This can be overridden by subclasses if there is any extra cleanup to do when removing a * listener. In particular AsyncEventQueues can clean up queues in the LiveListenerBus. diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index da2be84723a07..4aa8d45ec7404 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -29,8 +29,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} -import org.apache.spark.storage.{EncryptedManagedBuffer, StorageUtils} -import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.storage.StorageUtils import org.apache.spark.util.Utils /** @@ -170,27 +169,24 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { } -private[spark] object ChunkedByteBuffer { - - - // TODO SPARK-25905 eliminate this method if we switch BlockManager to getting InputStreams - def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { +object ChunkedByteBuffer { + // TODO eliminate this method if we switch BlockManager to getting InputStreams + def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => - fromFile(f.getFile, f.getOffset, f.getLength) - case e: EncryptedManagedBuffer => - e.blockData.toChunkedByteBuffer(ByteBuffer.allocate _) + fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def fromFile(file: File): ChunkedByteBuffer = { - fromFile(file, 0, file.length()) + def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { + fromFile(file, maxChunkSize, 0, file.length()) } private def fromFile( file: File, + maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = { // We do *not* memory map the file, because we may end up putting this into the memory store, @@ -199,7 +195,7 @@ private[spark] object ChunkedByteBuffer { val is = new FileInputStream(file) ByteStreams.skipFully(is, offset) val in = new LimitedInputStream(is, length) - val chunkSize = math.min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, length).toInt + val chunkSize = math.min(maxChunkSize, length).toInt val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _) Utils.tryWithSafeFinally { IOUtils.copy(in, out) diff --git a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java index 40a7c9486ae55..a6589d2898144 100644 --- a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java +++ b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java @@ -39,28 +39,30 @@ public void setUp() throws ClassNotFoundException, SQLException { sc = new JavaSparkContext("local", "JavaAPISuite"); Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); + Connection connection = + DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb;create=true"); - try (Connection connection = DriverManager.getConnection( - "jdbc:derby:target/JavaJdbcRDDSuiteDb;create=true")) { - - try (Statement create = connection.createStatement()) { - create.execute( - "CREATE TABLE FOO(ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY" + - " (START WITH 1, INCREMENT BY 1), DATA INTEGER)"); - } + try { + Statement create = connection.createStatement(); + create.execute( + "CREATE TABLE FOO(" + + "ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1)," + + "DATA INTEGER)"); + create.close(); - try (PreparedStatement insert = connection.prepareStatement( - "INSERT INTO FOO(DATA) VALUES(?)")) { - for (int i = 1; i <= 100; i++) { - insert.setInt(1, i * 2); - insert.executeUpdate(); - } + PreparedStatement insert = connection.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)"); + for (int i = 1; i <= 100; i++) { + insert.setInt(1, i * 2); + insert.executeUpdate(); } + insert.close(); } catch (SQLException e) { // If table doesn't exist... if (e.getSQLState().compareTo("X0Y32") != 0) { throw e; } + } finally { + connection.close(); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index a07d0e84ea854..0d5c5ea7903e9 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -186,14 +186,14 @@ private List> readRecordsFromFile() throws IOException { if (conf.getBoolean("spark.shuffle.compress", true)) { in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); } - try (DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in)) { - Iterator> records = recordsStream.asKeyValueIterator(); - while (records.hasNext()) { - Tuple2 record = records.next(); - assertEquals(i, hashPartitioner.getPartition(record._1())); - recordsList.add(record); - } + DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in); + Iterator> records = recordsStream.asKeyValueIterator(); + while (records.hasNext()) { + Tuple2 record = records.next(); + assertEquals(i, hashPartitioner.getPartition(record._1())); + recordsList.add(record); } + recordsStream.close(); startOffset += partitionSize; } } diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index f979f9e8bb956..01b5fb7b46684 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -33,6 +33,8 @@ import java.util.Map; import java.util.concurrent.*; +import org.apache.spark.Accumulator; +import org.apache.spark.AccumulatorParam; import org.apache.spark.Partitioner; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; @@ -106,6 +108,11 @@ public void sparkContextUnion() { // Varargs JavaRDD sUnion = sc.union(s1, s2); assertEquals(4, sUnion.count()); + // List + List> list = new ArrayList<>(); + list.add(s2); + sUnion = sc.union(s1, list); + assertEquals(4, sUnion.count()); // Union of JavaDoubleRDDs List doubles = Arrays.asList(1.0, 2.0); @@ -179,7 +186,7 @@ public void randomSplit() { long s1 = splits[1].count(); long s2 = splits[2].count(); assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250); - assertTrue(s1 + " not within expected range", s1 > 250 && s1 < 350); + assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350); assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570); } @@ -949,7 +956,7 @@ public void wholeTextFiles() throws Exception { } @Test - public void textFilesCompressed() { + public void textFilesCompressed() throws IOException { String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -990,10 +997,10 @@ public void binaryFiles() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - try (FileChannel channel1 = fos1.getChannel()) { - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - } + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); JavaPairRDD readRDD = sc.binaryFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { @@ -1011,10 +1018,10 @@ public void binaryFilesCaching() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - try (FileChannel channel1 = fos1.getChannel()) { - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - } + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); readRDD.foreach(pair -> pair._2().toArray()); // force the file to read @@ -1035,12 +1042,13 @@ public void binaryRecords() throws Exception { FileOutputStream fos1 = new FileOutputStream(file1); - try (FileChannel channel1 = fos1.getChannel()) { - for (int i = 0; i < numOfCopies; i++) { - ByteBuffer bbuf = ByteBuffer.wrap(content1); - channel1.write(bbuf); - } + FileChannel channel1 = fos1.getChannel(); + + for (int i = 0; i < numOfCopies; i++) { + ByteBuffer bbuf = ByteBuffer.wrap(content1); + channel1.write(bbuf); } + channel1.close(); JavaRDD readRDD = sc.binaryRecords(tempDirName, content1.length); assertEquals(numOfCopies,readRDD.count()); @@ -1176,6 +1184,46 @@ public void zipPartitions() { assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } + @SuppressWarnings("deprecation") + @Test + public void accumulators() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + + Accumulator intAccum = sc.intAccumulator(10); + rdd.foreach(intAccum::add); + assertEquals((Integer) 25, intAccum.value()); + + Accumulator doubleAccum = sc.doubleAccumulator(10.0); + rdd.foreach(x -> doubleAccum.add((double) x)); + assertEquals((Double) 25.0, doubleAccum.value()); + + // Try a custom accumulator type + AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + @Override + public Float addInPlace(Float r, Float t) { + return r + t; + } + + @Override + public Float addAccumulator(Float r, Float t) { + return r + t; + } + + @Override + public Float zero(Float initialValue) { + return 0.0f; + } + }; + + Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); + rdd.foreach(x -> floatAccum.add((float) x)); + assertEquals((Float) 25.0f, floatAccum.value()); + + // Test the setValue method + floatAccum.setValue(5.0f); + assertEquals((Float) 5.0f, floatAccum.value()); + } + @Test public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); @@ -1363,13 +1411,13 @@ public void sampleByKeyExact() { JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map wrExactCounts = wrExact.countByKey(); assertEquals(2, wrExactCounts.size()); - assertEquals(2, (long) wrExactCounts.get(0)); - assertEquals(4, (long) wrExactCounts.get(1)); + assertTrue(wrExactCounts.get(0) == 2); + assertTrue(wrExactCounts.get(1) == 4); JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map worExactCounts = worExact.countByKey(); assertEquals(2, worExactCounts.size()); - assertEquals(2, (long) worExactCounts.get(0)); - assertEquals(4, (long) worExactCounts.get(1)); + assertTrue(worExactCounts.get(0) == 2); + assertTrue(worExactCounts.get(1) == 4); } private static class SomeCustomClass implements Serializable { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 435665d8a1ce2..5d0ffd92647bc 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,6 +28,7 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException +import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} @@ -44,6 +45,21 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } + implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = + new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } + } + test("accumulator serialization") { val ser = new JavaSerializer(new SparkConf).newInstance() val acc = createLongAccum("x") @@ -65,6 +81,122 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(acc3.isAtDriverSide) } + test ("basic accumulation") { + sc = new SparkContext("local", "test") + val acc: Accumulator[Int] = sc.accumulator(0) + + val d = sc.parallelize(1 to 20) + d.foreach{x => acc += x} + acc.value should be (210) + + val longAcc = sc.accumulator(0L) + val maxInt = Integer.MAX_VALUE.toLong + d.foreach{x => longAcc += maxInt + x} + longAcc.value should be (210L + maxInt * 20) + } + + test("value not assignable from tasks") { + sc = new SparkContext("local", "test") + val acc: Accumulator[Int] = sc.accumulator(0) + + val d = sc.parallelize(1 to 20) + intercept[SparkException] { + d.foreach(x => acc.value = x) + } + } + + test ("add value to collection accumulators") { + val maxI = 1000 + for (nThreads <- List(1, 10)) { // test single & multi-threaded + sc = new SparkContext("local[" + nThreads + "]", "test") + val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val d = sc.parallelize(1 to maxI) + d.foreach { + x => acc += x + } + val v = acc.value.asInstanceOf[mutable.Set[Int]] + for (i <- 1 to maxI) { + v should contain(i) + } + resetSparkContext() + } + } + + test("value not readable in tasks") { + val maxI = 1000 + for (nThreads <- List(1, 10)) { // test single & multi-threaded + sc = new SparkContext("local[" + nThreads + "]", "test") + val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val d = sc.parallelize(1 to maxI) + an [SparkException] should be thrownBy { + d.foreach { + x => acc.value += x + } + } + resetSparkContext() + } + } + + test ("collection accumulators") { + val maxI = 1000 + for (nThreads <- List(1, 10)) { + // test single & multi-threaded + sc = new SparkContext("local[" + nThreads + "]", "test") + val setAcc = sc.accumulableCollection(mutable.HashSet[Int]()) + val bufferAcc = sc.accumulableCollection(mutable.ArrayBuffer[Int]()) + val mapAcc = sc.accumulableCollection(mutable.HashMap[Int, String]()) + val d = sc.parallelize((1 to maxI) ++ (1 to maxI)) + d.foreach { + x => {setAcc += x; bufferAcc += x; mapAcc += (x -> x.toString)} + } + + // Note that this is typed correctly -- no casts necessary + setAcc.value.size should be (maxI) + bufferAcc.value.size should be (2 * maxI) + mapAcc.value.size should be (maxI) + for (i <- 1 to maxI) { + setAcc.value should contain(i) + bufferAcc.value should contain(i) + mapAcc.value should contain (i -> i.toString) + } + resetSparkContext() + } + } + + test ("localValue readable in tasks") { + val maxI = 1000 + for (nThreads <- List(1, 10)) { // test single & multi-threaded + sc = new SparkContext("local[" + nThreads + "]", "test") + val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val groupedInts = (1 to (maxI/20)).map {x => (20 * (x - 1) to 20 * x).toSet} + val d = sc.parallelize(groupedInts) + d.foreach { + x => acc.localValue ++= x + } + acc.value should be ((0 to maxI).toSet) + resetSparkContext() + } + } + + test ("garbage collection") { + // Create an accumulator and let it go out of scope to test that it's properly garbage collected + sc = new SparkContext("local", "test") + var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) + val accId = acc.id + val ref = WeakReference(acc) + + // Ensure the accumulator is present + assert(ref.get.isDefined) + + // Remove the explicit reference to it and allow weak reference to get garbage collected + acc = null + System.gc() + assert(ref.get.isEmpty) + + AccumulatorContext.remove(accId) + assert(!AccumulatorContext.get(accId).isDefined) + } + test("get accum") { // Don't register with SparkContext for cleanup var acc = createLongAccum("a") @@ -89,6 +221,20 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(AccumulatorContext.get(100000).isEmpty) } + test("string accumulator param") { + val acc = new Accumulator("", StringAccumulatorParam, Some("darkness")) + assert(acc.value === "") + acc.setValue("feeds") + assert(acc.value === "feeds") + acc.add("your") + assert(acc.value === "your") // value is overwritten, not concatenated + acc += "soul" + assert(acc.value === "soul") + acc ++= "with" + assert(acc.value === "with") + acc.merge("kindness") + assert(acc.value === "kindness") + } } private[spark] object AccumulatorSuite { @@ -110,7 +256,7 @@ private[spark] object AccumulatorSuite { } /** - * Make an `AccumulableInfo` out of an `AccumulatorV2` with the intent to use the + * Make an `AccumulableInfo` out of an [[Accumulable]] with the intent to use the * info as an accumulator update. */ def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 34efcdf4bc886..81b18c71f30ee 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -320,19 +320,6 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { } } - test("minimum split size per node and per rack should be less than or equal to maxSplitSize") { - sc = new SparkContext("local", "test") - val testOutput = Array[Byte](1, 2, 3, 4, 5) - val outFile = writeBinaryData(testOutput, 1) - sc.hadoopConfiguration.setLong( - "mapreduce.input.fileinputformat.split.minsize.per.node", 5123456) - sc.hadoopConfiguration.setLong( - "mapreduce.input.fileinputformat.split.minsize.per.rack", 5123456) - - val (_, data) = sc.binaryFiles(outFile.getAbsolutePath).collect().head - assert(data.toArray === testOutput) - } - test("fixed record length binary file as byte array") { sc = new SparkContext("local", "test") val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 79192f3f3c92c..e1666a35271d3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -33,9 +33,7 @@ import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFor import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually -import org.apache.spark.internal.config.EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.util.{ThreadUtils, Utils} @@ -674,55 +672,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) } } - - test("cancel zombie tasks in a result stage when the job finishes") { - val conf = new SparkConf() - .setMaster("local-cluster[1,2,1024]") - .setAppName("test-cluster") - .set("spark.ui.enabled", "false") - // Disable this so that if a task is running, we can make sure the executor will always send - // task metrics via heartbeat to driver. - .set(EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES.key, "false") - // Set a short heartbeat interval to send SparkListenerExecutorMetricsUpdate fast - .set("spark.executor.heartbeatInterval", "1s") - sc = new SparkContext(conf) - sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true") - @volatile var runningTaskIds: Seq[Long] = null - val listener = new SparkListener { - override def onExecutorMetricsUpdate( - executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { - if (executorMetricsUpdate.execId != SparkContext.DRIVER_IDENTIFIER) { - runningTaskIds = executorMetricsUpdate.accumUpdates.map(_._1) - } - } - } - sc.addSparkListener(listener) - sc.range(0, 2).groupBy((x: Long) => x % 2, 2).map { case (x, _) => - val context = org.apache.spark.TaskContext.get() - if (context.stageAttemptNumber == 0) { - if (context.partitionId == 0) { - // Make the first task in the first stage attempt fail. - throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0, 0, - new java.io.IOException("fake")) - } else { - // Make the second task in the first stage attempt sleep to generate a zombie task - Thread.sleep(60000) - } - } else { - // Make the second stage attempt successful. - } - x - }.collect() - sc.listenerBus.waitUntilEmpty(10000) - // As executors will send the metrics of running tasks via heartbeat, we can use this to check - // whether there is any running task. - eventually(timeout(10.seconds)) { - // Make sure runningTaskIds has been set - assert(runningTaskIds != null) - // Verify there is no running task. - assert(runningTaskIds.isEmpty) - } - } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index bb389cdb39dfd..7a36b5f02dc4c 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -200,12 +200,11 @@ private[spark] object Benchmark { def getProcessorName(): String = { val cpu = if (SystemUtils.IS_OS_MAC_OSX) { Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) - .stripLineEnd } else if (SystemUtils.IS_OS_LINUX) { Try { val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")).stripLineEnd Utils.executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) - .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") + .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") }.getOrElse("Unknown processor") } else { System.getenv("PROCESSOR_IDENTIFIER") diff --git a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala index 24e596e1ecdaf..89e927e5784d2 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala @@ -30,7 +30,7 @@ abstract class BenchmarkBase { * Implementations of this method are supposed to use the wrapper method `runBenchmark` * for each benchmark scenario. */ - def runBenchmarkSuite(mainArgs: Array[String]): Unit + def runBenchmarkSuite(): Unit final def runBenchmark(benchmarkName: String)(func: => Any): Unit = { val separator = "=" * 96 @@ -51,7 +51,7 @@ abstract class BenchmarkBase { output = Some(new FileOutputStream(file)) } - runBenchmarkSuite(args) + runBenchmarkSuite() output.foreach { o => if (o != null) { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 652c36ffa6e71..9eae3605d0738 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -74,25 +74,20 @@ trait TestPrematureExit { @volatile var exitedCleanly = false mainObject.exitFn = (_) => exitedCleanly = true - @volatile var exception: Exception = null val thread = new Thread { override def run() = try { mainObject.main(input) } catch { - // Capture the exception to check whether the exception contains searchString or not - case e: Exception => exception = e + // If exceptions occur after the "exit" has happened, fine to ignore them. + // These represent code paths not reachable during normal execution. + case e: Exception => if (!exitedCleanly) throw e } } thread.start() thread.join() - if (exitedCleanly) { - val joined = printStream.lineBuffer.mkString("\n") - assert(joined.contains(searchString)) - } else { - assert(exception != null) - if (!exception.getMessage.contains(searchString)) { - throw exception - } + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 6a761d43a5a68..444e8d6e11f88 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -27,8 +27,8 @@ import scala.concurrent.duration._ import scala.language.postfixOps import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path} -import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hdfs.DistributedFileSystem import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ import org.mockito.ArgumentMatcher @@ -856,39 +856,6 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!mockedProvider.isBlacklisted(accessDeniedPath)) } - test("check in-progress event logs absolute length") { - val path = new Path("testapp.inprogress") - val provider = new FsHistoryProvider(createTestConf()) - val mockedProvider = spy(provider) - val mockedFs = mock(classOf[FileSystem]) - val in = mock(classOf[FSDataInputStream]) - val dfsIn = mock(classOf[DFSInputStream]) - when(mockedProvider.fs).thenReturn(mockedFs) - when(mockedFs.open(path)).thenReturn(in) - when(in.getWrappedStream).thenReturn(dfsIn) - when(dfsIn.getFileLength).thenReturn(200) - // FileStatus.getLen is more than logInfo fileSize - var fileStatus = new FileStatus(200, false, 0, 0, 0, path) - var logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) - - fileStatus = new FileStatus() - fileStatus.setPath(path) - // DFSInputStream.getFileLength is more than logInfo fileSize - logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100) - assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) - // DFSInputStream.getFileLength is equal to logInfo fileSize - logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 200) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) - // in.getWrappedStream returns other than DFSInputStream - val bin = mock(classOf[BufferedInputStream]) - when(in.getWrappedStream).thenReturn(bin) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) - // fs.open throws exception - when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally")) - assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) - } - /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 37954826af90c..de321db845a66 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -40,6 +40,18 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { assert(conf.get("spark.testing") === "true") } + test("Directory Arguments Parsing --dir or -d") { + val argStrings = Array("--dir", "src/test/resources/spark-events1") + val hsa = new HistoryServerArguments(conf, argStrings) + assert(conf.get("spark.history.fs.logDirectory") === "src/test/resources/spark-events1") + } + + test("Directory Param can also be set directly") { + val argStrings = Array("src/test/resources/spark-events2") + val hsa = new HistoryServerArguments(conf, argStrings) + assert(conf.get("spark.history.fs.logDirectory") === "src/test/resources/spark-events2") + } + test("Properties File Arguments Parsing --properties-file") { val tmpDir = Utils.createTempDir() val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 4839c842cc785..54c168a8218f3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -83,26 +83,6 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { assert(submitResponse.success) } - test("create submission with multiple masters") { - val submittedDriverId = "your-driver-id" - val submitMessage = "my driver is submitted" - val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) - val conf = new SparkConf(loadDefaults = false) - val RANDOM_PORT = 9000 - val allMasters = s"$masterUrl,${Utils.localHostName()}:$RANDOM_PORT" - conf.set("spark.master", allMasters) - conf.set("spark.app.name", "dreamer") - val appArgs = Array("one", "two", "six") - // main method calls this - val response = new RestSubmissionClientApp().run("app-resource", "main-class", appArgs, conf) - val submitResponse = getSubmitResponse(response) - assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) - assert(submitResponse.serverSparkVersion === SPARK_VERSION) - assert(submitResponse.message === submitMessage) - assert(submitResponse.submissionId === submittedDriverId) - assert(submitResponse.success) - } - test("create submission from main method") { val submittedDriverId = "your-driver-id" val submitMessage = "my driver is submitted" diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index e0e630e3be63b..2849a10a2c81e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -21,36 +21,94 @@ import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials +import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.util.Utils -class HadoopDelegationTokenManagerSuite extends SparkFunSuite { - private val hadoopConf = new Configuration() +class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { + private var delegationTokenManager: HadoopDelegationTokenManager = null + private var sparkConf: SparkConf = null + private var hadoopConf: Configuration = null - test("default configuration") { - val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf) - assert(manager.isProviderLoaded("hadoopfs")) - assert(manager.isProviderLoaded("hbase")) - assert(manager.isProviderLoaded("hive")) + override def beforeAll(): Unit = { + super.beforeAll() + + sparkConf = new SparkConf() + hadoopConf = new Configuration() + } + + test("Correctly load default credential providers") { + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("bogus") should be (None) } test("disable hive credential provider") { - val sparkConf = new SparkConf(false).set("spark.security.credentials.hive.enabled", "false") - val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) - assert(manager.isProviderLoaded("hadoopfs")) - assert(manager.isProviderLoaded("hbase")) - assert(!manager.isProviderLoaded("hive")) + sparkConf.set("spark.security.credentials.hive.enabled", "false") + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) } test("using deprecated configurations") { - val sparkConf = new SparkConf(false) - .set("spark.yarn.security.tokens.hadoopfs.enabled", "false") - .set("spark.yarn.security.credentials.hive.enabled", "false") - val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf) - assert(!manager.isProviderLoaded("hadoopfs")) - assert(manager.isProviderLoaded("hbase")) - assert(!manager.isProviderLoaded("hive")) + sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") + sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + } + + test("verify no credentials are obtained") { + delegationTokenManager = new HadoopDelegationTokenManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess) + val creds = new Credentials() + + // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. + delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) + val tokens = creds.getAllTokens + tokens.size() should be (0) + } + + test("obtain tokens For HiveMetastore") { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.kerberos.principal", "bob") + // thrift picks up on port 0 and bails out, without trying to talk to endpoint + hadoopConf.set("hive.metastore.uris", "http://localhost:0") + + val hiveCredentialProvider = new HiveDelegationTokenProvider() + val credentials = new Credentials() + hiveCredentialProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials) + + credentials.getAllTokens.size() should be (0) + } + + test("Obtain tokens For HBase") { + val hadoopConf = new Configuration() + hadoopConf.set("hbase.security.authentication", "kerberos") + + val hbaseTokenProvider = new HBaseDelegationTokenProvider() + val creds = new Credentials() + hbaseTokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + + creds.getAllTokens.size should be (0) } test("SPARK-23209: obtain tokens when Hive classes are not available") { @@ -65,41 +123,43 @@ class HadoopDelegationTokenManagerSuite extends SparkFunSuite { throw new ClassNotFoundException(name) } - val prefixBlacklist = Seq("java", "scala", "com.sun.", "sun.") - if (prefixBlacklist.exists(name.startsWith(_))) { - return currentLoader.loadClass(name) + if (name.startsWith("java") || name.startsWith("scala")) { + currentLoader.loadClass(name) + } else { + val classFileName = name.replaceAll("\\.", "/") + ".class" + val in = currentLoader.getResourceAsStream(classFileName) + if (in != null) { + val bytes = IOUtils.toByteArray(in) + defineClass(name, bytes, 0, bytes.length) + } else { + throw new ClassNotFoundException(name) + } } - - val found = findLoadedClass(name) - if (found != null) { - return found - } - - val classFileName = name.replaceAll("\\.", "/") + ".class" - val in = currentLoader.getResourceAsStream(classFileName) - if (in != null) { - val bytes = IOUtils.toByteArray(in) - return defineClass(name, bytes, 0, bytes.length) - } - - throw new ClassNotFoundException(name) } } - Utils.withContextClassLoader(noHive) { + try { + Thread.currentThread().setContextClassLoader(noHive) val test = noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$")) test.getMethod("runTest").invoke(null) + } finally { + Thread.currentThread().setContextClassLoader(currentLoader) } } + + private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = { + Set(FileSystem.get(hadoopConf)) + } } /** Test code for SPARK-23209 to avoid using too much reflection above. */ -private object NoHiveTest { +private object NoHiveTest extends Matchers { def runTest(): Unit = { try { - val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration()) - require(!manager.isProviderLoaded("hive")) + val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), + _ => Set()) + manager.getServiceDelegationTokenProvider("hive") should be (None) } catch { case e: Throwable => // Throw a better exception in case the test fails, since there may be a lot of nesting. diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index c5a39669366ce..80c9c6f0422a8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -30,8 +30,6 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer} class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with Eventually { - private val executorUpTimeout = 60.seconds - test("serialized task larger than max RPC message size") { val conf = new SparkConf conf.set("spark.rpc.message.maxSize", "1") @@ -53,7 +51,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(executorUpTimeout)) { + eventually(timeout(10.seconds)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -66,7 +64,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(executorUpTimeout)) { + eventually(timeout(10.seconds)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -98,7 +96,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo try { sc.addSparkListener(listener) - eventually(timeout(executorUpTimeout)) { + eventually(timeout(10.seconds)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 5f4ffa151d19b..b41d2acab7152 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1901,50 +1901,27 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } /** - * The job will be failed on first task throwing an error. + * The job will be failed on first task throwing a DAGSchedulerSuiteDummyException. * Any subsequent task WILL throw a legitimate java.lang.UnsupportedOperationException. * If multiple tasks, there exists a race condition between the SparkDriverExecutionExceptions * and their differing causes as to which will represent result for job... */ test("misbehaved resultHandler should not crash DAGScheduler and SparkContext") { - failAfter(1.minute) { // If DAGScheduler crashes, the following test will hang forever - for (error <- Seq( - new DAGSchedulerSuiteDummyException, - new AssertionError, // E.g., assert(foo == bar) fails - new NotImplementedError // E.g., call a method with `???` implementation. - )) { - val e = intercept[SparkDriverExecutionException] { - // Number of parallelized partitions implies number of tasks of job - val rdd = sc.parallelize(1 to 10, 2) - sc.runJob[Int, Int]( - rdd, - (context: TaskContext, iter: Iterator[Int]) => iter.size, - // For a robust test assertion, limit number of job tasks to 1; that is, - // if multiple RDD partitions, use id of any one partition, say, first partition id=0 - Seq(0), - (part: Int, result: Int) => throw error) - } - assert(e.getCause eq error) - - // Make sure we can still run commands on our SparkContext - assert(sc.parallelize(1 to 10, 2).count() === 10) - } + val e = intercept[SparkDriverExecutionException] { + // Number of parallelized partitions implies number of tasks of job + val rdd = sc.parallelize(1 to 10, 2) + sc.runJob[Int, Int]( + rdd, + (context: TaskContext, iter: Iterator[Int]) => iter.size, + // For a robust test assertion, limit number of job tasks to 1; that is, + // if multiple RDD partitions, use id of any one partition, say, first partition id=0 + Seq(0), + (part: Int, result: Int) => throw new DAGSchedulerSuiteDummyException) } - } + assert(e.getCause.isInstanceOf[DAGSchedulerSuiteDummyException]) - test(s"invalid ${SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL} should not crash DAGScheduler") { - sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "invalid") - try { - intercept[SparkException] { - sc.parallelize(1 to 1, 1).foreach { _ => - throw new DAGSchedulerSuiteDummyException - } - } - // Verify the above job didn't crash DAGScheduler by running a simple job - assert(sc.parallelize(1 to 10, 2).count() === 10) - } finally { - sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) - } + // Make sure we can still run commands on our SparkContext + assert(sc.parallelize(1 to 10, 2).count() === 10) } test("getPartitions exceptions should not crash DAGScheduler and SparkContext (SPARK-8606)") { diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala index 0d3611c80b8d0..78f618f8a2163 100644 --- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -16,16 +16,13 @@ */ package org.apache.spark.security -import java.io._ -import java.nio.ByteBuffer -import java.nio.channels.{Channels, ReadableByteChannel} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.nio.channels.Channels import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.Files import java.util.{Arrays, Random, UUID} import com.google.common.io.ByteStreams -import org.mockito.Matchers.any -import org.mockito.Mockito._ import org.apache.spark._ import org.apache.spark.internal.config._ @@ -167,36 +164,6 @@ class CryptoStreamUtilsSuite extends SparkFunSuite { } } - test("error handling wrapper") { - val wrapped = mock(classOf[ReadableByteChannel]) - val decrypted = mock(classOf[ReadableByteChannel]) - val errorHandler = new CryptoStreamUtils.ErrorHandlingReadableChannel(decrypted, wrapped) - - when(decrypted.read(any(classOf[ByteBuffer]))) - .thenThrow(new IOException()) - .thenThrow(new InternalError()) - .thenReturn(1) - - val out = ByteBuffer.allocate(1) - intercept[IOException] { - errorHandler.read(out) - } - intercept[InternalError] { - errorHandler.read(out) - } - - val e = intercept[IOException] { - errorHandler.read(out) - } - assert(e.getMessage().contains("is closed")) - errorHandler.close() - - verify(decrypted, times(2)).read(any(classOf[ByteBuffer])) - verify(wrapped, never()).read(any(classOf[ByteBuffer])) - verify(decrypted, never()).close() - verify(wrapped, times(1)).close() - } - private def createConf(extra: (String, String)*): SparkConf = { val conf = new SparkConf() extra.foreach { case (k, v) => conf.set(k, v) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index d7730f23da108..f4fc0080f3108 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -20,48 +20,58 @@ package org.apache.spark.serializer import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.SparkConf -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.serializer.KryoTest._ -/** - * Benchmark for Kryo Unsafe vs safe Serialization. - * To run this benchmark: - * {{{ - * 1. without sbt: - * bin/spark-submit --class --jars - * 2. build/sbt "core/test:runMain " - * 3. generate result: - * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " - * Results will be written to "benchmarks/KryoBenchmark-results.txt". - * }}} - */ -object KryoBenchmark extends BenchmarkBase { - - val N = 1000000 - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - val name = "Benchmark Kryo Unsafe vs safe Serialization" - runBenchmark(name) { - val benchmark = new Benchmark(name, N, 10, output = output) - Seq(true, false).foreach(useUnsafe => run(useUnsafe, benchmark)) - benchmark.run() - } +class KryoBenchmark extends SparkFunSuite { + val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) + + ignore(s"Benchmark Kryo Unsafe vs safe Serialization") { + Seq (true, false).foreach (runBenchmark) + benchmark.run() + + // scalastyle:off + /* + Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + basicTypes: Int with unsafe:true 151 / 170 104.2 9.6 1.0X + basicTypes: Long with unsafe:true 175 / 191 89.8 11.1 0.9X + basicTypes: Float with unsafe:true 177 / 184 88.8 11.3 0.9X + basicTypes: Double with unsafe:true 193 / 216 81.4 12.3 0.8X + Array: Int with unsafe:true 513 / 587 30.7 32.6 0.3X + Array: Long with unsafe:true 1211 / 1358 13.0 77.0 0.1X + Array: Float with unsafe:true 890 / 964 17.7 56.6 0.2X + Array: Double with unsafe:true 1335 / 1428 11.8 84.9 0.1X + Map of string->Double with unsafe:true 931 / 988 16.9 59.2 0.2X + basicTypes: Int with unsafe:false 197 / 217 79.9 12.5 0.8X + basicTypes: Long with unsafe:false 219 / 240 71.8 13.9 0.7X + basicTypes: Float with unsafe:false 208 / 217 75.7 13.2 0.7X + basicTypes: Double with unsafe:false 208 / 225 75.6 13.2 0.7X + Array: Int with unsafe:false 2559 / 2681 6.1 162.7 0.1X + Array: Long with unsafe:false 3425 / 3516 4.6 217.8 0.0X + Array: Float with unsafe:false 2025 / 2134 7.8 128.7 0.1X + Array: Double with unsafe:false 2241 / 2358 7.0 142.5 0.1X + Map of string->Double with unsafe:false 1044 / 1085 15.1 66.4 0.1X + */ + // scalastyle:on } - private def run(useUnsafe: Boolean, benchmark: Benchmark): Unit = { + private def runBenchmark(useUnsafe: Boolean): Unit = { def check[T: ClassTag](t: T, ser: SerializerInstance): Int = { - if (ser.deserialize[T](ser.serialize(t)) == t) 1 else 0 + if (ser.deserialize[T](ser.serialize(t)) === t) 1 else 0 } // Benchmark Primitives + val basicTypeCount = 1000000 def basicTypes[T: ClassTag](name: String, gen: () => T): Unit = { lazy val ser = createSerializer(useUnsafe) - val arrayOfBasicType: Array[T] = Array.fill(N)(gen()) + val arrayOfBasicType: Array[T] = Array.fill(basicTypeCount)(gen()) benchmark.addCase(s"basicTypes: $name with unsafe:$useUnsafe") { _ => var sum = 0L var i = 0 - while (i < N) { + while (i < basicTypeCount) { sum += check(arrayOfBasicType(i), ser) i += 1 } @@ -74,12 +84,11 @@ object KryoBenchmark extends BenchmarkBase { basicTypes("Double", () => Random.nextDouble()) // Benchmark Array of Primitives - val arrayCount = 4000 - val arrayLength = N / arrayCount + val arrayCount = 10000 def basicTypeArray[T: ClassTag](name: String, gen: () => T): Unit = { lazy val ser = createSerializer(useUnsafe) val arrayOfArrays: Array[Array[T]] = - Array.fill(arrayCount)(Array.fill[T](arrayLength + Random.nextInt(arrayLength / 4))(gen())) + Array.fill(arrayCount)(Array.fill[T](Random.nextInt(arrayCount))(gen())) benchmark.addCase(s"Array: $name with unsafe:$useUnsafe") { _ => var sum = 0L @@ -98,13 +107,11 @@ object KryoBenchmark extends BenchmarkBase { basicTypeArray("Double", () => Random.nextDouble()) // Benchmark Maps - val mapsCount = 200 - val mapKeyLength = 20 - val mapLength = N / mapsCount / mapKeyLength + val mapsCount = 1000 lazy val ser = createSerializer(useUnsafe) val arrayOfMaps: Array[Map[String, Double]] = Array.fill(mapsCount) { - Array.fill(mapLength + Random.nextInt(mapLength / 4)) { - (Random.nextString(mapKeyLength), Random.nextDouble()) + Array.fill(Random.nextInt(mapsCount)) { + (Random.nextString(mapsCount / 10), Random.nextDouble()) }.toMap } diff --git a/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala b/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala index cda98ae25a57a..74eeca282882a 100644 --- a/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/PagedTableSuite.scala @@ -73,7 +73,7 @@ class PagedTableSuite extends SparkFunSuite { override def goButtonFormPath: String = "" } - assert((pagedTable.pageNavigation(1, 10, 1).head \\ "li").map(_.text.trim) === Seq("1")) + assert(pagedTable.pageNavigation(1, 10, 1) === Nil) assert( (pagedTable.pageNavigation(1, 10, 2).head \\ "li").map(_.text.trim) === Seq("1", "2", ">")) assert( diff --git a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala index 621399af731f7..94c79388e3639 100644 --- a/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala +++ b/core/src/test/scala/org/apache/spark/util/AccumulatorV2Suite.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import org.apache.spark._ +import org.apache.spark.serializer.JavaSerializer class AccumulatorV2Suite extends SparkFunSuite { @@ -127,6 +128,58 @@ class AccumulatorV2Suite extends SparkFunSuite { assert(acc3.value.isEmpty) } + test("LegacyAccumulatorWrapper") { + val acc = new LegacyAccumulatorWrapper("default", AccumulatorParam.StringAccumulatorParam) + assert(acc.value === "default") + assert(!acc.isZero) + + acc.add("foo") + assert(acc.value === "foo") + assert(!acc.isZero) + + acc.add(new java.lang.String("bar")) + + val acc2 = acc.copyAndReset() + assert(acc2.value === "") + assert(acc2.isZero) + + assert(acc.value === "bar") + assert(!acc.isZero) + + acc2.add("baz") + assert(acc2.value === "baz") + assert(!acc2.isZero) + + // Test merging + acc.merge(acc2) + assert(acc.value === "baz") + assert(!acc.isZero) + + val acc3 = acc.copy() + assert(acc3.value === "baz") + assert(!acc3.isZero) + + acc3.reset() + assert(acc3.isZero) + assert(acc3.value === "") + } + + test("LegacyAccumulatorWrapper with AccumulatorParam that has no equals/hashCode") { + val param = new AccumulatorParam[MyData] { + override def zero(initialValue: MyData): MyData = new MyData(0) + override def addInPlace(r1: MyData, r2: MyData): MyData = new MyData(r1.i + r2.i) + } + + val acc = new LegacyAccumulatorWrapper(new MyData(0), param) + acc.metadata = AccumulatorMetadata( + AccumulatorContext.newId(), + Some("test"), + countFailedValues = false) + AccumulatorContext.register(acc) + + val ser = new JavaSerializer(new SparkConf).newInstance() + ser.serialize(acc) + } } class MyData(val i: Int) extends Serializable diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala index 24762ea2f4e6b..2fb09ead4b2d8 100644 --- a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala +++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala @@ -74,6 +74,26 @@ class ImplicitSuite { rdd.stats() } + def testDoubleAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123.4) + } + + def testIntAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123) + } + + def testLongAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123L) + } + + def testFloatAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123F) + } + def testIntWritableConverter(): Unit = { val sc = mockSparkContext sc.sequenceFile[Int, Int]("/a/test/path") diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index c91882851847b..8a04b621f8ce4 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -95,8 +95,7 @@ $env:MAVEN_OPTS = "-Xmx2g -XX:ReservedCodeCacheSize=512m" Pop-Location # ========================== Hadoop bin package -# This must match the version at https://github.com/steveloughran/winutils/tree/master/hadoop-2.7.1 -$hadoopVer = "2.7.1" +$hadoopVer = "2.6.4" $hadoopPath = "$tools\hadoop" if (!(Test-Path $hadoopPath)) { New-Item -ItemType Directory -Force -Path $hadoopPath | Out-Null diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 2fdb5c8dd38a1..cce5f8b6975ca 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -114,10 +114,8 @@ PUBLISH_SCALA_2_10=0 SCALA_2_10_PROFILES="-Pscala-2.10" SCALA_2_11_PROFILES= if [[ $SPARK_VERSION > "2.3" ]]; then - BASE_PROFILES="$BASE_PROFILES -Pkubernetes" - if [[ $SPARK_VERSION < "3.0." ]]; then - SCALA_2_11_PROFILES="-Pkafka-0-8" - fi + BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume" + SCALA_2_11_PROFILES="-Pkafka-0-8" else PUBLISH_SCALA_2_10=1 fi @@ -193,19 +191,9 @@ if [[ "$1" == "package" ]]; then make_binary_release() { NAME=$1 FLAGS="$MVN_EXTRA_OPTS -B $BASE_RELEASE_PROFILES $2" - # BUILD_PACKAGE can be "withpip", "withr", or both as "withpip,withr" BUILD_PACKAGE=$3 SCALA_VERSION=$4 - PIP_FLAG="" - if [[ $BUILD_PACKAGE == *"withpip"* ]]; then - PIP_FLAG="--pip" - fi - R_FLAG="" - if [[ $BUILD_PACKAGE == *"withr"* ]]; then - R_FLAG="--r" - fi - # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. ZINC_PORT=$((ZINC_PORT + 1)) @@ -229,13 +217,18 @@ if [[ "$1" == "package" ]]; then # Get maven home set by MVN MVN_HOME=`$MVN -version 2>&1 | grep 'Maven home' | awk '{print $NF}'` - echo "Creating distribution" - ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz \ - $PIP_FLAG $R_FLAG $FLAGS \ - -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log - cd .. - if [[ -n $R_FLAG ]]; then + if [ -z "$BUILD_PACKAGE" ]; then + echo "Creating distribution without PIP/R package" + ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz $FLAGS \ + -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log + cd .. + elif [[ "$BUILD_PACKAGE" == "withr" ]]; then + echo "Creating distribution with R package" + ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --r $FLAGS \ + -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log + cd .. + echo "Copying and signing R source package" R_DIST_NAME=SparkR_$SPARK_VERSION.tar.gz cp spark-$SPARK_VERSION-bin-$NAME/R/$R_DIST_NAME . @@ -246,9 +239,12 @@ if [[ "$1" == "package" ]]; then echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \ SHA512 $R_DIST_NAME > \ $R_DIST_NAME.sha512 - fi + else + echo "Creating distribution with PIP package" + ./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --pip $FLAGS \ + -DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log + cd .. - if [[ -n $PIP_FLAG ]]; then echo "Copying and signing python distribution" PYTHON_DIST_NAME=pyspark-$PYSPARK_VERSION.tar.gz cp spark-$SPARK_VERSION-bin-$NAME/python/dist/$PYTHON_DIST_NAME . @@ -281,10 +277,8 @@ if [[ "$1" == "package" ]]; then declare -A BINARY_PKGS_ARGS BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" if ! is_dry_run; then + BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" BINARY_PKGS_ARGS["without-hadoop"]="-Phadoop-provided" - if [[ $SPARK_VERSION < "3.0." ]]; then - BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" - fi if [[ $SPARK_VERSION < "2.2." ]]; then BINARY_PKGS_ARGS["hadoop2.4"]="-Phadoop-2.4 $HIVE_PROFILES" BINARY_PKGS_ARGS["hadoop2.3"]="-Phadoop-2.3 $HIVE_PROFILES" @@ -292,7 +286,10 @@ if [[ "$1" == "package" ]]; then fi declare -A BINARY_PKGS_EXTRA - BINARY_PKGS_EXTRA["hadoop2.7"]="withpip,withr" + BINARY_PKGS_EXTRA["hadoop2.7"]="withpip" + if ! is_dry_run; then + BINARY_PKGS_EXTRA["hadoop2.6"]="withr" + fi echo "Packages to build: ${!BINARY_PKGS_ARGS[@]}" for key in ${!BINARY_PKGS_ARGS[@]}; do @@ -326,7 +323,7 @@ if [[ "$1" == "package" ]]; then svn add "svn-spark/${DEST_DIR_NAME}-bin" cd svn-spark - svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION" --no-auth-cache + svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION" cd .. rm -rf svn-spark fi @@ -354,7 +351,7 @@ if [[ "$1" == "docs" ]]; then svn add "svn-spark/${DEST_DIR_NAME}-docs" cd svn-spark - svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION docs" --no-auth-cache + svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION docs" cd .. rm -rf svn-spark fi @@ -389,6 +386,9 @@ if [[ "$1" == "publish-snapshot" ]]; then #$MVN -DzincPort=$ZINC_PORT --settings $tmp_settings \ # -DskipTests $SCALA_2_12_PROFILES $PUBLISH_PROFILES clean deploy + # Clean-up Zinc nailgun process + $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill + rm $tmp_settings cd .. exit 0 @@ -433,6 +433,9 @@ if [[ "$1" == "publish-release" ]]; then -DskipTests $PUBLISH_PROFILES $SCALA_2_12_PROFILES clean install fi + # Clean-up Zinc nailgun process + $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill + ./dev/change-scala-version.sh 2.11 pushd $tmp_repo/org/apache/spark diff --git a/dev/deps/spark-deps-hadoop-palantir b/dev/deps/spark-deps-hadoop-palantir index 521baac144304..fc0bfb412de1c 100644 --- a/dev/deps/spark-deps-hadoop-palantir +++ b/dev/deps/spark-deps-hadoop-palantir @@ -18,8 +18,6 @@ avro-1.8.2.jar avro-ipc-1.8.2.jar avro-mapred-1.8.2-hadoop2.jar aws-java-sdk-bundle-1.11.201.jar -azure-keyvault-core-0.8.0.jar -azure-storage-5.4.0.jar breeze-macros_2.11-0.13.2.jar breeze_2.11-0.13.2.jar chill-java-0.9.3.jar @@ -64,7 +62,6 @@ guice-servlet-3.0.jar hadoop-annotations-2.9.1-palantir.11.jar hadoop-auth-2.9.1-palantir.11.jar hadoop-aws-2.9.1-palantir.11.jar -hadoop-azure-2.9.1-palantir.11.jar hadoop-client-2.9.1-palantir.11.jar hadoop-common-2.9.1-palantir.11.jar hadoop-hdfs-client-2.9.1-palantir.11.jar @@ -141,12 +138,12 @@ jtransforms-2.4.0.jar jul-to-slf4j-1.7.25.jar kafka-clients-0.11.0.0.jar kryo-shaded-4.0.2.jar -kubernetes-client-4.1.0.jar -kubernetes-model-4.1.0.jar +kubernetes-client-3.0.0.jar +kubernetes-model-2.0.0.jar leveldbjni-all-1.8.jar log4j-1.2.17.jar logging-interceptor-3.11.0.jar -lz4-java-1.5.0.jar +lz4-java-1.4.1.jar machinist_2.11-0.6.1.jar macro-compat_2.11-1.1.1.jar metrics-core-3.2.6.jar @@ -157,16 +154,16 @@ metrics-jvm-3.2.6.jar minlog-1.3.0.jar mssql-jdbc-6.2.1.jre7.jar netty-3.10.6.Final.jar -netty-all-4.1.30.Final.jar +netty-all-4.1.29.Final.jar nimbus-jose-jwt-4.41.1.jar objenesis-2.6.jar okhttp-2.7.5.jar okhttp-3.11.0.jar okio-1.15.0.jar opencsv-2.3.jar -orc-core-1.5.3-nohive.jar -orc-mapreduce-1.5.3-nohive.jar -orc-shims-1.5.3.jar +orc-core-1.5.2-nohive.jar +orc-mapreduce-1.5.2-nohive.jar +orc-shims-1.5.2.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar @@ -177,7 +174,7 @@ parquet-format-2.4.0.jar parquet-hadoop-1.10.1-palantir.3.jar parquet-jackson-1.10.1-palantir.3.jar protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar +py4j-0.10.7.jar pyrolite-4.13.jar safe-logging-1.5.1.jar scala-compiler-2.11.12.jar diff --git a/dev/lint-python b/dev/lint-python index 2e353e142c143..e26bd4bd4517c 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -21,14 +21,9 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" # Exclude auto-generated configuration file. PATHS_TO_CHECK="$( cd "$SPARK_ROOT_DIR" && find . -name "*.py" )" -DOC_PATHS_TO_CHECK="$( cd "$SPARK_ROOT_DIR" && find . -name "*.py" | grep -vF 'functions.py' )" PYCODESTYLE_REPORT_PATH="$SPARK_ROOT_DIR/dev/pycodestyle-report.txt" -PYDOCSTYLE_REPORT_PATH="$SPARK_ROOT_DIR/dev/pydocstyle-report.txt" PYLINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/pylint-report.txt" PYLINT_INSTALL_INFO="$SPARK_ROOT_DIR/dev/pylint-info.txt" -PYDOCSTYLEBUILD="pydocstyle" -EXPECTED_PYDOCSTYLEVERSION="3.0.0" -PYDOCSTYLEVERSION=$(python -c 'import pkg_resources; print(pkg_resources.get_distribution("pydocstyle").version)' 2> /dev/null) SPHINXBUILD=${SPHINXBUILD:=sphinx-build} SPHINX_REPORT_PATH="$SPARK_ROOT_DIR/dev/sphinx-report.txt" @@ -104,29 +99,6 @@ else echo "flake8 checks passed." fi -# Check python document style, skip check if pydocstyle is not installed. -if hash "$PYDOCSTYLEBUILD" 2> /dev/null; then - if [[ "$PYDOCSTYLEVERSION" == "$EXPECTED_PYDOCSTYLEVERSION" ]]; then - pydocstyle --config=dev/tox.ini $DOC_PATHS_TO_CHECK >> "$PYDOCSTYLE_REPORT_PATH" - pydocstyle_status="${PIPESTATUS[0]}" - - if [ "$compile_status" -eq 0 -a "$pydocstyle_status" -eq 0 ]; then - echo "pydocstyle checks passed." - rm "$PYDOCSTYLE_REPORT_PATH" - else - echo "pydocstyle checks failed." - cat "$PYDOCSTYLE_REPORT_PATH" - rm "$PYDOCSTYLE_REPORT_PATH" - exit 1 - fi - - else - echo "The pydocstyle version needs to be latest 3.0.0. Skipping pydoc checks for now" - fi -else - echo >&2 "The pydocstyle command was not found. Skipping pydoc checks for now" -fi - # Check that the documentation builds acceptably, skip check if sphinx is not installed. if hash "$SPHINXBUILD" 2> /dev/null; then cd python/docs diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index e4405d89a1c17..8aed8ab5663e7 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -223,13 +223,9 @@ mkdir -p "$DISTDIR/examples/src/main" cp -r "$SPARK_HOME/examples/src/main" "$DISTDIR/examples/src/" # Copy license and ASF files -if [ -e "$SPARK_HOME/LICENSE-binary" ]; then - cp "$SPARK_HOME/LICENSE-binary" "$DISTDIR/LICENSE" - cp -r "$SPARK_HOME/licenses-binary" "$DISTDIR/licenses" - cp "$SPARK_HOME/NOTICE-binary" "$DISTDIR/NOTICE" -else - echo "Skipping copying LICENSE files" -fi +cp "$SPARK_HOME/LICENSE-binary" "$DISTDIR/LICENSE" +cp -r "$SPARK_HOME/licenses-binary" "$DISTDIR/licenses" +cp "$SPARK_HOME/NOTICE-binary" "$DISTDIR/NOTICE" if [ -e "$SPARK_HOME/CHANGES.txt" ]; then cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR" diff --git a/dev/mima b/dev/mima index c93e20557a253..e6402fb3771f6 100755 --- a/dev/mima +++ b/dev/mima @@ -24,7 +24,7 @@ set -e FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" -SPARK_PROFILES="-Phadoop-palantir -Pmesos -Phadoop-cloud -Pkubernetes -Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" +SPARK_PROFILES="-Phadoop-palantir -Pmesos -Phadoop-cloud -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive" TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export tools/fullClasspath" | tail -n1)" OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 77d751f49b9f1..eca88f2391bf8 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -39,8 +39,7 @@ def print_err(msg): def post_message_to_github(msg, ghprb_pull_id): print("Attempting to post to Github...") - api_url = os.getenv("GITHUB_API_BASE", "https://api.github.com/repos/apache/spark") - url = api_url + "/issues/" + ghprb_pull_id + "/comments" + url = "https://api.github.com/repos/apache/spark/issues/" + ghprb_pull_id + "/comments" github_oauth_key = os.environ["GITHUB_OAUTH_KEY"] posted_message = json.dumps({"body": msg}) @@ -177,8 +176,7 @@ def main(): build_display_name = os.environ["BUILD_DISPLAY_NAME"] build_url = os.environ["BUILD_URL"] - project_url = os.getenv("SPARK_PROJECT_URL", "https://github.com/apache/spark") - commit_url = project_url + "/commit/" + ghprb_actual_commit + commit_url = "https://github.com/apache/spark/commit/" + ghprb_actual_commit # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( short_commit_hash = ghprb_actual_commit[0:7] diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle index c63887eff0ac2..1fb913d1f4805 100755 --- a/dev/sbt-checkstyle +++ b/dev/sbt-checkstyle @@ -24,8 +24,10 @@ ERRORS=$(echo -e "q\n" \ -Phadoop-palantir \ -Pkinesis-asl \ -Pmesos \ + -Pkafka-0-8 \ -Pkubernetes \ -Pyarn \ + -Pflume \ -Phive \ -Phive-thriftserver \ checkstyle test:checkstyle \ diff --git a/dev/scalastyle b/dev/scalastyle index 62a4199858004..e071c9a73968d 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -24,12 +24,14 @@ ERRORS=$(echo -e "q\n" \ -Phadoop-cloud \ -Pkinesis-asl \ -Pmesos \ + -Pkafka-0-8 \ -Pkubernetes \ -Pyarn \ + -Pflume \ -Phive \ -Phive-thriftserver \ + -Pkubernetes \ -Pkubernetes-integration-tests \ - -Pspark-ganglia-lgpl \ scalastyle test:scalastyle \ | awk '{if($1~/error/)print}' \ ) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 5db8efda90774..a3dfb19941d4f 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -252,6 +252,24 @@ def __hash__(self): ) +streaming_kafka = Module( + name="streaming-kafka-0-8", + dependencies=[streaming], + source_file_regexes=[ + "external/kafka-0-8", + "external/kafka-0-8-assembly", + ], + build_profile_flags=[ + "-Pkafka-0-8", + ], + environ={ + "ENABLE_KAFKA_0_8_TESTS": "1" + }, + sbt_test_goals=[ + "streaming-kafka-0-8/test", + ] +) + streaming_kafka_0_10 = Module( name="streaming-kafka-0-10", dependencies=[streaming], @@ -265,6 +283,56 @@ def __hash__(self): ] ) +streaming_flume_sink = Module( + name="streaming-flume-sink", + dependencies=[streaming], + source_file_regexes=[ + "external/flume-sink", + ], + build_profile_flags=[ + "-Pflume", + ], + environ={ + "ENABLE_FLUME_TESTS": "1" + }, + sbt_test_goals=[ + "streaming-flume-sink/test", + ] +) + + +streaming_flume = Module( + name="streaming-flume", + dependencies=[streaming], + source_file_regexes=[ + "external/flume", + ], + build_profile_flags=[ + "-Pflume", + ], + environ={ + "ENABLE_FLUME_TESTS": "1" + }, + sbt_test_goals=[ + "streaming-flume/test", + ] +) + + +streaming_flume_assembly = Module( + name="streaming-flume-assembly", + dependencies=[streaming_flume, streaming_flume_sink], + source_file_regexes=[ + "external/flume-assembly", + ], + build_profile_flags=[ + "-Pflume", + ], + environ={ + "ENABLE_FLUME_TESTS": "1" + } +) + mllib_local = Module( name="mllib-local", @@ -356,11 +424,17 @@ def __hash__(self): dependencies=[ pyspark_core, streaming, + streaming_kafka, + streaming_flume_assembly, streaming_kinesis_asl ], source_file_regexes=[ "python/pyspark/streaming" ], + environ={ + "ENABLE_FLUME_TESTS": "1", + "ENABLE_KAFKA_0_8_TESTS": "1" + }, python_test_goals=[ "pyspark.streaming.util", "pyspark.streaming.tests", @@ -483,16 +557,6 @@ def __hash__(self): sbt_test_goals=["kubernetes/test"] ) - -spark_ganglia_lgpl = Module( - name="spark-ganglia-lgpl", - dependencies=[], - build_profile_flags=["-Pspark-ganglia-lgpl"], - source_file_regexes=[ - "external/spark-ganglia-lgpl", - ] -) - cloud = Module( name="hadoop-cloud", dependencies=[], diff --git a/dev/test_functions.py b/dev/test_functions.py index 3c9c7ccdefd25..35d9d20595a7d 100755 --- a/dev/test_functions.py +++ b/dev/test_functions.py @@ -249,6 +249,15 @@ def get_zinc_port(): return random.randrange(3030, 4030) +def kill_zinc_on_port(zinc_port): + """ + Kill the Zinc process running on the given port, if one exists. + """ + cmd = "%s -P |grep %s | grep LISTEN | awk '{ print $2; }' | xargs kill" + lsof_exe = which("lsof") + subprocess.check_call(cmd % (lsof_exe if lsof_exe else "/usr/sbin/lsof", zinc_port), shell=True) + + def exec_maven(mvn_args=()): """Will call Maven in the current directory with the list of mvn_args passed in and returns the subprocess for any further processing""" @@ -258,6 +267,7 @@ def exec_maven(mvn_args=()): zinc_flag = "-DzincPort=%s" % zinc_port flags = [os.path.join(SPARK_HOME, "build", "mvn"), "--force", zinc_flag] run_cmd(flags + mvn_args) + kill_zinc_on_port(zinc_port) def exec_sbt(sbt_args=()): @@ -301,6 +311,7 @@ def get_hadoop_profiles(hadoop_version): """ sbt_maven_hadoop_profiles = { + "hadoop2.6": ["-Phadoop-2.6"], "hadoop2.7": ["-Phadoop-2.7"], "hadooppalantir": ["-Phadoop-palantir"], } @@ -329,6 +340,8 @@ def build_spark_sbt(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags sbt_goals = ["test:package", # Build test jars as some tests depend on them + "streaming-kafka-0-8-assembly/assembly", + "streaming-flume-assembly/assembly", "streaming-kinesis-asl-assembly/assembly"] profiles_and_goals = build_profiles + sbt_goals @@ -363,7 +376,15 @@ def build_spark_assembly_sbt(hadoop_version, checkstyle=False): if checkstyle: run_java_style_checks() - build_spark_unidoc_sbt(hadoop_version) + # Note that we skip Unidoc build only if Hadoop 2.6 is explicitly set in this SBT build. + # Due to a different dependency resolution in SBT & Unidoc by an unknown reason, the + # documentation build fails on a specific machine & environment in Jenkins but it was unable + # to reproduce. Please see SPARK-20343. This is a band-aid fix that should be removed in + # the future. + is_hadoop_version_2_6 = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE") == "hadoop2.6" + if not is_hadoop_version_2_6: + # Make sure that Java and Scala API documentation can be generated + build_spark_unidoc_sbt(hadoop_version) def build_apache_spark(build_tool, hadoop_version): @@ -528,14 +549,14 @@ def main(): # if we're on the Amplab Jenkins build servers setup variables # to reflect the environment settings build_tool = os.environ.get("AMPLAB_JENKINS_BUILD_TOOL", "sbt") - hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.7") + hadoop_version = os.environ.get("AMPLAB_JENKINS_BUILD_PROFILE", "hadoop2.6") test_env = "amplab_jenkins" # add path for Python3 in Jenkins if we're calling from a Jenkins machine os.environ["PATH"] = "/home/anaconda/envs/py3k/bin:" + os.environ.get("PATH") else: # else we're running locally and can use local settings build_tool = "sbt" - hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.7") + hadoop_version = os.environ.get("HADOOP_PROFILE", "hadoop2.6") test_env = "local" print("[info] Using build tool", build_tool, "with Hadoop profile", hadoop_version, diff --git a/dev/tox.ini b/dev/tox.ini index 11b1b040035b0..6ec223b743b4e 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -17,5 +17,3 @@ ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/* -[pydocstyle] -ignore=D100,D101,D102,D103,D104,D105,D106,D107,D200,D201,D202,D203,D204,D205,D206,D207,D208,D209,D210,D211,D212,D213,D214,D215,D300,D301,D302,D400,D401,D402,D403,D404,D405,D406,D407,D408,D409,D410,D411,D412,D413,D414 diff --git a/docs/_data/menu-ml.yaml b/docs/_data/menu-ml.yaml index 8e366f7f029aa..b5a6641e2e7e2 100644 --- a/docs/_data/menu-ml.yaml +++ b/docs/_data/menu-ml.yaml @@ -1,7 +1,5 @@ - text: Basic statistics url: ml-statistics.html -- text: Data sources - url: ml-datasource - text: Pipelines url: ml-pipeline.html - text: Extracting, transforming and selecting features diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml deleted file mode 100644 index cd065ea01dda4..0000000000000 --- a/docs/_data/menu-sql.yaml +++ /dev/null @@ -1,81 +0,0 @@ -- text: Getting Started - url: sql-getting-started.html - subitems: - - text: "Starting Point: SparkSession" - url: sql-getting-started.html#starting-point-sparksession - - text: Creating DataFrames - url: sql-getting-started.html#creating-dataframes - - text: Untyped Dataset Operations (DataFrame operations) - url: sql-getting-started.html#untyped-dataset-operations-aka-dataframe-operations - - text: Running SQL Queries Programmatically - url: sql-getting-started.html#running-sql-queries-programmatically - - text: Global Temporary View - url: sql-getting-started.html#global-temporary-view - - text: Creating Datasets - url: sql-getting-started.html#creating-datasets - - text: Interoperating with RDDs - url: sql-getting-started.html#interoperating-with-rdds - - text: Aggregations - url: sql-getting-started.html#aggregations -- text: Data Sources - url: sql-data-sources.html - subitems: - - text: "Generic Load/Save Functions" - url: sql-data-sources-load-save-functions.html - - text: Parquet Files - url: sql-data-sources-parquet.html - - text: ORC Files - url: sql-data-sources-orc.html - - text: JSON Files - url: sql-data-sources-json.html - - text: Hive Tables - url: sql-data-sources-hive-tables.html - - text: JDBC To Other Databases - url: sql-data-sources-jdbc.html - - text: Avro Files - url: sql-data-sources-avro.html - - text: Troubleshooting - url: sql-data-sources-troubleshooting.html -- text: Performance Tuning - url: sql-performance-tuning.html - subitems: - - text: Caching Data In Memory - url: sql-performance-tuning.html#caching-data-in-memory - - text: Other Configuration Options - url: sql-performance-tuning.html#other-configuration-options - - text: Broadcast Hint for SQL Queries - url: sql-performance-tuning.html#broadcast-hint-for-sql-queries -- text: Distributed SQL Engine - url: sql-distributed-sql-engine.html - subitems: - - text: "Running the Thrift JDBC/ODBC server" - url: sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server - - text: Running the Spark SQL CLI - url: sql-distributed-sql-engine.html#running-the-spark-sql-cli -- text: PySpark Usage Guide for Pandas with Apache Arrow - url: sql-pyspark-pandas-with-arrow.html - subitems: - - text: Apache Arrow in Spark - url: sql-pyspark-pandas-with-arrow.html#apache-arrow-in-spark - - text: "Enabling for Conversion to/from Pandas" - url: sql-pyspark-pandas-with-arrow.html#enabling-for-conversion-tofrom-pandas - - text: "Pandas UDFs (a.k.a. Vectorized UDFs)" - url: sql-pyspark-pandas-with-arrow.html#pandas-udfs-aka-vectorized-udfs - - text: Usage Notes - url: sql-pyspark-pandas-with-arrow.html#usage-notes -- text: Migration Guide - url: sql-migration-guide.html - subitems: - - text: Spark SQL Upgrading Guide - url: sql-migration-guide-upgrade.html - - text: Compatibility with Apache Hive - url: sql-migration-guide-hive-compatibility.html -- text: Reference - url: sql-reference.html - subitems: - - text: Data Types - url: sql-reference.html#data-types - - text: NaN Semantics - url: sql-reference.html#nan-semantics - - text: Arithmetic operations - url: sql-reference.html#arithmetic-operations diff --git a/docs/_includes/nav-left-wrapper-sql.html b/docs/_includes/nav-left-wrapper-sql.html deleted file mode 100644 index edc4cf4514d01..0000000000000 --- a/docs/_includes/nav-left-wrapper-sql.html +++ /dev/null @@ -1,6 +0,0 @@ -
    -
    -

    Spark SQL Guide

    - {% include nav-left.html nav=include.nav-sql %} -
    -
    \ No newline at end of file diff --git a/docs/_includes/nav-left.html b/docs/_includes/nav-left.html index 19d68fd191635..73176f4132554 100644 --- a/docs/_includes/nav-left.html +++ b/docs/_includes/nav-left.html @@ -10,8 +10,7 @@ {% endif %} - {% assign tag = item.url | remove: ".html" %} - {% if item.subitems and navurl contains tag %} + {% if item.subitems and navurl contains item.url %} {% include nav-left.html nav=item.subitems %} {% endif %} {% endfor %} diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index cbe4306799d43..88d549c3f1010 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -126,12 +126,8 @@
    - {% if page.url contains "/ml" or page.url contains "/sql" %} - {% if page.url contains "/ml" %} - {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %} - {% else %} - {% include nav-left-wrapper-sql.html nav-sql=site.data.menu-sql %} - {% endif %} + {% if page.url contains "/ml" %} + {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %}
    @@ -188,8 +184,7 @@

    {{ page.title }}

    }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + - '?config=TeX-AMS-MML_HTMLorMML'; + 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; d.getElementsByTagName('head')[0].appendChild(script); }(document)); diff --git a/docs/sql-data-sources-avro.md b/docs/avro-data-source-guide.md similarity index 94% rename from docs/sql-data-sources-avro.md rename to docs/avro-data-source-guide.md index bfe641d1c6d1d..d3b81f029d377 100644 --- a/docs/sql-data-sources-avro.md +++ b/docs/avro-data-source-guide.md @@ -142,10 +142,7 @@ StreamingQuery query = output ## Data Source Option -Data source options of Avro can be set via: - * the `.option` method on `DataFrameReader` or `DataFrameWriter`. - * the `options` parameter in function `from_avro`. - +Data source options of Avro can be set using the `.option` method on `DataFrameReader` or `DataFrameWriter`. @@ -180,19 +177,6 @@ Data source options of Avro can be set via: Currently supported codecs are uncompressed, snappy, deflate, bzip2 and xz.
    If the option is not set, the configuration spark.sql.avro.compression.codec config is taken into account. - - - - - -
    Property NameDefaultMeaningScope
    write
    modeFAILFASTThe mode option allows to specify parse mode for function from_avro.
    - Currently supported modes are: -
      -
    • FAILFAST: Throws an exception on processing corrupted record.
    • -
    • PERMISSIVE: Corrupt records are processed as null result. Therefore, the - data schema is forced to be fully nullable, which might be different from the one user provided.
    • -
    -
    function from_avro
    ## Configuration diff --git a/docs/building-spark.md b/docs/building-spark.md index 8af90db9a19dd..1501f0bb84544 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -12,7 +12,7 @@ redirect_from: "building-with-maven.html" ## Apache Maven The Maven-based build is the build of reference for Apache Spark. -Building Spark using Maven requires Maven 3.5.4 and Java 8. +Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+. Note that support for Java 7 was removed as of Spark 2.2.0. ### Setting up Maven's Memory Usage @@ -49,20 +49,25 @@ To create a Spark distribution like those distributed by the to be runnable, use `./dev/make-distribution.sh` in the project root directory. It can be configured with Maven profile settings and so on like the direct Maven build. Example: - ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes + ./dev/make-distribution.sh --name custom-spark --pip --r --tgz -Psparkr -Phadoop-2.7 -Phive -Phive-thriftserver -Pmesos -Pyarn -Pkubernetes This will build Spark distribution along with Python pip and R packages. For more information on usage, run `./dev/make-distribution.sh --help` ## Specifying the Hadoop Version and Enabling YARN You can specify the exact version of Hadoop to compile against through the `hadoop.version` property. +If unset, Spark will build against Hadoop 2.6.X by default. You can enable the `yarn` profile and optionally set the `yarn.version` property if it is different from `hadoop.version`. -Example: +Examples: - ./build/mvn -Pyarn -Dhadoop.version=2.8.5 -DskipTests clean package + # Apache Hadoop 2.6.X + ./build/mvn -Pyarn -DskipTests clean package + + # Apache Hadoop 2.7.X and later + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.3 -DskipTests clean package ## Building With Hive and JDBC Support @@ -89,6 +94,22 @@ like ZooKeeper and Hadoop itself. ## Building with Kubernetes support ./build/mvn -Pkubernetes -DskipTests clean package + +## Building with Kafka 0.8 support + +Kafka 0.8 support must be explicitly enabled with the `kafka-0-8` profile. +Note: Kafka 0.8 support is deprecated as of Spark 2.3.0. + + ./build/mvn -Pkafka-0-8 -DskipTests clean package + +Kafka 0.10 support is still automatically built. + +## Building with Flume support + +Apache Flume support must be explicitly enabled with the `flume` profile. +Note: Flume support is deprecated as of Spark 2.3.0. + + ./build/mvn -Pflume -DskipTests clean package ## Building submodules individually @@ -251,31 +272,3 @@ For SBT, specify a complete scala version using (e.g. 2.12.6): ./build/sbt -Dscala.version=2.12.6 Otherwise, the sbt-pom-reader plugin will use the `scala.version` specified in the spark-parent pom. - -## Running Jenkins tests with Github Enterprise - -To run tests with Jenkins: - - ./dev/run-tests-jenkins - -If use an individual repository or a repository on GitHub Enterprise, export below environment variables before running above command. - -### Related environment variables - - - - - - - - - - - - - -
    Variable NameDefaultMeaning
    SPARK_PROJECT_URLhttps://github.com/apache/spark - The Spark project URL of GitHub Enterprise. -
    GITHUB_API_BASEhttps://api.github.com/repos/apache/spark - The Spark project API server URL of GitHub Enterprise. -
    diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 1f0822f7a317b..7277e2fb2731d 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -45,7 +45,7 @@ There are several useful things to note about this architecture: # Cluster Manager Types -The system currently supports several cluster managers: +The system currently supports three cluster managers: * [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it easy to set up a cluster. diff --git a/docs/configuration.md b/docs/configuration.md index 8cb0ed1502126..55773937d4d71 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -761,17 +761,6 @@ Apart from these, the following properties are also available, and may be useful Compression will use spark.io.compression.codec. - - spark.eventLog.allowErasureCoding - false - - Whether to allow event logs to use erasure coding, or turn erasure coding off, regardless of - filesystem defaults. On HDFS, erasure coded files will not update as quickly as regular - replicated files, so the application updates will take longer to appear in the History Server. - Note that even if this is true, Spark will still not force the file to use erasure coding, it - will simply use filesystem defaults. - - spark.eventLog.dir file:///tmp/spark-events @@ -965,7 +954,7 @@ Apart from these, the following properties are also available, and may be useful org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, org.apache.spark.io.SnappyCompressionCodec, - and org.apache.spark.io.ZStdCompressionCodec. + and org.apache.spark.io.ZstdCompressionCodec. @@ -2028,7 +2017,7 @@ showDF(properties, numRows = 200, truncate = FALSE) Maximum rate (number of records per second) at which data will be read from each Kafka partition when using the new Kafka direct stream API. See the - Kafka Integration guide + Kafka Integration guide for more details. @@ -2040,6 +2029,16 @@ showDF(properties, numRows = 200, truncate = FALSE) partition when using the new Kafka direct stream API. + + spark.streaming.kafka.maxRetries + 1 + + Maximum number of consecutive retries the driver will make in order to find + the latest offsets on the leader of each partition (a default value of 1 + means that the driver will make a maximum of 2 attempts). Only applies to + the new Kafka direct stream API. + + spark.streaming.ui.retainedBatches 1000 diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index cb96fd773aa5a..35293348e3f3d 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -726,7 +726,7 @@ class GraphOps[VD, ED] { var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) + var messages = g.mapReduceTriplets(sendMsg, mergeMsg) var activeMessages = messages.count() // Loop until no messages remain or maxIterations is achieved var i = 0 diff --git a/docs/index.md b/docs/index.md index b9996cc8645d9..509267563a872 100644 --- a/docs/index.md +++ b/docs/index.md @@ -10,11 +10,6 @@ It provides high-level APIs in Java, Scala, Python and R, and an optimized engine that supports general execution graphs. It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) before downloading and running Spark. - # Downloading Get Spark from the [downloads page](https://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark uses Hadoop's client libraries for HDFS and YARN. Downloads are pre-packaged for a handful of popular Hadoop versions. @@ -35,6 +30,9 @@ Spark runs on Java 8+, Python 2.7+/3.4+ and R 3.1+. For the Scala API, Spark {{s uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). +Note that support for Java 7, Python 2.6 and old Hadoop versions before 2.6.5 were removed as of Spark 2.2.0. +Support for Scala 2.10 was removed as of 2.3.0. + # Running the Examples and Shell Spark comes with several sample programs. Scala, Java, Python and R examples are in the diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 287fce61540e9..13514e11b9332 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -58,7 +58,6 @@ $(document).ready(function() { }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + - '?config=TeX-AMS-MML_HTMLorMML'; + 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; document.getElementsByTagName('head')[0].appendChild(script); }); diff --git a/docs/ml-datasource.md b/docs/ml-datasource.md deleted file mode 100644 index 15083326240ac..0000000000000 --- a/docs/ml-datasource.md +++ /dev/null @@ -1,108 +0,0 @@ ---- -layout: global -title: Data sources -displayTitle: Data sources ---- - -In this section, we introduce how to use data source in ML to load data. -Beside some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML. - -**Table of Contents** - -* This will become a table of contents (this text will be scraped). -{:toc} - -## Image data source - -This image data source is used to load image files from a directory, it can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` in Java library. -The loaded DataFrame has one `StructType` column: "image", containing image data stored as image schema. -The schema of the `image` column is: - - origin: `StringType` (represents the file path of the image) - - height: `IntegerType` (height of the image) - - width: `IntegerType` (width of the image) - - nChannels: `IntegerType` (number of image channels) - - mode: `IntegerType` (OpenCV-compatible type) - - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) - - -
    -
    -[`ImageDataSource`](api/scala/index.html#org.apache.spark.ml.source.image.ImageDataSource) -implements a Spark SQL data source API for loading image data as a DataFrame. - -{% highlight scala %} -scala> val df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") -df: org.apache.spark.sql.DataFrame = [image: struct] - -scala> df.select("image.origin", "image.width", "image.height").show(truncate=false) -+-----------------------------------------------------------------------+-----+------+ -|origin |width|height| -+-----------------------------------------------------------------------+-----+------+ -|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | -|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | -|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | -|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | -+-----------------------------------------------------------------------+-----+------+ -{% endhighlight %} -
    - -
    -[`ImageDataSource`](api/java/org/apache/spark/ml/source/image/ImageDataSource.html) -implements Spark SQL data source API for loading image data as DataFrame. - -{% highlight java %} -Dataset imagesDF = spark.read().format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens"); -imageDF.select("image.origin", "image.width", "image.height").show(false); -/* -Will output: -+-----------------------------------------------------------------------+-----+------+ -|origin |width|height| -+-----------------------------------------------------------------------+-----+------+ -|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | -|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | -|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | -|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | -+-----------------------------------------------------------------------+-----+------+ -*/ -{% endhighlight %} -
    - -
    -In PySpark we provide Spark SQL data source API for loading image data as DataFrame. - -{% highlight python %} ->>> df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") ->>> df.select("image.origin", "image.width", "image.height").show(truncate=False) -+-----------------------------------------------------------------------+-----+------+ -|origin |width|height| -+-----------------------------------------------------------------------+-----+------+ -|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | -|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | -|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | -|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | -+-----------------------------------------------------------------------+-----+------+ -{% endhighlight %} -
    - -
    -In SparkR we provide Spark SQL data source API for loading image data as DataFrame. - -{% highlight r %} -> df = read.df("data/mllib/images/origin/kittens", "image") -> head(select(df, df$image.origin, df$image.width, df$image.height)) - -1 file:///spark/data/mllib/images/origin/kittens/54893.jpg -2 file:///spark/data/mllib/images/origin/kittens/DP802813.jpg -3 file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg -4 file:///spark/data/mllib/images/origin/kittens/DP153539.jpg - width height -1 300 311 -2 199 313 -3 300 200 -4 300 296 - -{% endhighlight %} -
    - - -
    diff --git a/docs/ml-frequent-pattern-mining.md b/docs/ml-frequent-pattern-mining.md index c2043d495c149..81634de8aade7 100644 --- a/docs/ml-frequent-pattern-mining.md +++ b/docs/ml-frequent-pattern-mining.md @@ -85,56 +85,3 @@ Refer to the [R API docs](api/R/spark.fpGrowth.html) for more details.
    - -## PrefixSpan - -PrefixSpan is a sequential pattern mining algorithm described in -[Pei et al., Mining Sequential Patterns by Pattern-Growth: The -PrefixSpan Approach](http://dx.doi.org/10.1109%2FTKDE.2004.77). We refer -the reader to the referenced paper for formalizing the sequential -pattern mining problem. - -`spark.ml`'s PrefixSpan implementation takes the following parameters: - -* `minSupport`: the minimum support required to be considered a frequent - sequential pattern. -* `maxPatternLength`: the maximum length of a frequent sequential - pattern. Any frequent pattern exceeding this length will not be - included in the results. -* `maxLocalProjDBSize`: the maximum number of items allowed in a - prefix-projected database before local iterative processing of the - projected database begins. This parameter should be tuned with respect - to the size of your executors. -* `sequenceCol`: the name of the sequence column in dataset (default "sequence"), rows with - nulls in this column are ignored. - -**Examples** - -
    - -
    -Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.fpm.PrefixSpan) for more details. - -{% include_example scala/org/apache/spark/examples/ml/PrefixSpanExample.scala %} -
    - -
    -Refer to the [Java API docs](api/java/org/apache/spark/ml/fpm/PrefixSpan.html) for more details. - -{% include_example java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java %} -
    - -
    -Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.fpm.PrefixSpan) for more details. - -{% include_example python/ml/prefixspan_example.py %} -
    - -
    - -Refer to the [R API docs](api/R/spark.prefixSpan.html) for more details. - -{% include_example r/ml/prefixSpan.R %} -
    - -
    \ No newline at end of file diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md index 8c01ccb94c75f..e22e9003c30f6 100644 --- a/docs/ml-pipeline.md +++ b/docs/ml-pipeline.md @@ -57,7 +57,7 @@ E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and p Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. This API adopts the `DataFrame` from Spark SQL in order to support a variety of data types. -`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-reference.html#data-types) for a list of supported types. +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#data-types) for a list of supported types. In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. diff --git a/docs/quick-start.md b/docs/quick-start.md index 28186c11887fc..ef7af6c3f6cec 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -17,11 +17,6 @@ you can download a package for any version of Hadoop. Note that, before Spark 2.0, the main programming interface of Spark was the Resilient Distributed Dataset (RDD). After Spark 2.0, RDDs are replaced by Dataset, which is strongly-typed like an RDD, but with richer optimizations under the hood. The RDD interface is still supported, and you can get a more detailed reference at the [RDD programming guide](rdd-programming-guide.html). However, we highly recommend you to switch to use Dataset, which has better performance than RDD. See the [SQL programming guide](sql-programming-guide.html) to get more information about Dataset. -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) before running Spark. - # Interactive Analysis with the Spark Shell ## Basics diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 29a4d86fecb44..205d1a3f3915c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -12,11 +12,6 @@ Kubernetes scheduler that has been added to Spark. In future versions, there may be behavioral changes around configuration, container images and entrypoints.** -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. - # Prerequisites * A runnable distribution of Spark 2.3 or above. @@ -50,8 +45,7 @@ logs and remains in "completed" state in the Kubernetes API until it's eventuall Note that in the completed state, the driver pod does *not* use any computational or memory resources. -The driver and executor pod scheduling is handled by Kubernetes. Communication to the Kubernetes API is done via fabric8, and we are -currently running kubernetes-client version 4.1.0. Make sure that when you are making infrastructure additions that you are aware of said version. It is possible to schedule the +The driver and executor pod scheduling is handled by Kubernetes. It is possible to schedule the driver and executor pods on a subset of available nodes through a [node selector](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector) using the configuration property for it. It will be possible to use more advanced scheduling hints like [node/pod affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity) in a future release. @@ -197,7 +191,6 @@ Spark users can similarly use template files to define the driver or executor po To do so, specify the spark properties `spark.kubernetes.driver.podTemplateFile` and `spark.kubernetes.executor.podTemplateFile` to point to local files accessible to the `spark-submit` process. To allow the driver pod access the executor pod template file, the file will be automatically mounted onto a volume in the driver pod when it's created. -Spark does not do any validation after unmarshalling these template files and relies on the Kubernetes API server for validation. It is important to note that Spark is opinionated about certain pod configurations so there are values in the pod template that will always be overwritten by Spark. Therefore, users of this feature should note that specifying @@ -838,52 +831,11 @@ specific to Spark on Kubernetes. spark.kubernetes.pyspark.pythonVersion - "3" + "2" This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. - - spark.kubernetes.kerberos.krb5.path - (none) - - Specify the local location of the krb5.conf file to be mounted on the driver and executors for Kerberos interaction. - It is important to note that the KDC defined needs to be visible from inside the containers. - - - - spark.kubernetes.kerberos.krb5.configMapName - (none) - - Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted on the driver and executors - for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also - be in the same namespace of the driver and executor pods. - - - - spark.kubernetes.hadoop.configMapName - (none) - - Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, to be mounted on the driver - and executors for custom Hadoop configuration. - - - - spark.kubernetes.kerberos.tokenSecret.name - (none) - - Specify the name of the secret where your existing delegation tokens are stored. This removes the need for the job user - to provide any kerberos credentials for launching a job. - - - - spark.kubernetes.kerberos.tokenSecret.itemKey - (none) - - Specify the item key of the data where your existing delegation tokens are stored. This removes the need for the job user - to provide any kerberos credentials for launching a job. - - spark.kubernetes.driver.podTemplateFile (none) @@ -1048,4 +1000,4 @@ The following affect the driver and executor containers. All other containers in spark conf and pod template files. - + \ No newline at end of file diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 2502cd4ca86f4..b473e654563d6 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -13,11 +13,6 @@ The advantages of deploying Spark with Mesos include: [frameworks](https://mesos.apache.org/documentation/latest/frameworks/) - scalable partitioning between multiple instances of Spark -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. - # How it Works In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index a7a448fbeb65e..687f9e46c3285 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -9,11 +9,6 @@ Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html) was added to Spark in version 0.6.0, and improved in subsequent releases. -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. - # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. @@ -126,43 +121,6 @@ To use a custom metrics.properties for the application master and executors, upd Use lower-case suffixes, e.g. k, m, g, t, and p, for kibi-, mebi-, gibi-, tebi-, and pebibytes, respectively. - - spark.yarn.am.resource.{resource-type} - (none) - - Amount of resource to use for the YARN Application Master in client mode. - In cluster mode, use spark.yarn.driver.resource.<resource-type> instead. - Please note that this feature can be used only with YARN 3.0+ - For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html -

    - Example: - To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu - - - - spark.yarn.driver.resource.{resource-type} - (none) - - Amount of resource to use for the YARN Application Master in cluster mode. - Please note that this feature can be used only with YARN 3.0+ - For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html -

    - Example: - To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu - - - - spark.yarn.executor.resource.{resource-type} - (none) - - Amount of resource to use per executor process. - Please note that this feature can be used only with YARN 3.0+ - For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html -

    - Example: - To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu - - spark.yarn.am.cores 1 @@ -438,7 +396,8 @@ To use a custom metrics.properties for the application master and executors, upd and those log files will be aggregated in a rolling fashion. This will be used with YARN's rolling log aggregation, to enable this feature in YARN side yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds should be - configured in yarn-site.xml. The Spark log4j appender needs be changed to use + configured in yarn-site.xml. + This feature can only be used with Hadoop 2.6.4+. The Spark log4j appender needs be changed to use FileAppender or another appender that can handle the files being removed while it is running. Based on the file name configured in the log4j configuration (like spark.log), the user should set the regex (spark*) to include all the log files that need to be aggregated. diff --git a/docs/security.md b/docs/security.md index 2f7fa9c6179f4..7fb3e17de94c9 100644 --- a/docs/security.md +++ b/docs/security.md @@ -6,20 +6,7 @@ title: Security * This will become a table of contents (this text will be scraped). {:toc} -# Spark Security: Things You Need To Know - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Spark supports multiple deployments types and each one supports different levels of security. Not -all deployment types will be secure in all environments and none are secure by default. Be -sure to evaluate your environment, what Spark supports, and take the appropriate measure to secure -your Spark deployment. - -There are many different types of security concerns. Spark does not necessarily protect against -all things. Listed below are some of the things Spark supports. Also check the deployment -documentation for the type of deployment you are using for deployment specific settings. Anything -not documented, Spark does not support. - -# Spark RPC (Communication protocol between Spark processes) +# Spark RPC ## Authentication @@ -136,7 +123,7 @@ The following table describes the different options available for configuring th Spark supports encrypting temporary data written to local disks. This covers shuffle files, shuffle spills and data blocks stored on disk (for both caching and broadcast variables). It does not cover encrypting output data generated by applications with APIs such as `saveAsHadoopFile` or -`saveAsTable`. It also may not cover temporary files created explicitly by the user. +`saveAsTable`. The following settings cover enabling encryption for data written to disk: @@ -735,82 +722,7 @@ with encryption, at least. The Kerberos login will be periodically renewed using the provided credentials, and new delegation tokens for supported will be created. -## Secure Interaction with Kubernetes - -When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens -so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are -shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: - -In all cases you must define the environment variable: `HADOOP_CONF_DIR` or -`spark.kubernetes.hadoop.configMapName.` - -It also important to note that the KDC needs to be visible from inside the containers. - -If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be -achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing ConfigMap. - -1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: -```bash -/usr/bin/kinit -kt / -/opt/spark/bin/spark-submit \ - --deploy-mode cluster \ - --class org.apache.spark.examples.HdfsTest \ - --master k8s:// \ - --conf spark.executor.instances=1 \ - --conf spark.app.name=spark-hdfs \ - --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_.jar \ - -``` -2. Submitting with a local Keytab and Principal -```bash -/opt/spark/bin/spark-submit \ - --deploy-mode cluster \ - --class org.apache.spark.examples.HdfsTest \ - --master k8s:// \ - --conf spark.executor.instances=1 \ - --conf spark.app.name=spark-hdfs \ - --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kerberos.keytab= \ - --conf spark.kerberos.principal= \ - --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_.jar \ - -``` - -3. Submitting with pre-populated secrets, that contain the Delegation Token, already existing within the namespace -```bash -/opt/spark/bin/spark-submit \ - --deploy-mode cluster \ - --class org.apache.spark.examples.HdfsTest \ - --master k8s:// \ - --conf spark.executor.instances=1 \ - --conf spark.app.name=spark-hdfs \ - --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.tokenSecret.name= \ - --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ - --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ - local:///opt/spark/examples/jars/spark-examples_.jar \ - -``` -3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap and pre-created `HADOOP_CONF_DIR` ConfigMap -```bash -/opt/spark/bin/spark-submit \ - --deploy-mode cluster \ - --class org.apache.spark.examples.HdfsTest \ - --master k8s:// \ - --conf spark.executor.instances=1 \ - --conf spark.app.name=spark-hdfs \ - --conf spark.kubernetes.container.image=spark:latest \ - --conf spark.kubernetes.kerberos.tokenSecret.name= \ - --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ - --conf spark.kubernetes.hadoop.configMapName= \ - --conf spark.kubernetes.kerberos.krb5.configMapName= \ - local:///opt/spark/examples/jars/spark-examples_.jar \ - -``` # Event Logging If your applications are using event logging, the directory where the event logs go diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 49ef2e1ce2a1b..7975b0c8b11ca 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -8,11 +8,6 @@ title: Spark Standalone Mode In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing. -# Security - -Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. - # Installing Spark Standalone to a Cluster To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](building-spark.html). diff --git a/docs/sparkr.md b/docs/sparkr.md index cc6bc6d14853d..55e8f15da17ca 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -104,7 +104,7 @@ The following Spark driver properties can be set in `sparkConfig` with `sparkR.s ## Creating SparkDataFrames -With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-data-sources-hive-tables.html), or from other [data sources](sql-data-sources.html). +With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). ### From local data frames The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically, we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. @@ -125,7 +125,7 @@ head(df) ### From Data Sources -SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-data-sources-load-save-functions.html#manually-specifying-options) that are available for the built-in data sources. +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](https://spark.apache.org/third-party-projects.html), you can find data source connectors for popular file formats like Avro. These packages can either be added by @@ -180,7 +180,7 @@ write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite" ### From Hive tables -You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-getting-started.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).

    {% highlight r %} @@ -450,48 +450,6 @@ print(model.summaries) {% endhighlight %}
    -### Eager execution - -If eager execution is enabled, the data will be returned to R client immediately when the `SparkDataFrame` is created. By default, eager execution is not enabled and can be enabled by setting the configuration property `spark.sql.repl.eagerEval.enabled` to `true` when the `SparkSession` is started up. - -Maximum number of rows and maximum number of characters per column of data to display can be controlled by `spark.sql.repl.eagerEval.maxNumRows` and `spark.sql.repl.eagerEval.truncate` configuration properties, respectively. These properties are only effective when eager execution is enabled. If these properties are not set explicitly, by default, data up to 20 rows and up to 20 characters per column will be showed. - -
    -{% highlight r %} - -# Start up spark session with eager execution enabled -sparkR.session(master = "local[*]", - sparkConfig = list(spark.sql.repl.eagerEval.enabled = "true", - spark.sql.repl.eagerEval.maxNumRows = as.integer(10))) - -# Create a grouped and sorted SparkDataFrame -df <- createDataFrame(faithful) -df2 <- arrange(summarize(groupBy(df, df$waiting), count = n(df$waiting)), "waiting") - -# Similar to R data.frame, displays the data returned, instead of SparkDataFrame class string -df2 - -##+-------+-----+ -##|waiting|count| -##+-------+-----+ -##| 43.0| 1| -##| 45.0| 3| -##| 46.0| 5| -##| 47.0| 4| -##| 48.0| 3| -##| 49.0| 5| -##| 50.0| 5| -##| 51.0| 6| -##| 52.0| 5| -##| 53.0| 7| -##+-------+-----+ -##only showing top 10 rows - -{% endhighlight %} -
    - -Note that to enable eager execution in `sparkR` shell, add `spark.sql.repl.eagerEval.enabled=true` configuration property to the `--conf` option. - ## Running SQL Queries from SparkR A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. @@ -552,7 +510,6 @@ SparkR supports the following machine learning algorithms currently: #### Frequent Pattern Mining * [`spark.fpGrowth`](api/R/spark.fpGrowth.html) : [`FP-growth`](ml-frequent-pattern-mining.html#fp-growth) -* [`spark.prefixSpan`](api/R/spark.prefixSpan.html) : [`PrefixSpan`](ml-frequent-pattern-mining.html#prefixSpan) #### Statistics @@ -709,13 +666,8 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma ## Upgrading to SparkR 2.3.1 and above - - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. + - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-base. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. ## Upgrading to SparkR 2.4.0 - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. - -## Upgrading to SparkR 3.0.0 - - - The deprecated methods `sparkR.init`, `sparkRSQL.init`, `sparkRHive.init` have been removed. Use `sparkR.session` instead. - - The deprecated methods `parquetFile`, `saveAsParquetFile`, `jsonFile`, `registerTempTable`, `createExternalTable`, `dropTempTable`, `unionAll` have been removed. Use `read.parquet`, `write.parquet`, `read.json`, `createOrReplaceTempView`, `createTable`, `dropTempView`, `union` instead. diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md deleted file mode 100644 index 687e6f8e0a7cc..0000000000000 --- a/docs/sql-data-sources-hive-tables.md +++ /dev/null @@ -1,166 +0,0 @@ ---- -layout: global -title: Hive Tables -displayTitle: Hive Tables ---- - -* Table of contents -{:toc} - -Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). -However, since Hive has a large number of dependencies, these dependencies are not included in the -default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them -automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as -they will need access to the Hive serialization and deserialization libraries (SerDes) in order to -access data stored in Hive. - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), -and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the Spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the Spark application. - -
    - -
    -{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} -
    - -
    -{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} -
    - -
    -{% include_example spark_hive python/sql/hive.py %} -
    - -
    - -When working with Hive one must instantiate `SparkSession` with Hive support. This -adds support for finding tables in the MetaStore and writing queries using HiveQL. - -{% include_example spark_hive r/RSparkSQLExample.R %} - -
    -
    - -### Specifying storage format for Hive tables - -When you create a Hive table, you need to define how this table should read/write data from/to file system, -i.e. the "input format" and "output format". You also need to define how this table should deserialize the data -to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage -format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. -By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when -creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    fileFormat - A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and - "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. -
    inputFormat, outputFormat - These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, - e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not - specify them if you already specified the `fileFormat` option. -
    serde - This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option - if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" - don't include the serde information and you can use this option with these 3 fileFormats. -
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim - These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. -
    - -All other properties defined with `OPTIONS` will be regarded as Hive serde properties. - -### Interacting with Different Versions of Hive Metastore - -One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, -which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary -build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. -Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL -will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). - -The following options can be used to configure the version of Hive that is used to retrieve metadata: - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 - Version of the Hive metastore. Available - options are 0.12.0 through 2.3.3. -
    spark.sql.hive.metastore.jarsbuiltin - Location of the jars that should be used to instantiate the HiveMetastoreClient. This - property can be one of three options: -
      -
    1. builtin
    2. - Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is - enabled. When this option is chosen, spark.sql.hive.metastore.version must be - either 1.2.1 or not defined. -
    3. maven
    4. - Use Hive jars of specified version downloaded from Maven repositories. This configuration - is not generally recommended for production deployments. -
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive - and its dependencies, including the correct version of Hadoop. These jars only need to be - present on the driver, but if you are running in yarn cluster mode then you must ensure - they are packaged with your application.
    6. -
    -
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    -

    - A comma-separated list of class prefixes that should be loaded using the classloader that is - shared between Spark SQL and a specific version of Hive. An example of classes that should - be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need - to be shared are those that interact with classes that are already shared. For example, - custom appenders that are used by log4j. -

    -
    spark.sql.hive.metastore.barrierPrefixes(empty) -

    - A comma separated list of class prefixes that should explicitly be reloaded for each version - of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a - prefix that typically would be shared (i.e. org.apache.spark.*). -

    -
    diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md deleted file mode 100644 index 057e8217241aa..0000000000000 --- a/docs/sql-data-sources-jdbc.md +++ /dev/null @@ -1,223 +0,0 @@ ---- -layout: global -title: JDBC To Other Databases -displayTitle: JDBC To Other Databases ---- - -* Table of contents -{:toc} - -Spark SQL also includes a data source that can read data from other databases using JDBC. This -functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). -This is because the results are returned -as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. -The JDBC data source is also easier to use from Java or Python as it does not require the user to -provide a ClassTag. -(Note that this is different than the Spark SQL JDBC server, which allows other applications to -run queries using Spark SQL). - -To get started you will need to include the JDBC driver for your particular database on the -spark classpath. For example, to connect to postgres from the Spark Shell you would run the -following command: - -{% highlight bash %} -bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar -{% endhighlight %} - -Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using -the Data Sources API. Users can specify the JDBC connection properties in the data source options. -user and password are normally provided as connection properties for -logging into the data sources. In addition to the connection properties, Spark also supports -the following case-insensitive options: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    url - The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret -
    dbtable - The JDBC table that should be read from or written into. Note that when using it in the read - path anything that is valid in a FROM clause of a SQL query can be used. - For example, instead of a full table you could also use a subquery in parentheses. It is not - allowed to specify `dbtable` and `query` options at the same time. -
    query - A query that will be used to read data into Spark. The specified query will be parenthesized and used - as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. - As an example, spark will issue a query of the following form to the JDBC Source.

    - SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    - Below are couple of restrictions while using this option.
    -
      -
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. -
    3. It is not allowed to specify `query` and `partitionColumn` options at the same time. When specifying - `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and - partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      - Example:
      - - spark.read.format("jdbc")
      -    .option("dbtable", "(select c1, c2 from t1) as subq")
      -    .option("partitionColumn", "subq.c1"
      -    .load() -
    4. -
    -
    driver - The class name of the JDBC driver to use to connect to this URL. -
    partitionColumn, lowerBound, upperBound - These options must all be specified if any of them is specified. In addition, - numPartitions must be specified. They describe how to partition the table when - reading in parallel from multiple workers. - partitionColumn must be a numeric, date, or timestamp column from the table in question. - Notice that lowerBound and upperBound are just used to decide the - partition stride, not for filtering the rows in table. So all rows in the table will be - partitioned and returned. This option applies only to reading. -
    numPartitions - The maximum number of partitions that can be used for parallelism in table reading and - writing. This also determines the maximum number of concurrent JDBC connections. - If the number of partitions to write exceeds this limit, we decrease it to this limit by - calling coalesce(numPartitions) before writing. -
    queryTimeout - The number of seconds the driver will wait for a Statement object to execute to the given - number of seconds. Zero means there is no limit. In the write path, this option depends on - how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver - checks the timeout of each query instead of an entire JDBC batch. - It defaults to 0. -
    fetchsize - The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. -
    batchsize - The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. -
    isolationLevel - The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. -
    sessionInitStatement - After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") -
    truncate - This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. -
    cascadeTruncate - This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. -
    createTableOptions - This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. -
    createTableColumnTypes - The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. -
    customSchema - The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. -
    pushDownPredicate - The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. -
    - -
    - -
    -{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example jdbc_dataset python/sql/datasource.py %} -
    - -
    -{% include_example jdbc_dataset r/RSparkSQLExample.R %} -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jdbcTable -USING org.apache.spark.sql.jdbc -OPTIONS ( - url "jdbc:postgresql:dbserver", - dbtable "schema.tablename", - user 'username', - password 'password' -) - -INSERT INTO TABLE jdbcTable -SELECT * FROM resultTable -{% endhighlight %} - -
    -
    diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md deleted file mode 100644 index f84336b5716db..0000000000000 --- a/docs/sql-data-sources-json.md +++ /dev/null @@ -1,81 +0,0 @@ ---- -layout: global -title: JSON Files -displayTitle: JSON Files ---- - -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. -This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. -This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SparkSession.read.json` on a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` parameter to `True`. - -{% include_example json_dataset python/sql/datasource.py %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using -the `read.json()` function, which loads data from a directory of JSON files where each line of the -files is a JSON object. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. - -{% include_example json_dataset r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jsonTable -USING org.apache.spark.sql.json -OPTIONS ( - path "examples/src/main/resources/people.json" -) - -SELECT * FROM jsonTable - -{% endhighlight %} - -
    - -
    \ No newline at end of file diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md deleted file mode 100644 index e4c7b1766f918..0000000000000 --- a/docs/sql-data-sources-load-save-functions.md +++ /dev/null @@ -1,327 +0,0 @@ ---- -layout: global -title: Generic Load/Save Functions -displayTitle: Generic Load/Save Functions ---- - -* Table of contents -{:toc} - - -In the simplest form, the default data source (`parquet` unless otherwise configured by -`spark.sql.sources.default`) will be used for all operations. - - -
    -
    -{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example generic_load_save_functions python/sql/datasource.py %} -
    - -
    - -{% include_example generic_load_save_functions r/RSparkSQLExample.R %} - -
    -
    - -### Manually Specifying Options - -You can also manually specify the data source that will be used along with any extra options -that you would like to pass to the data source. Data sources are specified by their fully qualified -name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data -source type can be converted into other types using this syntax. - -To load a JSON file you can use: - -
    -
    -{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options r/RSparkSQLExample.R %} -
    -
    - -To load a CSV file you can use: - -
    -
    -{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options_csv python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options_csv r/RSparkSQLExample.R %} - -
    -
    - -The extra options are also used during write operation. -For example, you can control bloom filters and dictionary encodings for ORC data sources. -The following ORC example will create bloom filter and use dictionary encoding only for `favorite_color`. -For Parquet, there exists `parquet.enable.dictionary`, too. -To find more detailed information about the extra ORC/Parquet options, -visit the official Apache ORC/Parquet websites. - -
    - -
    -{% include_example manual_save_options_orc scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_save_options_orc java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_save_options_orc python/sql/datasource.py %} -
    - -
    -{% include_example manual_save_options_orc r/RSparkSQLExample.R %} -
    - -
    - -{% highlight sql %} -CREATE TABLE users_with_options ( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING ORC -OPTIONS ( - orc.bloom.filter.columns 'favorite_color', - orc.dictionary.key.threshold '1.0', - orc.column.encoding.direct 'name' -) -{% endhighlight %} - -
    - -
    - -### Run SQL on files directly - -Instead of using read API to load a file into DataFrame and query it, you can also query that -file directly with SQL. - -
    -
    -{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example direct_sql python/sql/datasource.py %} -
    - -
    -{% include_example direct_sql r/RSparkSQLExample.R %} - -
    -
    - -### Save Modes - -Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if -present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the -new data. - - - - - - - - - - - - - - - - - - - - - - - -
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) - When saving a DataFrame to a data source, if data already exists, - an exception is expected to be thrown. -
    SaveMode.Append"append" - When saving a DataFrame to a data source, if data/table already exists, - contents of the DataFrame are expected to be appended to existing data. -
    SaveMode.Overwrite"overwrite" - Overwrite mode means that when saving a DataFrame to a data source, - if data/table already exists, existing data is expected to be overwritten by the contents of - the DataFrame. -
    SaveMode.Ignore"ignore" - Ignore mode means that when saving a DataFrame to a data source, if data already exists, - the save operation is expected not to save the contents of the DataFrame and not to - change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. -
    - -### Saving to Persistent Tables - -`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` -command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a -default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, -`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the -Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as -long as you maintain your connection to the same metastore. A DataFrame for a persistent table can -be created by calling the `table` method on a `SparkSession` with the name of the table. - -For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the -`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, -the custom table path will not be removed and the table data is still there. If no custom table path is -specified, Spark will write data to a default table path under the warehouse directory. When the table is -dropped, the default table path will be removed too. - -Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: - -- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. -- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - -Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. - -### Bucketing, Sorting and Partitioning - -For file-based data source, it is also possible to bucket and sort or partition the output. -Bucketing and sorting are applicable only to persistent tables: - -
    - -
    -{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_sorting_and_bucketing python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_by_name( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -CLUSTERED BY(name) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. - - -
    - -
    -{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partitioning python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_by_favorite_color( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING csv PARTITIONED BY(favorite_color); - -{% endhighlight %} - -
    - -
    - -It is possible to use both partitioning and bucketing for a single table: - -
    - -
    -{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partition_and_bucket python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_and_partitioned( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -PARTITIONED BY (favorite_color) -CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -`partitionBy` creates a directory structure as described in the [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) section. -Thus, it has limited applicability to columns with high cardinality. In contrast - `bucketBy` distributes -data across a fixed number of buckets and can be used when a number of unique values is unbounded. diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md deleted file mode 100644 index ef07d2fa2e53c..0000000000000 --- a/docs/sql-data-sources-orc.md +++ /dev/null @@ -1,26 +0,0 @@ ---- -layout: global -title: ORC Files -displayTitle: ORC Files ---- - -Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. -To do that, the following configurations are newly added. The vectorized reader is used for the -native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` -is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC -serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), -the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables vectorized orc decoding in native implementation. If false, a new non-vectorized ORC reader is used in native implementation. For hive implementation, this is ignored.
    diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md deleted file mode 100644 index 4fed3eaf83e5d..0000000000000 --- a/docs/sql-data-sources-parquet.md +++ /dev/null @@ -1,321 +0,0 @@ ---- -layout: global -title: Parquet Files -displayTitle: Parquet Files ---- - -* Table of contents -{:toc} - -[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. -Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. When writing Parquet files, all columns are automatically converted to be nullable for -compatibility reasons. - -### Loading Data Programmatically - -Using the data from the above example: - -
    - -
    -{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example basic_parquet_example python/sql/datasource.py %} -
    - -
    - -{% include_example basic_parquet_example r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW parquetTable -USING org.apache.spark.sql.parquet -OPTIONS ( - path "examples/src/main/resources/people.parquet" -) - -SELECT * FROM parquetTable - -{% endhighlight %} - -
    - -
    - -### Partition Discovery - -Table partitioning is a common optimization approach used in systems like Hive. In a partitioned -table, data are usually stored in different directories, with partitioning column values encoded in -the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) -are able to discover and infer partitioning information automatically. -For example, we can store all our previously used -population data into a partitioned table using the following directory structure, with two extra -columns, `gender` and `country` as partitioning columns: - -{% highlight text %} - -path -└── to - └── table - ├── gender=male - │   ├── ... - │   │ - │   ├── country=US - │   │   └── data.parquet - │   ├── country=CN - │   │   └── data.parquet - │   └── ... - └── gender=female -    ├── ... -    │ -    ├── country=US -    │   └── data.parquet -    ├── country=CN -    │   └── data.parquet -    └── ... - -{% endhighlight %} - -By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL -will automatically extract the partitioning information from the paths. -Now the schema of the returned DataFrame becomes: - -{% highlight text %} - -root -|-- name: string (nullable = true) -|-- age: long (nullable = true) -|-- gender: string (nullable = true) -|-- country: string (nullable = true) - -{% endhighlight %} - -Notice that the data types of the partitioning columns are automatically inferred. Currently, -numeric data types, date, timestamp and string type are supported. Sometimes users may not want -to automatically infer the data types of the partitioning columns. For these use cases, the -automatic type inference can be configured by -`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type -inference is disabled, string type will be used for the partitioning columns. - -Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths -by default. For the above example, if users pass `path/to/table/gender=male` to either -`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a -partitioning column. If users need to specify the base path that partition discovery -should start with, they can set `basePath` in the data source options. For example, -when `path/to/table/gender=male` is the path of the data and -users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. - -### Schema Merging - -Like Protocol Buffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with -a simple schema, and gradually add more columns to the schema as needed. In this way, users may end -up with multiple Parquet files with different but mutually compatible schemas. The Parquet data -source is now able to automatically detect this case and merge schemas of all these files. - -Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we -turned it off by default starting from 1.5.0. You may enable it by - -1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the - examples below), or -2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. - -
    - -
    -{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example schema_merging python/sql/datasource.py %} -
    - -
    - -{% include_example schema_merging r/RSparkSQLExample.R %} - -
    - -
    - -### Hive metastore Parquet table conversion - -When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own -Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the -`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default. - -#### Hive/Parquet Schema Reconciliation - -There are two key differences between Hive and Parquet from the perspective of table schema -processing. - -1. Hive is case insensitive, while Parquet is not -1. Hive considers all columns nullable, while nullability in Parquet is significant - -Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a -Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: - -1. Fields that have the same name in both schema must have the same data type regardless of - nullability. The reconciled field should have the data type of the Parquet side, so that - nullability is respected. - -1. The reconciled schema contains exactly those fields defined in Hive metastore schema. - - - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. - - Any fields that only appear in the Hive metastore schema are added as nullable field in the - reconciled schema. - -#### Metadata Refreshing - -Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table -conversion is enabled, metadata of those converted tables are also cached. If these tables are -updated by Hive or other external tools, you need to refresh them manually to ensure consistent -metadata. - -
    - -
    - -{% highlight scala %} -// spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight java %} -// spark is an existing SparkSession -spark.catalog().refreshTable("my_table"); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -# spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight r %} -refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight sql %} -REFRESH TABLE my_table; -{% endhighlight %} - -
    - -
    - -### Configuration - -Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse - Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the Parquet schema. This - flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. -
    spark.sql.parquet.int96AsTimestamptrue - Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This - flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. -
    spark.sql.parquet.compression.codecsnappy - Sets the compression codec used when writing Parquet files. If either `compression` or - `parquet.compression` is specified in the table-specific options/properties, the precedence would be - `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: - none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. - Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires - `BrotliCodec` to be installed. -
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue - When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in - support. -
    spark.sql.parquet.mergeSchemafalse -

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

    -
    spark.sql.optimizer.metadataOnlytrue -

    - When true, enable the metadata-only query optimization that use the table's metadata to - produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator that satisfies distinct - semantics. -

    -
    spark.sql.parquet.writeLegacyFormatfalse - If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values - will be written in Apache Parquet's fixed-length byte array format, which other systems such as - Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For - example, decimals will be written in int-based format. If Parquet output is intended for use - with systems that do not support this newer format, set to true. -
    diff --git a/docs/sql-data-sources-troubleshooting.md b/docs/sql-data-sources-troubleshooting.md deleted file mode 100644 index 5775eb8b5c956..0000000000000 --- a/docs/sql-data-sources-troubleshooting.md +++ /dev/null @@ -1,9 +0,0 @@ ---- -layout: global -title: Troubleshooting -displayTitle: Troubleshooting ---- - - * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. - * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. - * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md deleted file mode 100644 index 636636af6263c..0000000000000 --- a/docs/sql-data-sources.md +++ /dev/null @@ -1,42 +0,0 @@ ---- -layout: global -title: Data Sources -displayTitle: Data Sources ---- - - -Spark SQL supports operating on a variety of data sources through the DataFrame interface. -A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. -Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section -describes the general methods for loading and saving data using the Spark Data Sources and then -goes into specific options that are available for the built-in data sources. - - -* [Generic Load/Save Functions](sql-data-sources-load-save-functions.html) - * [Manually Specifying Options](sql-data-sources-load-save-functions.html#manually-specifying-options) - * [Run SQL on files directly](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) - * [Save Modes](sql-data-sources-load-save-functions.html#save-modes) - * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#saving-to-persistent-tables) - * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#bucketing-sorting-and-partitioning) -* [Parquet Files](sql-data-sources-parquet.html) - * [Loading Data Programmatically](sql-data-sources-parquet.html#loading-data-programmatically) - * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) - * [Schema Merging](sql-data-sources-parquet.html#schema-merging) - * [Hive metastore Parquet table conversion](sql-data-sources-parquet.html#hive-metastore-parquet-table-conversion) - * [Configuration](sql-data-sources-parquet.html#configuration) -* [ORC Files](sql-data-sources-orc.html) -* [JSON Files](sql-data-sources-json.html) -* [Hive Tables](sql-data-sources-hive-tables.html) - * [Specifying storage format for Hive tables](sql-data-sources-hive-tables.html#specifying-storage-format-for-hive-tables) - * [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore) -* [JDBC To Other Databases](sql-data-sources-jdbc.html) -* [Avro Files](sql-data-sources-avro.html) - * [Deploying](sql-data-sources-avro.html#deploying) - * [Load and Save Functions](sql-data-sources-avro.html#load-and-save-functions) - * [to_avro() and from_avro()](sql-data-sources-avro.html#to_avro-and-from_avro) - * [Data Source Option](sql-data-sources-avro.html#data-source-option) - * [Configuration](sql-data-sources-avro.html#configuration) - * [Compatibility with Databricks spark-avro](sql-data-sources-avro.html#compatibility-with-databricks-spark-avro) - * [Supported types for Avro -> Spark SQL conversion](sql-data-sources-avro.html#supported-types-for-avro---spark-sql-conversion) - * [Supported types for Spark SQL -> Avro conversion](sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion) -* [Troubleshooting](sql-data-sources-troubleshooting.html) diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md deleted file mode 100644 index 66d6fdaf90a08..0000000000000 --- a/docs/sql-distributed-sql-engine.md +++ /dev/null @@ -1,84 +0,0 @@ ---- -layout: global -title: Distributed SQL Engine -displayTitle: Distributed SQL Engine ---- - -* Table of contents -{:toc} - -Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. -In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, -without the need to write any code. - -## Running the Thrift JDBC/ODBC server - -The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 1.2.1. You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. - -To start the JDBC/ODBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to -specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of -all available options. By default, the server listens on localhost:10000. You may override this -behaviour via either environment variables, i.e.: - -{% highlight bash %} -export HIVE_SERVER2_THRIFT_PORT= -export HIVE_SERVER2_THRIFT_BIND_HOST= -./sbin/start-thriftserver.sh \ - --master \ - ... -{% endhighlight %} - -or system properties: - -{% highlight bash %} -./sbin/start-thriftserver.sh \ - --hiveconf hive.server2.thrift.port= \ - --hiveconf hive.server2.thrift.bind.host= \ - --master - ... -{% endhighlight %} - -Now you can use beeline to test the Thrift JDBC/ODBC server: - - ./bin/beeline - -Connect to the JDBC/ODBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. - -You may also use the beeline script that comes with Hive. - -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - - hive.server2.transport.mode - Set this to value: http - hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 - hive.server2.http.endpoint - HTTP endpoint; default is cliservice - -To test, use beeline to connect to the JDBC/ODBC server in http mode with: - - beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= - - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available options. diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md deleted file mode 100644 index 88512205894ab..0000000000000 --- a/docs/sql-getting-started.md +++ /dev/null @@ -1,369 +0,0 @@ ---- -layout: global -title: Getting Started -displayTitle: Getting Started ---- - -* Table of contents -{:toc} - -## Starting Point: SparkSession - -
    -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: - -{% include_example init_session python/sql/basic.py %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: - -{% include_example init_session r/RSparkSQLExample.R %} - -Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. -
    -
    - -`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to -write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. -To use these features, you do not need to have an existing Hive setup. - -## Creating DataFrames - -
    -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](sql-data-sources.html). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](sql-data-sources.html). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](sql-data-sources.html). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df python/sql/basic.py %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from a local R data.frame, -from a Hive table, or from [Spark data sources](sql-data-sources.html). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df r/RSparkSQLExample.R %} - -
    -
    - - -## Untyped Dataset Operations (aka DataFrame Operations) - -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). - -As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. - -Here we include some basic examples of structured data processing using Datasets: - -
    -
    -{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). -
    - -
    - -{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). -
    - -
    -In Python, it's possible to access a DataFrame's columns either by attribute -(`df.age`) or by indexing (`df['age']`). While the former is convenient for -interactive data exploration, users are highly encouraged to use the -latter form, which is future proof and won't break with column names that -are also attributes on the DataFrame class. - -{% include_example untyped_ops python/sql/basic.py %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). - -
    - -
    - -{% include_example untyped_ops r/RSparkSQLExample.R %} - -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). - -
    - -
    - -## Running SQL Queries Programmatically - -
    -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. - -{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql python/sql/basic.py %} -
    - -
    -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. - -{% include_example run_sql r/RSparkSQLExample.R %} - -
    -
    - - -## Global Temporary View - -Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it -terminates. If you want to have a temporary view that is shared among all sessions and keep alive -until the Spark application terminates, you can create a global temporary view. Global temporary -view is tied to a system preserved database `global_temp`, and we must use the qualified name to -refer it, e.g. `SELECT * FROM global_temp.view1`. - -
    -
    -{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -{% include_example global_temp_view python/sql/basic.py %} -
    - -
    - -{% highlight sql %} - -CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl - -SELECT * FROM global_temp.temp_view - -{% endhighlight %} - -
    -
    - - -## Creating Datasets - -Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use -a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects -for processing or transmitting over the network. While both encoders and standard serialization are -responsible for turning an object into bytes, encoders are code generated dynamically and use a format -that allows Spark to perform many operations like filtering, sorting and hashing without deserializing -the bytes back into an object. - -
    -
    -{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    -
    - -## Interoperating with RDDs - -Spark SQL supports two different methods for converting existing RDDs into Datasets. The first -method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection-based approach leads to more concise code and works well when you already know the schema -while writing your Spark application. - -The second method for creating Datasets is through a programmatic interface that allows you to -construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows -you to construct Datasets when the columns and their types are not known until runtime. - -### Inferring the Schema Using Reflection -
    - -
    - -The Scala interface for Spark SQL supports automatically converting an RDD containing case classes -to a DataFrame. The case class -defines the schema of the table. The names of the arguments to the case class are read using -reflection and become the names of the columns. Case classes can also be nested or contain complex -types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be -registered as a table. Tables can be used in subsequent SQL statements. - -{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -Spark SQL supports automatically converting an RDD of -[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. -The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL -does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` -fields are supported though. You can create a JavaBean by creating a class that implements -Serializable and has getters and setters for all of its fields. - -{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of -key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, -and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. - -{% include_example schema_inferring python/sql/basic.py %} -
    - -
    - -### Programmatically Specifying the Schema - -
    - -
    - -When case classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed -and fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -When JavaBean classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `Dataset` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -When a dictionary of kwargs cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of tuples or lists from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. - -For example: - -{% include_example programmatic_schema python/sql/basic.py %} -
    - -
    - -## Aggregations - -The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common -aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. -While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in -[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and -[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. -Moreover, users are not limited to the predefined aggregate functions and can create their own. - -### Untyped User-Defined Aggregate Functions -Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) -abstract class to implement a custom untyped aggregate function. For example, a user-defined average -can look like: - -
    -
    -{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} -
    -
    -{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} -
    -
    - -### Type-Safe User-Defined Aggregate Functions - -User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. -For example, a type-safe user-defined average can look like: - -
    -
    -{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} -
    -
    -{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} -
    -
    diff --git a/docs/sql-migration-guide-hive-compatibility.md b/docs/sql-migration-guide-hive-compatibility.md deleted file mode 100644 index 94849418030ef..0000000000000 --- a/docs/sql-migration-guide-hive-compatibility.md +++ /dev/null @@ -1,152 +0,0 @@ ---- -layout: global -title: Compatibility with Apache Hive -displayTitle: Compatibility with Apache Hive ---- - -* Table of contents -{:toc} - -Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, -and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). - -#### Deploying in Existing Hive Warehouses - -The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY` - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathematical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDes) -* Window functions -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables including dynamic partition insertion -* View - * If column aliases are not specified in view definition queries, both Spark and Hive will - generate alias names, but in different ways. In order for Spark to be able to read views created - by Hive, users should explicitly specify column aliases in view definition queries. As an - example, Spark cannot read `v1` created as below by Hive. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; - ``` - - Instead, you should create `v1` as below with column aliases explicitly specified. - - ``` - CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; - ``` - -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `DATE` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - - -**Esoteric Hive Features** - -* `UNION` type -* Unique join -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment and only supports populating the sizeInBytes field of the hive metastore. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -less important due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block-level bitmap indexes and virtual columns (used to build indexes) -* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". -* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -**Hive UDF/UDTF/UDAF** - -Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: - -* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically - include additional resources required by this UDF. -* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses - a deprecated interface `initialize(ObjectInspector[])` only. -* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize - functions with `MapredContext`, which is inapplicable to Spark. -* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. - Spark SQL does not call this function when tasks finish. -* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. - Spark SQL currently does not support the reuse of aggregation. -* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating - an aggregate over a fixed window. - -### Incompatible Hive UDF - -Below are the scenarios in which Hive and Spark generate different results: - -* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. -* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. -* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md deleted file mode 100644 index c9685b866774f..0000000000000 --- a/docs/sql-migration-guide-upgrade.md +++ /dev/null @@ -1,582 +0,0 @@ ---- -layout: global -title: Spark SQL Upgrading Guide -displayTitle: Spark SQL Upgrading Guide ---- - -* Table of contents -{:toc} - -## Upgrading From Spark SQL 2.4 to 3.0 - - - In PySpark, when creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 3.0, the builder comes to not update the configurations. This is the same behavior as Java/Scala API in 2.3 and above. If you want to update them, you need to update them prior to creating a `SparkSession`. - - - In Spark version 2.4 and earlier, the parser of JSON data source treats empty strings as null for some data types such as `IntegerType`. For `FloatType` and `DoubleType`, it fails on empty strings and throws exceptions. Since Spark 3.0, we disallow empty strings and will throw exceptions for data types except for `StringType` and `BinaryType`. - - - Since Spark 3.0, the `from_json` functions supports two modes - `PERMISSIVE` and `FAILFAST`. The modes can be set via the `mode` option. The default mode became `PERMISSIVE`. In previous versions, behavior of `from_json` did not conform to either `PERMISSIVE` nor `FAILFAST`, especially in processing of malformed JSON records. For example, the JSON string `{"a" 1}` with the schema `a INT` is converted to `null` by previous versions but Spark 3.0 converts it to `Row(null)`. - - - The `ADD JAR` command previously returned a result set with the single value 0. It now returns an empty result set. - -## Upgrading From Spark SQL 2.3 to 2.4 - - - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Query - - Result Spark 2.3 or Prior - - Result Spark 2.4 - - Remarks -
    - SELECT
    array_contains(array(1), 1.34D);
    -
    - true - - false - - In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. -
    - SELECT
    array_contains(array(1), '1');
    -
    - true - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explicit cast -
    - SELECT
    array_contains(array(1), 'anystring');
    -
    - null - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explicit cast -
    - - - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - - - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - - - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - - - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - - - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. - - - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - - - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - - - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - - - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - - - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - - - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - - - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - - - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - - - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - - - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - - - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. - - - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. - -## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - - - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. - -## Upgrading From Spark SQL 2.2 to 2.3 - - - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. - - - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - - - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - - - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
    - NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    - IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    - LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    - DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    - TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    - StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    - - Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - - - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. - - - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - - - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. - - - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - - - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - - - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). - - - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. - - - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. - - - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - - - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - - - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. - -## Upgrading From Spark SQL 2.1 to 2.2 - - - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. - - - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). - - - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. - -## Upgrading From Spark SQL 2.0 to 2.1 - - - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - - - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. - - - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. - - - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. - - - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. - -## Upgrading From Spark SQL 1.6 to 2.0 - - - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - - - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for - `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., - `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ - APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the - single-node data frame notion in these languages. - - - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` - - - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` - - - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - - - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. - - - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` - in order to prevent accidental dropping the existing data in the user-provided locations. - That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. - Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. - Note that this is different from the Hive behavior. - - - As a result, `DROP TABLE` statements on those tables will not remove the data. - - - `spark.sql.parquet.cacheMetadata` is no longer used. - See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. - -## Upgrading From Spark SQL 1.5 to 1.6 - - - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC - connection owns a copy of their own SQL configuration and temporary function registry. Cached - tables are still shared though. If you prefer to run the Thrift server in the old single-session - mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add - this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: - - {% highlight bash %} - ./sbin/start-thriftserver.sh \ - --conf spark.sql.hive.thriftServer.singleSession=true \ - ... - {% endhighlight %} - - - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns - of the same name of a DataFrame. - - - 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 - code generation for expression evaluation. These features can both be disabled by setting - `spark.sql.tungsten.enabled` to `false`. - - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting - `spark.sql.parquet.mergeSchema` to `true`. - - - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - - - In-memory columnar storage partition pruning is on by default. It can be disabled by setting - `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - - - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now - used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - - - Timestamps are now stored at a precision of 1us, rather than 1ns - - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains - unchanged. - - - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - - - JSON data source will not automatically load new files that are created by other applications - (i.e. files that are not inserted to the dataset through Spark SQL). - For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), - users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method - to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate - the DataFrame and the new DataFrame will include new files. - - - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. - -## Upgrading from Spark SQL 1.3 to 1.4 - -#### DataFrame data reader/writer interface - -Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) -and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). - -See the API docs for `SQLContext.read` ( - Scala, - Java, - Python -) and `DataFrame.write` ( - Scala, - Java, - Python -) more information. - - -#### DataFrame.groupBy retains grouping columns - -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the -grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. - -
    -
    -{% highlight scala %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg($"department", max("age"), sum("expense")) - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")) - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    -{% highlight java %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(col("department"), max("age"), sum("expense")); - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")); - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false"); - -{% endhighlight %} -
    - -
    -{% highlight python %} - -import pyspark.sql.functions as func - -# In 1.3.x, in order for the grouping column "department" to show up, -# it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) - -# In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(func.max("age"), func.sum("expense")) - -# Revert to 1.3.x behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    - - -#### Behavior change on DataFrame.withColumn - -Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added -as a new column with its specified name in the result DataFrame even if there may be any existing -columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different -name from names of all existing columns or replacing existing columns of the same name. - -Note that this change is only for Scala API, not for PySpark and SparkR. - - -## Upgrading from Spark SQL 1.0-1.2 to 1.3 - -In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked -as unstable (i.e., DeveloperAPI or Experimental). - -#### Rename of SchemaRDD to DataFrame - -The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD -directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. - -In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. -Java and Python users will need to update their code. - -#### Unification of the Java and Scala APIs - -Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general these classes try to -use types that are usable from both languages (i.e. `Array` instead of language-specific collections). -In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading -is used instead. - -Additionally, the Java specific types API has been removed. Users of both Scala and Java should -use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. - - -#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) - -Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit -conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. -Users should now write `import sqlContext.implicits._`. - -Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., -case classes or tuples) with a method `toDF`, instead of applying automatically. - -When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: -`import org.apache.spark.sql.functions._`. - -#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) - -Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users -should instead import the classes in `org.apache.spark.sql.types` - -#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) - -Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been -moved into the udf object in `SQLContext`. - -
    -
    -{% highlight scala %} - -sqlContext.udf.register("strLen", (s: String) => s.length()) - -{% endhighlight %} -
    - -
    -{% highlight java %} - -sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); - -{% endhighlight %} -
    - -
    - -Python UDF registration is unchanged. - -#### Python DataTypes No Longer Singletons - -When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of -referencing a singleton. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md deleted file mode 100644 index 71d83e8a5570b..0000000000000 --- a/docs/sql-migration-guide.md +++ /dev/null @@ -1,23 +0,0 @@ ---- -layout: global -title: Migration Guide -displayTitle: Migration Guide ---- - -* [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html) - * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) - * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) - * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) - * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) - * [Upgrading From Spark SQL 2.1 to 2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22) - * [Upgrading From Spark SQL 2.0 to 2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21) - * [Upgrading From Spark SQL 1.6 to 2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20) - * [Upgrading From Spark SQL 1.5 to 1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16) - * [Upgrading From Spark SQL 1.4 to 1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15) - * [Upgrading from Spark SQL 1.3 to 1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14) - * [Upgrading from Spark SQL 1.0-1.2 to 1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13) -* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html) - * [Deploying in Existing Hive Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses) - * [Supported Hive Features](sql-migration-guide-hive-compatibility.html#supported-hive-features) - * [Unsupported Hive Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality) - * [Incompatible Hive UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf) diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md deleted file mode 100644 index 7c7c4a815545f..0000000000000 --- a/docs/sql-performance-tuning.md +++ /dev/null @@ -1,151 +0,0 @@ ---- -layout: global -title: Performance Tuning -displayTitle: Performance Tuning ---- - -* Table of contents -{:toc} - -For some workloads, it is possible to improve performance by either caching data in memory, or by -turning on some experimental options. - -## Caching Data In Memory - -Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. -Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. - -Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue - When set to true Spark SQL will automatically select a compression codec for each column based - on statistics of the data. -
    spark.sql.inMemoryColumnarStorage.batchSize10000 - Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization - and compression, but risk OOMs when caching data. -
    - -## Other Configuration Options - -The following options can also be used to tune the performance of query execution. It is possible -that these options will be deprecated in future release as more optimizations are performed automatically. - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) - The maximum number of bytes to pack into a single partition when reading files. -
    spark.sql.files.openCostInBytes4194304 (4 MB) - The estimated cost to open a file, measured by the number of bytes could be scanned in the same - time. This is used when putting multiple files into a partition. It is better to over-estimated, - then the partitions with small files will be faster than partitions with bigger files (which is - scheduled first). -
    spark.sql.broadcastTimeout300 -

    - Timeout in seconds for the broadcast wait time in broadcast joins -

    -
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) - Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when - performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently - statistics are only supported for Hive Metastore tables where the command - ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. -
    spark.sql.shuffle.partitions200 - Configures the number of partitions to use when shuffling data for joins or aggregations. -
    - -## Broadcast Hint for SQL Queries - -The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. -When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, -even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. -When both sides of a join are specified, Spark broadcasts the one having the lower statistics. -Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) -support BHJ. When the broadcast nested loop join is selected, we still respect the hint. - -
    - -
    - -{% highlight scala %} -import org.apache.spark.sql.functions.broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight java %} -import static org.apache.spark.sql.functions.broadcast; -broadcast(spark.table("src")).join(spark.table("records"), "key").show(); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -from pyspark.sql.functions import broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight r %} -src <- sql("SELECT * FROM src") -records <- sql("SELECT * FROM records") -head(join(broadcast(src), records, src$key == records$key)) -{% endhighlight %} - -
    - -
    - -{% highlight sql %} --- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint -SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key -{% endhighlight %} - -
    -
    diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index eca8915dfa975..a1d7b1108bf73 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -4,6 +4,11 @@ displayTitle: Spark SQL, DataFrames and Datasets Guide title: Spark SQL and DataFrames --- +* This will become a table of contents (this text will be scraped). +{:toc} + +# Overview + Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to @@ -19,17 +24,17 @@ the `spark-shell`, `pyspark` shell, or `sparkR` shell. One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to -configure this feature, please refer to the [Hive Tables](sql-data-sources-hive-tables.html) section. When running +configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). -You can also interact with the SQL interface using the [command-line](sql-distributed-sql-engine.html#running-the-spark-sql-cli) -or over [JDBC/ODBC](sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server). +You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) +or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). ## Datasets and DataFrames A Dataset is a distributed collection of data. Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized -execution engine. A Dataset can be [constructed](sql-getting-started.html#creating-datasets) from JVM objects and then +execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). The Dataset API is available in [Scala][scala-datasets] and [Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature, @@ -38,7 +43,7 @@ many of the benefits of the Dataset API are already available (i.e. you can acce A DataFrame is a *Dataset* organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of [sources](sql-data-sources.html) such +optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such as: structured data files, tables in Hive, external databases, or existing RDDs. The DataFrame API is available in Scala, Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). @@ -50,3 +55,3108 @@ While, in [Java API][java-datasets], users need to use `Dataset` to represe [java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. + +# Getting Started + +## Starting Point: SparkSession + +
    +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: + +{% include_example init_session python/sql/basic.py %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: + +{% include_example init_session r/RSparkSQLExample.R %} + +Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. +
    +
    + +`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to +write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. +To use these features, you do not need to have an existing Hive setup. + +## Creating DataFrames + +
    +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df python/sql/basic.py %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from a local R data.frame, +from a Hive table, or from [Spark data sources](#data-sources). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df r/RSparkSQLExample.R %} + +
    +
    + + +## Untyped Dataset Operations (aka DataFrame Operations) + +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). + +As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. + +Here we include some basic examples of structured data processing using Datasets: + +
    +
    +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). +
    + +
    + +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} + +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +
    + +
    +In Python, it's possible to access a DataFrame's columns either by attribute +(`df.age`) or by indexing (`df['age']`). While the former is convenient for +interactive data exploration, users are highly encouraged to use the +latter form, which is future proof and won't break with column names that +are also attributes on the DataFrame class. + +{% include_example untyped_ops python/sql/basic.py %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). + +
    + +
    + +{% include_example untyped_ops r/RSparkSQLExample.R %} + +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). + +
    + +
    + +## Running SQL Queries Programmatically + +
    +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. + +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql python/sql/basic.py %} +
    + +
    +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +{% include_example run_sql r/RSparkSQLExample.R %} + +
    +
    + + +## Global Temporary View + +Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it +terminates. If you want to have a temporary view that is shared among all sessions and keep alive +until the Spark application terminates, you can create a global temporary view. Global temporary +view is tied to a system preserved database `global_temp`, and we must use the qualified name to +refer it, e.g. `SELECT * FROM global_temp.view1`. + +
    +
    +{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +{% include_example global_temp_view python/sql/basic.py %} +
    + +
    + +{% highlight sql %} + +CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl + +SELECT * FROM global_temp.temp_view + +{% endhighlight %} + +
    +
    + + +## Creating Datasets + +Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use +a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects +for processing or transmitting over the network. While both encoders and standard serialization are +responsible for turning an object into bytes, encoders are code generated dynamically and use a format +that allows Spark to perform many operations like filtering, sorting and hashing without deserializing +the bytes back into an object. + +
    +
    +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    +
    + +## Interoperating with RDDs + +Spark SQL supports two different methods for converting existing RDDs into Datasets. The first +method uses reflection to infer the schema of an RDD that contains specific types of objects. This +reflection-based approach leads to more concise code and works well when you already know the schema +while writing your Spark application. + +The second method for creating Datasets is through a programmatic interface that allows you to +construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows +you to construct Datasets when the columns and their types are not known until runtime. + +### Inferring the Schema Using Reflection +
    + +
    + +The Scala interface for Spark SQL supports automatically converting an RDD containing case classes +to a DataFrame. The case class +defines the schema of the table. The names of the arguments to the case class are read using +reflection and become the names of the columns. Case classes can also be nested or contain complex +types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be +registered as a table. Tables can be used in subsequent SQL statements. + +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +Spark SQL supports automatically converting an RDD of +[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. +The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL +does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` +fields are supported though. You can create a JavaBean by creating a class that implements +Serializable and has getters and setters for all of its fields. + +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, +and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. + +{% include_example schema_inferring python/sql/basic.py %} +
    + +
    + +### Programmatically Specifying the Schema + +
    + +
    + +When case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed +and fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +When JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `Dataset` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +When a dictionary of kwargs cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of tuples or lists from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +tuples or lists in the RDD created in the step 1. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. + +For example: + +{% include_example programmatic_schema python/sql/basic.py %} +
    + +
    + +## Aggregations + +The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common +aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. +While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in +[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and +[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. +Moreover, users are not limited to the predefined aggregate functions and can create their own. + +### Untyped User-Defined Aggregate Functions +Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) +abstract class to implement a custom untyped aggregate function. For example, a user-defined average +can look like: + +
    +
    +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +
    +
    +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +
    +
    + +### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like: + +
    +
    +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +
    +
    +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} +
    +
    + +# Data Sources + +Spark SQL supports operating on a variety of data sources through the DataFrame interface. +A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. +Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section +describes the general methods for loading and saving data using the Spark Data Sources and then +goes into specific options that are available for the built-in data sources. + +## Generic Load/Save Functions + +In the simplest form, the default data source (`parquet` unless otherwise configured by +`spark.sql.sources.default`) will be used for all operations. + +
    +
    +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example generic_load_save_functions python/sql/datasource.py %} +
    + +
    + +{% include_example generic_load_save_functions r/RSparkSQLExample.R %} + +
    +
    + +### Manually Specifying Options + +You can also manually specify the data source that will be used along with any extra options +that you would like to pass to the data source. Data sources are specified by their fully qualified +name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax. + +To load a JSON file you can use: + +
    +
    +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options r/RSparkSQLExample.R %} +
    +
    + +To load a CSV file you can use: + +
    +
    +{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options_csv python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options_csv r/RSparkSQLExample.R %} + +
    +
    + +### Run SQL on files directly + +Instead of using read API to load a file into DataFrame and query it, you can also query that +file directly with SQL. + +
    +
    +{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example direct_sql python/sql/datasource.py %} +
    + +
    +{% include_example direct_sql r/RSparkSQLExample.R %} + +
    +
    + +### Save Modes + +Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if +present. It is important to realize that these save modes do not utilize any locking and are not +atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the +new data. + + + + + + + + + + + + + + + + + + + + + + + +
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) + When saving a DataFrame to a data source, if data already exists, + an exception is expected to be thrown. +
    SaveMode.Append"append" + When saving a DataFrame to a data source, if data/table already exists, + contents of the DataFrame are expected to be appended to existing data. +
    SaveMode.Overwrite"overwrite" + Overwrite mode means that when saving a DataFrame to a data source, + if data/table already exists, existing data is expected to be overwritten by the contents of + the DataFrame. +
    SaveMode.Ignore"ignore" + Ignore mode means that when saving a DataFrame to a data source, if data already exists, + the save operation is expected to not save the contents of the DataFrame and to not + change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. +
    + +### Saving to Persistent Tables + +`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` +command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a +default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, +`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the +Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as +long as you maintain your connection to the same metastore. A DataFrame for a persistent table can +be created by calling the `table` method on a `SparkSession` with the name of the table. + +For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the +`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, +the custom table path will not be removed and the table data is still there. If no custom table path is +specified, Spark will write data to a default table path under the warehouse directory. When the table is +dropped, the default table path will be removed too. + +Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: + +- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. +- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + +Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. + +### Bucketing, Sorting and Partitioning + +For file-based data source, it is also possible to bucket and sort or partition the output. +Bucketing and sorting are applicable only to persistent tables: + +
    + +
    +{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_sorting_and_bucketing python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_by_name( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +CLUSTERED BY(name) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. + + +
    + +
    +{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partitioning python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_by_favorite_color( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING csv PARTITIONED BY(favorite_color); + +{% endhighlight %} + +
    + +
    + +It is possible to use both partitioning and bucketing for a single table: + +
    + +
    +{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partition_and_bucket python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_and_partitioned( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +PARTITIONED BY (favorite_color) +CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +`partitionBy` creates a directory structure as described in the [Partition Discovery](#partition-discovery) section. +Thus, it has limited applicability to columns with high cardinality. In contrast + `bucketBy` distributes +data across a fixed number of buckets and can be used when a number of unique values is unbounded. + +## Parquet Files + +[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. +Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema +of the original data. When writing Parquet files, all columns are automatically converted to be nullable for +compatibility reasons. + +### Loading Data Programmatically + +Using the data from the above example: + +
    + +
    +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example basic_parquet_example python/sql/datasource.py %} +
    + +
    + +{% include_example basic_parquet_example r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW parquetTable +USING org.apache.spark.sql.parquet +OPTIONS ( + path "examples/src/main/resources/people.parquet" +) + +SELECT * FROM parquetTable + +{% endhighlight %} + +
    + +
    + +### Partition Discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) +are able to discover and infer partitioning information automatically. +For example, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL +will automatically extract the partitioning information from the paths. +Now the schema of the returned DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types, date, timestamp and string type are supported. Sometimes users may not want +to automatically infer the data types of the partitioning columns. For these use cases, the +automatic type inference can be configured by +`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type +inference is disabled, string type will be used for the partitioning columns. + +Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths +by default. For the above example, if users pass `path/to/table/gender=male` to either +`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a +partitioning column. If users need to specify the base path that partition discovery +should start with, they can set `basePath` in the data source options. For example, +when `path/to/table/gender=male` is the path of the data and +users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. + +### Schema Merging + +Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default starting from 1.5.0. You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the + examples below), or +2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. + +
    + +
    +{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example schema_merging python/sql/datasource.py %} +
    + +
    + +{% include_example schema_merging r/RSparkSQLExample.R %} + +
    + +
    + +### Hive metastore Parquet table conversion + +When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own +Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the +`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default. + +#### Hive/Parquet Schema Reconciliation + +There are two key differences between Hive and Parquet from the perspective of table schema +processing. + +1. Hive is case insensitive, while Parquet is not +1. Hive considers all columns nullable, while nullability in Parquet is significant + +Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a +Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: + +1. Fields that have the same name in both schema must have the same data type regardless of + nullability. The reconciled field should have the data type of the Parquet side, so that + nullability is respected. + +1. The reconciled schema contains exactly those fields defined in Hive metastore schema. + + - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. + - Any fields that only appear in the Hive metastore schema are added as nullable field in the + reconciled schema. + +#### Metadata Refreshing + +Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table +conversion is enabled, metadata of those converted tables are also cached. If these tables are +updated by Hive or other external tools, you need to refresh them manually to ensure consistent +metadata. + +
    + +
    + +{% highlight scala %} +// spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight java %} +// spark is an existing SparkSession +spark.catalog().refreshTable("my_table"); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +# spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight r %} +refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +REFRESH TABLE my_table; +{% endhighlight %} + +
    + +
    + +### Configuration + +Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse + Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This + flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. +
    spark.sql.parquet.int96AsTimestamptrue + Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This + flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. +
    spark.sql.parquet.compression.codecsnappy + Sets the compression codec used when writing Parquet files. If either `compression` or + `parquet.compression` is specified in the table-specific options/properties, the precedence would be + `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: + none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. + Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires + `BrotliCodec` to be installed. +
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue + When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in + support. +
    spark.sql.parquet.mergeSchemafalse +

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

    +
    spark.sql.optimizer.metadataOnlytrue +

    + When true, enable the metadata-only query optimization that use the table's metadata to + produce the partition columns instead of table scans. It applies when all the columns scanned + are partition columns and the query has an aggregate operator that satisfies distinct + semantics. +

    +
    spark.sql.parquet.writeLegacyFormatfalse + If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values + will be written in Apache Parquet's fixed-length byte array format, which other systems such as + Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For + example, decimals will be written in int-based format. If Parquet output is intended for use + with systems that do not support this newer format, set to true. +
    + +## ORC Files + +Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. +To do that, the following configurations are newly added. The vectorized reader is used for the +native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` +is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC +serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), +the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables vectorized orc decoding in native implementation. If false, a new non-vectorized ORC reader is used in native implementation. For hive implementation, this is ignored.
    + +## JSON Datasets +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. +This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. +This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. +This conversion can be done using `SparkSession.read.json` on a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` parameter to `True`. + +{% include_example json_dataset python/sql/datasource.py %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using +the `read.json()` function, which loads data from a directory of JSON files where each line of the +files is a JSON object. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. + +{% include_example json_dataset r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jsonTable +USING org.apache.spark.sql.json +OPTIONS ( + path "examples/src/main/resources/people.json" +) + +SELECT * FROM jsonTable + +{% endhighlight %} + +
    + +
    + +## Hive Tables + +Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). +However, since Hive has a large number of dependencies, these dependencies are not included in the +default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them +automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as +they will need access to the Hive serialization and deserialization libraries (SerDes) in order to +access data stored in Hive. + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), +and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. + +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the Spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the Spark application. + +
    + +
    +{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} +
    + +
    +{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} +
    + +
    +{% include_example spark_hive python/sql/hive.py %} +
    + +
    + +When working with Hive one must instantiate `SparkSession` with Hive support. This +adds support for finding tables in the MetaStore and writing queries using HiveQL. + +{% include_example spark_hive r/RSparkSQLExample.R %} + +
    +
    + +### Specifying storage format for Hive tables + +When you create a Hive table, you need to define how this table should read/write data from/to file system, +i.e. the "input format" and "output format". You also need to define how this table should deserialize the data +to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage +format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. +By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when +creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    fileFormat + A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and + "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. +
    inputFormat, outputFormat + These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, + e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not + specify them if you already specified the `fileFormat` option. +
    serde + This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option + if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" + don't include the serde information and you can use this option with these 3 fileFormats. +
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim + These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. +
    + +All other properties defined with `OPTIONS` will be regarded as Hive serde properties. + +### Interacting with Different Versions of Hive Metastore + +One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, +which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary +build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. +Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL +will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). + +The following options can be used to configure the version of Hive that is used to retrieve metadata: + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 + Version of the Hive metastore. Available + options are 0.12.0 through 2.3.3. +
    spark.sql.hive.metastore.jarsbuiltin + Location of the jars that should be used to instantiate the HiveMetastoreClient. This + property can be one of three options: +
      +
    1. builtin
    2. + Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is + enabled. When this option is chosen, spark.sql.hive.metastore.version must be + either 1.2.1 or not defined. +
    3. maven
    4. + Use Hive jars of specified version downloaded from Maven repositories. This configuration + is not generally recommended for production deployments. +
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive + and its dependencies, including the correct version of Hadoop. These jars only need to be + present on the driver, but if you are running in yarn cluster mode then you must ensure + they are packaged with your application.
    6. +
    +
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    +

    + A comma-separated list of class prefixes that should be loaded using the classloader that is + shared between Spark SQL and a specific version of Hive. An example of classes that should + be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need + to be shared are those that interact with classes that are already shared. For example, + custom appenders that are used by log4j. +

    +
    spark.sql.hive.metastore.barrierPrefixes(empty) +

    + A comma separated list of class prefixes that should explicitly be reloaded for each version + of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + prefix that typically would be shared (i.e. org.apache.spark.*). +

    +
    + + +## JDBC To Other Databases + +Spark SQL also includes a data source that can read data from other databases using JDBC. This +functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). +This is because the results are returned +as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. +The JDBC data source is also easier to use from Java or Python as it does not require the user to +provide a ClassTag. +(Note that this is different than the Spark SQL JDBC server, which allows other applications to +run queries using Spark SQL). + +To get started you will need to include the JDBC driver for your particular database on the +spark classpath. For example, to connect to postgres from the Spark Shell you would run the +following command: + +{% highlight bash %} +bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar +{% endhighlight %} + +Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using +the Data Sources API. Users can specify the JDBC connection properties in the data source options. +user and password are normally provided as connection properties for +logging into the data sources. In addition to the connection properties, Spark also supports +the following case-insensitive options: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    url + The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret +
    dbtable + The JDBC table that should be read from or written into. Note that when using it in the read + path anything that is valid in a FROM clause of a SQL query can be used. + For example, instead of a full table you could also use a subquery in parentheses. It is not + allowed to specify `dbtable` and `query` options at the same time. +
    query + A query that will be used to read data into Spark. The specified query will be parenthesized and used + as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. + As an example, spark will issue a query of the following form to the JDBC Source.

    + SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    + Below are couple of restrictions while using this option.
    +
      +
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. +
    3. It is not allowed to spcify `query` and `partitionColumn` options at the same time. When specifying + `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and + partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      + Example:
      + + spark.read.format("jdbc")
      +    .option("dbtable", "(select c1, c2 from t1) as subq")
      +    .option("partitionColumn", "subq.c1"
      +    .load() +
    4. +
    +
    driver + The class name of the JDBC driver to use to connect to this URL. +
    partitionColumn, lowerBound, upperBound + These options must all be specified if any of them is specified. In addition, + numPartitions must be specified. They describe how to partition the table when + reading in parallel from multiple workers. + partitionColumn must be a numeric, date, or timestamp column from the table in question. + Notice that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. This option applies only to reading. +
    numPartitions + The maximum number of partitions that can be used for parallelism in table reading and + writing. This also determines the maximum number of concurrent JDBC connections. + If the number of partitions to write exceeds this limit, we decrease it to this limit by + calling coalesce(numPartitions) before writing. +
    queryTimeout + The number of seconds the driver will wait for a Statement object to execute to the given + number of seconds. Zero means there is no limit. In the write path, this option depends on + how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver + checks the timeout of each query instead of an entire JDBC batch. + It defaults to 0. +
    fetchsize + The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. +
    batchsize + The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. +
    isolationLevel + The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. +
    sessionInitStatement + After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") +
    truncate + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. +
    cascadeTruncate + This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. +
    createTableOptions + This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. +
    createTableColumnTypes + The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. +
    customSchema + The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. +
    pushDownPredicate + The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. +
    + +
    + +
    +{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example jdbc_dataset python/sql/datasource.py %} +
    + +
    +{% include_example jdbc_dataset r/RSparkSQLExample.R %} +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jdbcTable +USING org.apache.spark.sql.jdbc +OPTIONS ( + url "jdbc:postgresql:dbserver", + dbtable "schema.tablename", + user 'username', + password 'password' +) + +INSERT INTO TABLE jdbcTable +SELECT * FROM resultTable +{% endhighlight %} + +
    +
    + +## Avro Files +See the [Apache Avro Data Source Guide](avro-data-source-guide.html). + +## Troubleshooting + + * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. + * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. + * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. + +# Performance Tuning + +For some workloads, it is possible to improve performance by either caching data in memory, or by +turning on some experimental options. + +## Caching Data In Memory + +Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. + +Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue + When set to true Spark SQL will automatically select a compression codec for each column based + on statistics of the data. +
    spark.sql.inMemoryColumnarStorage.batchSize10000 + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + and compression, but risk OOMs when caching data. +
    + +## Other Configuration Options + +The following options can also be used to tune the performance of query execution. It is possible +that these options will be deprecated in future release as more optimizations are performed automatically. + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.sql.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over estimated, + then the partitions with small files will be faster than partitions with bigger files (which is + scheduled first). +
    spark.sql.broadcastTimeout300 +

    + Timeout in seconds for the broadcast wait time in broadcast joins +

    +
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) + Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + statistics are only supported for Hive Metastore tables where the command + ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. +
    spark.sql.shuffle.partitions200 + Configures the number of partitions to use when shuffling data for joins or aggregations. +
    + +## Broadcast Hint for SQL Queries + +The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. +When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, +even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. +When both sides of a join are specified, Spark broadcasts the one having the lower statistics. +Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) +support BHJ. When the broadcast nested loop join is selected, we still respect the hint. + +
    + +
    + +{% highlight scala %} +import org.apache.spark.sql.functions.broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight java %} +import static org.apache.spark.sql.functions.broadcast; +broadcast(spark.table("src")).join(spark.table("records"), "key").show(); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +from pyspark.sql.functions import broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight r %} +src <- sql("SELECT * FROM src") +records <- sql("SELECT * FROM records") +head(join(broadcast(src), records, src$key == records$key)) +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint +SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key +{% endhighlight %} + +
    +
    + +# Distributed SQL Engine + +Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. +In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, +without the need to write any code. + +## Running the Thrift JDBC/ODBC server + +The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +in Hive 1.2.1 You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. + +To start the JDBC/ODBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to +specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of +all available options. By default, the server listens on localhost:10000. You may override this +behaviour via either environment variables, i.e.: + +{% highlight bash %} +export HIVE_SERVER2_THRIFT_PORT= +export HIVE_SERVER2_THRIFT_BIND_HOST= +./sbin/start-thriftserver.sh \ + --master \ + ... +{% endhighlight %} + +or system properties: + +{% highlight bash %} +./sbin/start-thriftserver.sh \ + --hiveconf hive.server2.thrift.port= \ + --hiveconf hive.server2.thrift.bind.host= \ + --master + ... +{% endhighlight %} + +Now you can use beeline to test the Thrift JDBC/ODBC server: + + ./bin/beeline + +Connect to the JDBC/ODBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. + +You may also use the beeline script that comes with Hive. + +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: + + hive.server2.transport.mode - Set this to value: http + hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 + hive.server2.http.endpoint - HTTP endpoint; default is cliservice + +To test, use beeline to connect to the JDBC/ODBC server in http mode with: + + beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= + + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. + +# PySpark Usage Guide for Pandas with Apache Arrow + +## Apache Arrow in Spark + +Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer +data between JVM and Python processes. This currently is most beneficial to Python users that +work with Pandas/NumPy data. Its usage is not automatic and might require some minor +changes to configuration or code to take full advantage and ensure compatibility. This guide will +give a high-level description of how to use Arrow in Spark and highlight any differences when +working with Arrow-enabled data. + +### Ensure PyArrow Installed + +If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the +SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow +is installed and available on all cluster nodes. The current supported version is 0.8.0. +You can install using pip or conda from the conda-forge channel. See PyArrow +[installation](https://arrow.apache.org/docs/python/install.html) for details. + +## Enabling for Conversion to/from Pandas + +Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame +using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with +`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set +the Spark configuration 'spark.sql.execution.arrow.enabled' to 'true'. This is disabled by default. + +In addition, optimizations enabled by 'spark.sql.execution.arrow.enabled' could fallback automatically +to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. +This can be controlled by 'spark.sql.execution.arrow.fallback.enabled'. + +
    +
    +{% include_example dataframe_with_arrow python/sql/arrow.py %} +
    +
    + +Using the above optimizations with Arrow will produce the same results as when Arrow is not +enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the +DataFrame to the driver program and should be done on a small subset of the data. Not all Spark +data types are currently supported and an error can be raised if a column has an unsupported type, +see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, +Spark will fall back to create the DataFrame without Arrow. + +## Pandas UDFs (a.k.a. Vectorized UDFs) + +Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and +Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator +or to wrap the function, no additional configuration is required. Currently, there are two types of +Pandas UDF: Scalar and Grouped Map. + +### Scalar + +Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such +as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return +a `pandas.Series` of the same length. Internally, Spark will execute a Pandas UDF by splitting +columns into batches and calling the function for each batch as a subset of the data, then +concatenating the results together. + +The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. + +
    +
    +{% include_example scalar_pandas_udf python/sql/arrow.py %} +
    +
    + +### Grouped Map +Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. +Split-apply-combine consists of three steps: +* Split the data into groups by using `DataFrame.groupBy`. +* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The + input data contains all the rows and columns for each group. +* Combine the results into a new `DataFrame`. + +To use `groupBy().apply()`, the user needs to define the following: +* A Python function that defines the computation for each group. +* A `StructType` object or a string that defines the schema of the output `DataFrame`. + +The column labels of the returned `pandas.DataFrame` must either match the field names in the +defined output schema if specified as strings, or match the field data types by position if not +strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) +on how to label columns when constructing a `pandas.DataFrame`. + +Note that all data for a group will be loaded into memory before the function is applied. This can +lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for +[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user +to ensure that the grouped data will fit into the available memory. + +The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. + +
    +
    +{% include_example grouped_map_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and +[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). + +### Grouped Aggregate + +Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and +[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` +to a scalar value, where each `pandas.Series` represents a column within the group or window. + +Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, +only unbounded window is supported with Grouped aggregate Pandas UDFs currently. + +The following example shows how to use this type of UDF to compute mean with groupBy and window operations: + +
    +
    +{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) + +## Usage Notes + +### Supported SQL Types + +Currently, all Spark SQL data types are supported by Arrow-based conversion except `BinaryType`, `MapType`, +`ArrayType` of `TimestampType`, and nested `StructType`. + +### Setting Arrow Batch Size + +Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to +high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow +record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" +to an integer that will determine the maximum number of rows for each batch. The default value is +10,000 records per batch. If the number of columns is large, the value should be adjusted +accordingly. Using this limit, each data partition will be made into 1 or more record batches for +processing. + +### Timestamp with Time Zone Semantics + +Spark internally stores timestamps as UTC values, and timestamp data that is brought in without +a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp +data is exported or displayed in Spark, the session time zone is used to localize the timestamp +values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will +default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond +resolution, `datetime64[ns]`, with optional time zone on a per-column basis. + +When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds +and each column will be converted to the Spark session time zone then localized to that time +zone, which removes the time zone and displays values as local time. This will occur +when calling `toPandas()` or `pandas_udf` with timestamp columns. + +When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This +occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a +`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the +expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond +values will be truncated. + +Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is +different than a Pandas timestamp. It is recommended to use Pandas time series functionality when +working with timestamps in `pandas_udf`s to get the best performance, see +[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. + +# Migration Guide + +## Upgrading From Spark SQL 2.3 to 2.4 + + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Result Spark 2.3 or Prior + + Result Spark 2.4 + + Remarks +
    + SELECT
    array_contains(array(1), 1.34D);
    +
    + true + + false + + In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. +
    + SELECT
    array_contains(array(1), '1');
    +
    + true + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explict cast +
    + SELECT
    array_contains(array(1), 'anystring');
    +
    + null + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explict cast +
    + + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, an column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + +## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above + + - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. + +## Upgrading From Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + InputA \ InputB + + NullType + + IntegerType + + LongType + + DecimalType(38,0)* + + DoubleType + + DateType + + TimestampType + + StringType +
    + NullType + NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    + IntegerType + IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    + LongType + LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DecimalType(38,0)* + DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DoubleType + DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    + DateType + DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    + TimestampType + TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    + StringType + StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    + + Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. + - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + +## Upgrading From Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + + - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + +## Upgrading From Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading From Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading From Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. + + - 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 + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false`. + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + - Timestamps are now stored at a precision of 1us, rather than 1ns + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. + - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. + +## Upgrading from Spark SQL 1.3 to 1.4 + +#### DataFrame data reader/writer interface + +Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) +and writing data out (`DataFrame.write`), +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). + +See the API docs for `SQLContext.read` ( + Scala, + Java, + Python +) and `DataFrame.write` ( + Scala, + Java, + Python +) more information. + + +#### DataFrame.groupBy retains grouping columns + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + +#### Behavior change on DataFrame.withColumn + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general these classes try to +use types that are usable from both languages (i.e. `Array` instead of language-specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally, the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight scala %} + +sqlContext.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + +#### Python DataTypes No Longer Singletons + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. + +## Compatibility with Apache Hive + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. +Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +and Spark SQL can be connected to different versions of Hive Metastore +(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive Metastore](#interacting-with-different-versions-of-hive-metastore)). + +#### Deploying in Existing Hive Warehouses + +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDes) +* Window functions +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables including dynamic partition insertion +* View +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `DATE` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + + +**Esoteric Hive Features** + +* `UNION` type +* Unique join +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment and only supports populating the sizeInBytes field of the hive metastore. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +less important due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block-level bitmap indexes and virtual columns (used to build indexes) +* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". +* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + +### Incompatible Hive UDF + +Below are the scenarios in which Hive and Spark generate different results: + +* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. +* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. +* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. + +# Reference + +## Data Types + +Spark SQL and DataFrames support the following data types: + +* Numeric types + - `ByteType`: Represents 1-byte signed integer numbers. + The range of numbers is from `-128` to `127`. + - `ShortType`: Represents 2-byte signed integer numbers. + The range of numbers is from `-32768` to `32767`. + - `IntegerType`: Represents 4-byte signed integer numbers. + The range of numbers is from `-2147483648` to `2147483647`. + - `LongType`: Represents 8-byte signed integer numbers. + The range of numbers is from `-9223372036854775808` to `9223372036854775807`. + - `FloatType`: Represents 4-byte single-precision floating point numbers. + - `DoubleType`: Represents 8-byte double-precision floating point numbers. + - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. +* String type + - `StringType`: Represents character string values. +* Binary type + - `BinaryType`: Represents byte sequence values. +* Boolean type + - `BooleanType`: Represents boolean values. +* Datetime type + - `TimestampType`: Represents values comprising values of fields year, month, day, + hour, minute, and second. + - `DateType`: Represents values comprising values of fields year, month, day. +* Complex types + - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of + elements with the type of `elementType`. `containsNull` is used to indicate if + elements in a `ArrayType` value can have `null` values. + - `MapType(keyType, valueType, valueContainsNull)`: + Represents values comprising a set of key-value pairs. The data type of keys are + described by `keyType` and the data type of values are described by `valueType`. + For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` + is used to indicate if values of a `MapType` value can have `null` values. + - `StructType(fields)`: Represents values with the structure described by + a sequence of `StructField`s (`fields`). + * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. + The name of a field is indicated by `name`. The data type of a field is indicated + by `dataType`. `nullable` is used to indicate if values of this fields can have + `null` values. + +
    +
    + +All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. +You can access them by doing + +{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte + ByteType +
    ShortType Short + ShortType +
    IntegerType Int + IntegerType +
    LongType Long + LongType +
    FloatType Float + FloatType +
    DoubleType Double + DoubleType +
    DecimalType java.math.BigDecimal + DecimalType +
    StringType String + StringType +
    BinaryType Array[Byte] + BinaryType +
    BooleanType Boolean + BooleanType +
    TimestampType java.sql.Timestamp + TimestampType +
    DateType java.sql.Date + DateType +
    ArrayType scala.collection.Seq + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is true. +
    MapType scala.collection.Map + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is true. +
    StructType org.apache.spark.sql.Row + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Scala of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is true. +
    + +
    + +
    + +All data types of Spark SQL are located in the package of +`org.apache.spark.sql.types`. To access or create a data type, +please use factory methods provided in +`org.apache.spark.sql.types.DataTypes`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte + DataTypes.ByteType +
    ShortType short or Short + DataTypes.ShortType +
    IntegerType int or Integer + DataTypes.IntegerType +
    LongType long or Long + DataTypes.LongType +
    FloatType float or Float + DataTypes.FloatType +
    DoubleType double or Double + DataTypes.DoubleType +
    DecimalType java.math.BigDecimal + DataTypes.createDecimalType()
    + DataTypes.createDecimalType(precision, scale). +
    StringType String + DataTypes.StringType +
    BinaryType byte[] + DataTypes.BinaryType +
    BooleanType boolean or Boolean + DataTypes.BooleanType +
    TimestampType java.sql.Timestamp + DataTypes.TimestampType +
    DateType java.sql.Date + DataTypes.DateType +
    ArrayType java.util.List + DataTypes.createArrayType(elementType)
    + Note: The value of containsNull will be true
    + DataTypes.createArrayType(elementType, containsNull). +
    MapType java.util.Map + DataTypes.createMapType(keyType, valueType)
    + Note: The value of valueContainsNull will be true.
    + DataTypes.createMapType(keyType, valueType, valueContainsNull)
    +
    StructType org.apache.spark.sql.Row + DataTypes.createStructType(fields)
    + Note: fields is a List or an array of StructFields. + Also, two fields with the same name are not allowed. +
    StructField The value type in Java of the data type of this field + (For example, int for a StructField with the data type IntegerType) + DataTypes.createStructField(name, dataType, nullable) +
    + +
    + +
    + +All data types of Spark SQL are located in the package of `pyspark.sql.types`. +You can access them by doing +{% highlight python %} +from pyspark.sql.types import * +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in PythonAPI to access or create a data type
    ByteType + int or long
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + ByteType() +
    ShortType + int or long
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + ShortType() +
    IntegerType int or long + IntegerType() +
    LongType + long
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + LongType() +
    FloatType + float
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + FloatType() +
    DoubleType float + DoubleType() +
    DecimalType decimal.Decimal + DecimalType() +
    StringType string + StringType() +
    BinaryType bytearray + BinaryType() +
    BooleanType bool + BooleanType() +
    TimestampType datetime.datetime + TimestampType() +
    DateType datetime.date + DateType() +
    ArrayType list, tuple, or array + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is True. +
    MapType dict + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is True. +
    StructType list or tuple + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Python of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is True. +
    + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in RAPI to access or create a data type
    ByteType + integer
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + "byte" +
    ShortType + integer
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + "short" +
    IntegerType integer + "integer" +
    LongType + integer
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + "long" +
    FloatType + numeric
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + "float" +
    DoubleType numeric + "double" +
    DecimalType Not supported + Not supported +
    StringType character + "string" +
    BinaryType raw + "binary" +
    BooleanType logical + "bool" +
    TimestampType POSIXct + "timestamp" +
    DateType Date + "date" +
    ArrayType vector or list + list(type="array", elementType=elementType, containsNull=[containsNull])
    + Note: The default value of containsNull is TRUE. +
    MapType environment + list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    + Note: The default value of valueContainsNull is TRUE. +
    StructType named list + list(type="struct", fields=fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in R of the data type of this field + (For example, integer for a StructField with the data type IntegerType) + list(name=name, type=dataType, nullable=[nullable])
    + Note: The default value of nullable is TRUE. +
    + +
    + +
    + +## NaN Semantics + +There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that +does not exactly match standard floating point semantics. +Specifically: + + - NaN = NaN returns true. + - In aggregations, all NaN values are grouped together. + - NaN is treated as a normal value in join keys. + - NaN values go last when in ascending order, larger than any other numeric value. + + ## Arithmetic operations + +Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. +This means that in case an operation causes an overflow, the result is the same that the same operation +returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, +the result is a negative number). diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md deleted file mode 100644 index d04b955f9bf8b..0000000000000 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ /dev/null @@ -1,167 +0,0 @@ ---- -layout: global -title: PySpark Usage Guide for Pandas with Apache Arrow -displayTitle: PySpark Usage Guide for Pandas with Apache Arrow ---- - -* Table of contents -{:toc} - -## Apache Arrow in Spark - -Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer -data between JVM and Python processes. This currently is most beneficial to Python users that -work with Pandas/NumPy data. Its usage is not automatic and might require some minor -changes to configuration or code to take full advantage and ensure compatibility. This guide will -give a high-level description of how to use Arrow in Spark and highlight any differences when -working with Arrow-enabled data. - -### Ensure PyArrow Installed - -If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the -SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow -is installed and available on all cluster nodes. The current supported version is 0.8.0. -You can install using pip or conda from the conda-forge channel. See PyArrow -[installation](https://arrow.apache.org/docs/python/install.html) for details. - -## Enabling for Conversion to/from Pandas - -Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame -using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with -`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set -the Spark configuration 'spark.sql.execution.arrow.enabled' to 'true'. This is disabled by default. - -In addition, optimizations enabled by 'spark.sql.execution.arrow.enabled' could fallback automatically -to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. -This can be controlled by 'spark.sql.execution.arrow.fallback.enabled'. - -
    -
    -{% include_example dataframe_with_arrow python/sql/arrow.py %} -
    -
    - -Using the above optimizations with Arrow will produce the same results as when Arrow is not -enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the -DataFrame to the driver program and should be done on a small subset of the data. Not all Spark -data types are currently supported and an error can be raised if a column has an unsupported type, -see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, -Spark will fall back to create the DataFrame without Arrow. - -## Pandas UDFs (a.k.a. Vectorized UDFs) - -Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and -Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator -or to wrap the function, no additional configuration is required. Currently, there are two types of -Pandas UDF: Scalar and Grouped Map. - -### Scalar - -Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such -as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return -a `pandas.Series` of the same length. Internally, Spark will execute a Pandas UDF by splitting -columns into batches and calling the function for each batch as a subset of the data, then -concatenating the results together. - -The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. - -
    -
    -{% include_example scalar_pandas_udf python/sql/arrow.py %} -
    -
    - -### Grouped Map -Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. -Split-apply-combine consists of three steps: -* Split the data into groups by using `DataFrame.groupBy`. -* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The - input data contains all the rows and columns for each group. -* Combine the results into a new `DataFrame`. - -To use `groupBy().apply()`, the user needs to define the following: -* A Python function that defines the computation for each group. -* A `StructType` object or a string that defines the schema of the output `DataFrame`. - -The column labels of the returned `pandas.DataFrame` must either match the field names in the -defined output schema if specified as strings, or match the field data types by position if not -strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) -on how to label columns when constructing a `pandas.DataFrame`. - -Note that all data for a group will be loaded into memory before the function is applied. This can -lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for -[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user -to ensure that the grouped data will fit into the available memory. - -The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. - -
    -
    -{% include_example grouped_map_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). - -### Grouped Aggregate - -Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and -[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` -to a scalar value, where each `pandas.Series` represents a column within the group or window. - -Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, -only unbounded window is supported with Grouped aggregate Pandas UDFs currently. - -The following example shows how to use this type of UDF to compute mean with groupBy and window operations: - -
    -
    -{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) - -## Usage Notes - -### Supported SQL Types - -Currently, all Spark SQL data types are supported by Arrow-based conversion except `MapType`, -`ArrayType` of `TimestampType`, and nested `StructType`. `BinaryType` is supported only when -installed PyArrow is equal to or higher then 0.10.0. - -### Setting Arrow Batch Size - -Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to -high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow -record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" -to an integer that will determine the maximum number of rows for each batch. The default value is -10,000 records per batch. If the number of columns is large, the value should be adjusted -accordingly. Using this limit, each data partition will be made into 1 or more record batches for -processing. - -### Timestamp with Time Zone Semantics - -Spark internally stores timestamps as UTC values, and timestamp data that is brought in without -a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp -data is exported or displayed in Spark, the session time zone is used to localize the timestamp -values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will -default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond -resolution, `datetime64[ns]`, with optional time zone on a per-column basis. - -When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds -and each column will be converted to the Spark session time zone then localized to that time -zone, which removes the time zone and displays values as local time. This will occur -when calling `toPandas()` or `pandas_udf` with timestamp columns. - -When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This -occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a -`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the -expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond -values will be truncated. - -Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is -different than a Pandas timestamp. It is recommended to use Pandas time series functionality when -working with timestamps in `pandas_udf`s to get the best performance, see -[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. diff --git a/docs/sql-reference.md b/docs/sql-reference.md deleted file mode 100644 index 9e4239b6bad23..0000000000000 --- a/docs/sql-reference.md +++ /dev/null @@ -1,641 +0,0 @@ ---- -layout: global -title: Reference -displayTitle: Reference ---- - -* Table of contents -{:toc} - -## Data Types - -Spark SQL and DataFrames support the following data types: - -* Numeric types - - `ByteType`: Represents 1-byte signed integer numbers. - The range of numbers is from `-128` to `127`. - - `ShortType`: Represents 2-byte signed integer numbers. - The range of numbers is from `-32768` to `32767`. - - `IntegerType`: Represents 4-byte signed integer numbers. - The range of numbers is from `-2147483648` to `2147483647`. - - `LongType`: Represents 8-byte signed integer numbers. - The range of numbers is from `-9223372036854775808` to `9223372036854775807`. - - `FloatType`: Represents 4-byte single-precision floating point numbers. - - `DoubleType`: Represents 8-byte double-precision floating point numbers. - - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. -* String type - - `StringType`: Represents character string values. -* Binary type - - `BinaryType`: Represents byte sequence values. -* Boolean type - - `BooleanType`: Represents boolean values. -* Datetime type - - `TimestampType`: Represents values comprising values of fields year, month, day, - hour, minute, and second. - - `DateType`: Represents values comprising values of fields year, month, day. -* Complex types - - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of - elements with the type of `elementType`. `containsNull` is used to indicate if - elements in a `ArrayType` value can have `null` values. - - `MapType(keyType, valueType, valueContainsNull)`: - Represents values comprising a set of key-value pairs. The data type of keys are - described by `keyType` and the data type of values are described by `valueType`. - For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` - is used to indicate if values of a `MapType` value can have `null` values. - - `StructType(fields)`: Represents values with the structure described by - a sequence of `StructField`s (`fields`). - * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. - The name of a field is indicated by `name`. The data type of a field is indicated - by `dataType`. `nullable` is used to indicate if values of this fields can have - `null` values. - -
    -
    - -All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. -You can access them by doing - -{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte - ByteType -
    ShortType Short - ShortType -
    IntegerType Int - IntegerType -
    LongType Long - LongType -
    FloatType Float - FloatType -
    DoubleType Double - DoubleType -
    DecimalType java.math.BigDecimal - DecimalType -
    StringType String - StringType -
    BinaryType Array[Byte] - BinaryType -
    BooleanType Boolean - BooleanType -
    TimestampType java.sql.Timestamp - TimestampType -
    DateType java.sql.Date - DateType -
    ArrayType scala.collection.Seq - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is true. -
    MapType scala.collection.Map - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is true. -
    StructType org.apache.spark.sql.Row - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Scala of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is true. -
    - -
    - -
    - -All data types of Spark SQL are located in the package of -`org.apache.spark.sql.types`. To access or create a data type, -please use factory methods provided in -`org.apache.spark.sql.types.DataTypes`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte - DataTypes.ByteType -
    ShortType short or Short - DataTypes.ShortType -
    IntegerType int or Integer - DataTypes.IntegerType -
    LongType long or Long - DataTypes.LongType -
    FloatType float or Float - DataTypes.FloatType -
    DoubleType double or Double - DataTypes.DoubleType -
    DecimalType java.math.BigDecimal - DataTypes.createDecimalType()
    - DataTypes.createDecimalType(precision, scale). -
    StringType String - DataTypes.StringType -
    BinaryType byte[] - DataTypes.BinaryType -
    BooleanType boolean or Boolean - DataTypes.BooleanType -
    TimestampType java.sql.Timestamp - DataTypes.TimestampType -
    DateType java.sql.Date - DataTypes.DateType -
    ArrayType java.util.List - DataTypes.createArrayType(elementType)
    - Note: The value of containsNull will be true
    - DataTypes.createArrayType(elementType, containsNull). -
    MapType java.util.Map - DataTypes.createMapType(keyType, valueType)
    - Note: The value of valueContainsNull will be true.
    - DataTypes.createMapType(keyType, valueType, valueContainsNull)
    -
    StructType org.apache.spark.sql.Row - DataTypes.createStructType(fields)
    - Note: fields is a List or an array of StructFields. - Also, two fields with the same name are not allowed. -
    StructField The value type in Java of the data type of this field - (For example, int for a StructField with the data type IntegerType) - DataTypes.createStructField(name, dataType, nullable) -
    - -
    - -
    - -All data types of Spark SQL are located in the package of `pyspark.sql.types`. -You can access them by doing -{% highlight python %} -from pyspark.sql.types import * -{% endhighlight %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in PythonAPI to access or create a data type
    ByteType - int or long
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - ByteType() -
    ShortType - int or long
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - ShortType() -
    IntegerType int or long - IntegerType() -
    LongType - long
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - LongType() -
    FloatType - float
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - FloatType() -
    DoubleType float - DoubleType() -
    DecimalType decimal.Decimal - DecimalType() -
    StringType string - StringType() -
    BinaryType bytearray - BinaryType() -
    BooleanType bool - BooleanType() -
    TimestampType datetime.datetime - TimestampType() -
    DateType datetime.date - DateType() -
    ArrayType list, tuple, or array - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is True. -
    MapType dict - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is True. -
    StructType list or tuple - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Python of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is True. -
    - -
    - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in RAPI to access or create a data type
    ByteType - integer
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - "byte" -
    ShortType - integer
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - "short" -
    IntegerType integer - "integer" -
    LongType - integer
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - "long" -
    FloatType - numeric
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - "float" -
    DoubleType numeric - "double" -
    DecimalType Not supported - Not supported -
    StringType character - "string" -
    BinaryType raw - "binary" -
    BooleanType logical - "bool" -
    TimestampType POSIXct - "timestamp" -
    DateType Date - "date" -
    ArrayType vector or list - list(type="array", elementType=elementType, containsNull=[containsNull])
    - Note: The default value of containsNull is TRUE. -
    MapType environment - list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    - Note: The default value of valueContainsNull is TRUE. -
    StructType named list - list(type="struct", fields=fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in R of the data type of this field - (For example, integer for a StructField with the data type IntegerType) - list(name=name, type=dataType, nullable=[nullable])
    - Note: The default value of nullable is TRUE. -
    - -
    - -
    - -## NaN Semantics - -There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that -does not exactly match standard floating point semantics. -Specifically: - - - NaN = NaN returns true. - - In aggregations, all NaN values are grouped together. - - NaN is treated as a normal value in join keys. - - NaN values go last when in ascending order, larger than any other numeric value. - -## Arithmetic operations - -Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. -This means that in case an operation causes an overflow, the result is the same that the same operation -returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, -the result is a negative number). diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a83ebd9449fa4..44ae52e81cd64 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the ones for which it has built-in support (that is, beyond Kafka, Kinesis, files, sockets, etc.). +the ones for which it has built-in support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. Note that custom receivers can be implemented diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md new file mode 100644 index 0000000000000..a1b6942ffe0a4 --- /dev/null +++ b/docs/streaming-flume-integration.md @@ -0,0 +1,169 @@ +--- +layout: global +title: Spark Streaming + Flume Integration Guide +--- + +[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. + +**Note: Flume support is deprecated as of Spark 2.3.0.** + +## Approach 1: Flume-style Push-based Approach +Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. + +#### General Requirements +Choose a machine in your cluster such that + +- When your Flume + Spark Streaming application is launched, one of the Spark workers must run on that machine. + +- Flume can be configured to push data to a port on that machine. + +Due to the push model, the streaming application needs to be up, with the receiver scheduled and listening on the chosen port, for Flume to be able to push data. + +#### Configuring Flume +Configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = avroSink + agent.sinks.avroSink.type = avro + agent.sinks.avroSink.channel = memoryChannel + agent.sinks.avroSink.hostname = + agent.sinks.avroSink.port = + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$). +
    +
    + import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStream flumeStream = + FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html). +
    +
    + from pyspark.streaming.flume import FlumeUtils + + flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) + + By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. + See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils). +
    +
    + + Note that the hostname should be the same as the one used by the resource manager in the + cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch + the receiver in the right machine. + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-flume-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-flume-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Pull-based Approach using a Custom Sink +Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following. + +- Flume pushes data into the sink, and the data stays buffered. +- Spark Streaming uses a [reliable Flume receiver](streaming-programming-guide.html#receiver-reliability) + and transactions to pull data from the sink. Transactions succeed only after data is received and + replicated by Spark Streaming. + +This ensures stronger reliability and +[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) +than the previous approach. However, this requires configuring Flume to run a custom sink. +Here are the configuration steps. + +#### General Requirements +Choose a machine that will run the custom sink in a Flume agent. The rest of the Flume pipeline is configured to send data to that agent. Machines in the Spark cluster should have access to the chosen machine running the custom sink. + +#### Configuring Flume +Configuring Flume on the chosen machine requires the following two steps. + +1. **Sink JARs**: Add the following JARs to Flume's classpath (see [Flume's documentation](https://flume.apache.org/documentation.html) to see how) in the machine designated to run the custom sink. + + (i) *Custom sink JAR*: Download the JAR corresponding to the following artifact (or [direct link](http://search.maven.org/remotecontent?filepath=org/apache/spark/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}/{{site.SPARK_VERSION_SHORT}}/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION_SHORT}}.jar)). + + groupId = org.apache.spark + artifactId = spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + (ii) *Scala library JAR*: Download the Scala library JAR for Scala {{site.SCALA_VERSION}}. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/scala-lang/scala-library/{{site.SCALA_VERSION}}/scala-library-{{site.SCALA_VERSION}}.jar)). + + groupId = org.scala-lang + artifactId = scala-library + version = {{site.SCALA_VERSION}} + + (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)). + + groupId = org.apache.commons + artifactId = commons-lang3 + version = 3.5 + +2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = spark + agent.sinks.spark.type = org.apache.spark.streaming.flume.sink.SparkSink + agent.sinks.spark.hostname = + agent.sinks.spark.port = + agent.sinks.spark.channel = memoryChannel + + Also, make sure that the upstream Flume pipeline is configured to send the data to the Flume agent running this sink. + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide). + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]) +
    +
    + import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStreamflumeStream = + FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]); +
    +
    + from pyspark.streaming.flume import FlumeUtils + + addresses = [([sink machine hostname 1], [sink port 1]), ([sink machine hostname 2], [sink port 2])] + flumeStream = FlumeUtils.createPollingStream(streamingContext, addresses) + + By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type. + See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils). +
    +
    + + Note that each input DStream can be configured to receive data from multiple sinks. + +3. **Deploying:** This is same as the first approach. + + + diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index c78459cd27d1e..386066a85749f 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -3,10 +3,7 @@ layout: global title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) --- -The Spark Streaming integration for Kafka 0.10 provides simple parallelism, 1:1 correspondence between Kafka -partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses -the [new Kafka consumer API](https://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, -there are notable differences in usage. +The Spark Streaming integration for Kafka 0.10 is similar in design to the 0.8 [Direct Stream approach](streaming-kafka-0-8-integration.html#approach-2-direct-approach-no-receivers). It provides simple parallelism, 1:1 correspondence between Kafka partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses the [new Kafka consumer API](http://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, there are notable differences in usage. This version of the integration is marked as experimental, so the API is potentially subject to change. ### Linking For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md new file mode 100644 index 0000000000000..becf217738d26 --- /dev/null +++ b/docs/streaming-kafka-0-8-integration.md @@ -0,0 +1,196 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.8.2.1 or higher) +--- + +**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** + +Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. Both approaches are considered stable APIs as of the current version of Spark. + +## Approach 1: Receiver-based Approach +This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. + +However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write-Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write-ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write-Ahead Logs. + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val kafkaStream = KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream kafkaStream = + KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); + + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + + kafkaStream = KafkaUtils.createStream(streamingContext, \ + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). +
    +
    + + **Points to remember:** + + - Topic partitions in Kafka do not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. + + - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. + + - If you have enabled Write-Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use +`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the + [Maven repository](https://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Direct Approach (No Receivers) +This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this feature was introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. + +This approach has the following advantages over the receiver-based approach (i.e. Approach 1). + +- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. + +- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write-Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write-Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write-Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. + +- *Exactly-once semantics:* The first approach uses Kafka's high-level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with-write-ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). + +Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val directKafkaStream = KafkaUtils.createDirectStream[ + [key class], [value class], [key decoder class], [value decoder class] ]( + streamingContext, [map of Kafka parameters], [set of topics to consume]) + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairInputDStream directKafkaStream = + KafkaUtils.createDirectStream(streamingContext, + [key class], [value class], [key decoder class], [value decoder class], + [map of Kafka parameters], [set of topics to consume]); + + You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). + +
    +
    + from pyspark.streaming.kafka import KafkaUtils + directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) + + You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). +
    +
    + + In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. + By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. + + You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. + +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array.empty[OffsetRange] + + directKafkaStream.transform { rdd => + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.map { + ... + }.foreachRDD { rdd => + for (o <- offsetRanges) { + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + ... + } +
    +
    + // Hold a reference to the current offset ranges, so it can be used downstream + AtomicReference offsetRanges = new AtomicReference<>(); + + directKafkaStream.transformToPair(rdd -> { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + return rdd; + }).map( + ... + ).foreachRDD(rdd -> { + for (OffsetRange o : offsetRanges.get()) { + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() + ); + } + ... + }); +
    +
    + offsetRanges = [] + + def storeOffsetRanges(rdd): + global offsetRanges + offsetRanges = rdd.offsetRanges() + return rdd + + def printOffsetRanges(rdd): + for o in offsetRanges: + print "%s %s %s %s" % (o.topic, o.partition, o.fromOffset, o.untilOffset) + + directKafkaStream \ + .transform(storeOffsetRanges) \ + .foreachRDD(printOffsetRanges) +
    +
    + + You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. + + Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). + + Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. + +3. **Deploying:** This is same as the first approach. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 0ec5a31ddfd2f..4aca391e4ba1a 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -3,9 +3,52 @@ layout: global title: Spark Streaming + Kafka Integration Guide --- -[Apache Kafka](https://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, -replicated commit log service. Please read the [Kafka documentation](https://kafka.apache.org/documentation.html) -thoroughly before starting an integration using Spark. +[Apache Kafka](https://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Please read the [Kafka documentation](https://kafka.apache.org/documentation.html) thoroughly before starting an integration using Spark. -At the moment, Spark requires Kafka 0.10 and higher. See -Kafka 0.10 integration documentation for details. \ No newline at end of file +The Kafka project introduced a new consumer API between versions 0.8 and 0.10, so there are 2 separate corresponding Spark Streaming packages available. Please choose the correct package for your brokers and desired features; note that the 0.8 integration is compatible with later 0.9 and 0.10 brokers, but the 0.10 integration is not compatible with earlier brokers. + +**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    spark-streaming-kafka-0-8spark-streaming-kafka-0-10
    Broker Version0.8.2.1 or higher0.10.0 or higher
    API MaturityDeprecatedStable
    Language SupportScala, Java, PythonScala, Java
    Receiver DStreamYesNo
    Direct DStreamYesYes
    SSL / TLS SupportNoYes
    Offset Commit APINoYes
    Dynamic Topic SubscriptionNoYes
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 70bee5032a24d..0ca0f2a8b54d5 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -11,7 +11,7 @@ description: Spark Streaming programming guide and tutorial for Spark SPARK_VERS # Overview Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Kinesis, or TCP sockets, and can be processed using complex +like Kafka, Flume, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's @@ -40,7 +40,7 @@ stream of results in batches. Spark Streaming provides a high-level abstraction called *discretized stream* or *DStream*, which represents a continuous stream of data. DStreams can be created either from input data -streams from sources such as Kafka, and Kinesis, or by applying high-level +streams from sources such as Kafka, Flume, and Kinesis, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of [RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). @@ -393,7 +393,7 @@ Similar to Spark, Spark Streaming is available through Maven Central. To write y -For ingesting data from sources like Kafka and Kinesis that are not present in the Spark +For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark Streaming core API, you will have to add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, @@ -402,6 +402,7 @@ some of the common ones are as follows. +
    SourceArtifact
    Kafka spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Amazon Software License]
    @@ -576,7 +577,7 @@ Spark Streaming provides two categories of built-in streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. Examples: file systems, and socket connections. -- *Advanced sources*: Sources like Kafka, Kinesis, etc. are available through +- *Advanced sources*: Sources like Kafka, Flume, Kinesis, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. @@ -596,7 +597,7 @@ as well as to run the receiver(s). - When running a Spark Streaming program locally, do not use "local" or "local[1]" as the master URL. Either of these means that only one thread will be used for running tasks locally. If you are using - an input DStream based on a receiver (e.g. sockets, Kafka, etc.), then the single thread will + an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when running locally, always use "local[*n*]" as the master URL, where *n* > number of receivers to run (see [Spark Properties](configuration.html#spark-properties) for information on how to set @@ -731,10 +732,10 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea {:.no_toc} Python API As of Spark {{site.SPARK_VERSION_SHORT}}, -out of these sources, Kafka and Kinesis are available in the Python API. +out of these sources, Kafka, Kinesis and Flume are available in the Python API. This category of sources require interfacing with external non-Spark libraries, some of them with -complex dependencies (e.g., Kafka). Hence, to minimize issues related to version conflicts +complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate libraries that can be [linked](#linking) to explicitly when necessary. @@ -745,7 +746,9 @@ and add it to the classpath. Some of these advanced sources are as follows. -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.10 or higher. See the [Kafka Integration Guide](streaming-kafka-0-10-integration.html) for more details. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. + +- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.6.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. - **Kinesis:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kinesis Client Library 1.2.1. See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. @@ -763,7 +766,7 @@ Guide](streaming-custom-receivers.html) for details. {:.no_toc} There can be two kinds of data sources based on their *reliability*. Sources -(like Kafka) allow the transferred data to be acknowledged. If the system receiving +(like Kafka and Flume) allow the transferred data to be acknowledged. If the system receiving data from these *reliable* sources acknowledges the received data correctly, it can be ensured that no data will be lost due to any kind of failure. This leads to two kinds of receivers: @@ -1600,7 +1603,7 @@ operations on the same data). For window-based operations like `reduceByWindow` Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. -For input streams that receive data over the network (such as, Kafka, sockets, etc.), the +For input streams that receive data over the network (such as, Kafka, Flume, sockets, etc.), the default persistence level is set to replicate the data to two nodes for fault-tolerance. Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in @@ -1970,7 +1973,7 @@ To run a Spark Streaming applications, you need to have the following. - *Package the application JAR* - You have to compile your streaming application into a JAR. If you are using [`spark-submit`](submitting-applications.html) to start the application, then you will not need to provide Spark and Spark Streaming in the JAR. However, - if your application uses [advanced sources](#advanced-sources) (e.g. Kafka), + if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, Flume), then you will have to package the extra artifact they link to, along with their dependencies, in the JAR that is used to deploy the application. For example, an application using `KafkaUtils` will have to include `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and all its @@ -2057,7 +2060,7 @@ for graceful shutdown options) which ensure data that has been received is compl processed before shutdown. Then the upgraded application can be started, which will start processing from the same point where the earlier application left off. Note that this can be done only with input sources that support source-side buffering -(like Kafka) as data needs to be buffered while the previous application was down and +(like Kafka, and Flume) as data needs to be buffered while the previous application was down and the upgraded application is not yet up. And restarting from earlier checkpoint information of pre-upgrade code cannot be done. The checkpoint information essentially contains serialized Scala/Java/Python objects and trying to deserialize objects with new, @@ -2112,7 +2115,7 @@ highlights some of the most important ones. ### Level of Parallelism in Data Receiving {:.no_toc} -Receiving data over the network (like Kafka, socket, etc.) requires the data to be deserialized +Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to be deserialized and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider parallelizing the data receiving. Note that each input DStream creates a single receiver (running on a worker machine) that receives a single stream of data. @@ -2172,7 +2175,7 @@ the input data stream (using `inputStream.repartition()`). This distributes the received batches of data across the specified number of machines in the cluster before further processing. -For direct stream, please refer to [Spark Streaming + Kafka Integration Guide](streaming-kafka-0-10-integration.html) +For direct stream, please refer to [Spark Streaming + Kafka Integration Guide](streaming-kafka-integration.html) ### Level of Parallelism in Data Processing {:.no_toc} @@ -2433,7 +2436,7 @@ The following table summarizes the semantics under failures: ### With Kafka Direct API {:.no_toc} -In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-0-10-integration.html). +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). ## Semantics of output operations {:.no_toc} @@ -2463,7 +2466,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are # Where to Go from Here * Additional guides - - [Kafka Integration Guide](streaming-kafka-0-10-integration.html) + - [Kafka Integration Guide](streaming-kafka-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) * Third-party DStream data sources can be found in [Third Party Projects](https://spark.apache.org/third-party-projects.html) @@ -2472,12 +2475,14 @@ additional effort may be necessary to achieve exactly-once semantics. There are * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), - Java docs * [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html), [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), + [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) - Python docs * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 8cea98c2cc52b..73de1892977ac 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -497,7 +497,7 @@ There are a few built-in sources. - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, orc, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. - - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-0-10-integration.html) for more details. + - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. @@ -536,7 +536,7 @@ Here are the details of all the sources in Spark. href="api/R/read.stream.html">R). E.g. for "parquet" format options see DataStreamReader.parquet().

    - In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. + In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. Yes Supports glob paths, but does not support multiple comma-separated paths/globs. @@ -566,7 +566,7 @@ Here are the details of all the sources in Spark. Kafka Source - See the Kafka Integration Guide. + See the Kafka Integration Guide. Yes @@ -1560,35 +1560,6 @@ streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime") -### Policy for handling multiple watermarks -A streaming query can have multiple input streams that are unioned or joined together. -Each of the input streams can have a different threshold of late data that needs to -be tolerated for stateful operations. You specify these thresholds using -``withWatermarks("eventTime", delay)`` on each of the input streams. For example, consider -a query with stream-stream joins between `inputStream1` and `inputStream2`. - - inputStream1.withWatermark("eventTime1", "1 hour") - .join( - inputStream2.withWatermark("eventTime2", "2 hours"), - joinCondition) - -While executing the query, Structured Streaming individually tracks the maximum -event time seen in each input stream, calculates watermarks based on the corresponding delay, -and chooses a single global watermark with them to be used for stateful operations. By default, -the minimum is chosen as the global watermark because it ensures that no data is -accidentally dropped as too late if one of the streams falls behind the others -(for example, one of the streams stop receiving data due to upstream failures). In other words, -the global watermark will safely move at the pace of the slowest stream and the query output will -be delayed accordingly. - -However, in some cases, you may want to get faster results even if it means dropping data from the -slowest stream. Since Spark 2.4, you can set the multiple watermark policy to choose -the maximum value as the global watermark by setting the SQL configuration -``spark.sql.streaming.multipleWatermarkPolicy`` to ``max`` (default is ``min``). -This lets the global watermark move at the pace of the fastest stream. -However, as a side effect, data from the slower streams will be aggressively dropped. Hence, use -this configuration judiciously. - ### Arbitrary Stateful Operations Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)). @@ -1819,7 +1790,7 @@ Here are the details of all the sinks in Spark. Kafka Sink Append, Update, Complete - See the Kafka Integration Guide + See the Kafka Integration Guide Yes (at-least-once) More details in the Kafka Integration Guide @@ -1828,16 +1799,8 @@ Here are the details of all the sinks in Spark. Append, Update, Complete None Depends on ForeachWriter implementation - More details in the next section + More details in the next section - - ForeachBatch Sink - Append, Update, Complete - None - Depends on the implementation - More details in the next section - - Console Sink Append, Update, Complete @@ -2026,214 +1989,22 @@ head(sql("select * from aggregates")) -##### Using Foreach and ForeachBatch -The `foreach` and `foreachBatch` operations allow you to apply arbitrary operations and writing -logic on the output of a streaming query. They have slightly different use cases - while `foreach` -allows custom write logic on every row, `foreachBatch` allows arbitrary operations -and custom logic on the output of each micro-batch. Let's understand their usages in more detail. - -###### ForeachBatch -`foreachBatch(...)` allows you to specify a function that is executed on -the output data of every micro-batch of a streaming query. Since Spark 2.4, this is supported in Scala, Java and Python. -It takes two parameters: a DataFrame or Dataset that has the output data of a micro-batch and the unique ID of the micro-batch. - -
    -
    - -{% highlight scala %} -streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => - // Transform and write batchDF -}.start() -{% endhighlight %} - -
    -
    - -{% highlight java %} -streamingDatasetOfString.writeStream().foreachBatch( - new VoidFunction2, Long> { - public void call(Dataset dataset, Long batchId) { - // Transform and write batchDF - } - } -).start(); -{% endhighlight %} - -
    -
    - -{% highlight python %} -def foreach_batch_function(df, epoch_id): - # Transform and write batchDF - pass - -streamingDF.writeStream.foreachBatch(foreach_batch_function).start() -{% endhighlight %} - -
    -
    -R is not yet supported. -
    -
    - -With `foreachBatch`, you can do the following. - -- **Reuse existing batch data sources** - For many storage systems, there may not be a streaming sink available yet, - but there may already exist a data writer for batch queries. Using `foreachBatch`, you can use the batch - data writers on the output of each micro-batch. -- **Write to multiple locations** - If you want to write the output of a streaming query to multiple locations, - then you can simply write the output DataFrame/Dataset multiple times. However, each attempt to write can - cause the output data to be recomputed (including possible re-reading of the input data). To avoid recomputations, - you should cache the output DataFrame/Dataset, write it to multiple locations, and then uncache it. Here is an outline. - - streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => - batchDF.persist() - batchDF.write.format(...).save(...) // location 1 - batchDF.write.format(...).save(...) // location 2 - batchDF.unpersist() - } - -- **Apply additional DataFrame operations** - Many DataFrame and Dataset operations are not supported - in streaming DataFrames because Spark does not support generating incremental plans in those cases. - Using `foreachBatch`, you can apply some of these operations on each micro-batch output. However, you will have to reason about the end-to-end semantics of doing that operation yourself. - -**Note:** -- By default, `foreachBatch` provides only at-least-once write guarantees. However, you can use the - batchId provided to the function as way to deduplicate the output and get an exactly-once guarantee. -- `foreachBatch` does not work with the continuous processing mode as it fundamentally relies on the - micro-batch execution of a streaming query. If you write data in the continuous mode, use `foreach` instead. - - -###### Foreach -If `foreachBatch` is not an option (for example, corresponding batch data writer does not exist, or -continuous processing mode), then you can express you custom writer logic using `foreach`. -Specifically, you can express the data writing logic by dividing it into three methods: `open`, `process`, and `close`. -Since Spark 2.4, `foreach` is available in Scala, Java and Python. - -
    -
    - -In Scala, you have to extend the class `ForeachWriter` ([docs](api/scala/index.html#org.apache.spark.sql.ForeachWriter)). - -{% highlight scala %} -streamingDatasetOfString.writeStream.foreach( - new ForeachWriter[String] { - - def open(partitionId: Long, version: Long): Boolean = { - // Open connection - } - - def process(record: String): Unit = { - // Write string to connection - } - - def close(errorOrNull: Throwable): Unit = { - // Close the connection - } - } -).start() -{% endhighlight %} - -
    -
    - -In Java, you have to extend the class `ForeachWriter` ([docs](api/java/org/apache/spark/sql/ForeachWriter.html)). -{% highlight java %} -streamingDatasetOfString.writeStream().foreach( - new ForeachWriter[String] { - - @Override public boolean open(long partitionId, long version) { - // Open connection - } - - @Override public void process(String record) { - // Write string to connection - } - - @Override public void close(Throwable errorOrNull) { - // Close the connection - } - } -).start(); - -{% endhighlight %} - -
    -
    - -In Python, you can invoke foreach in two ways: in a function or in an object. -The function offers a simple way to express your processing logic but does not allow you to -deduplicate generated data when failures cause reprocessing of some input data. -For that situation you must specify the processing logic in an object. - -1. The function takes a row as input. - - {% highlight python %} - def process_row(row): - # Write row to storage - pass - - query = streamingDF.writeStream.foreach(process_row).start() - {% endhighlight %} - -2. The object has a process method and optional open and close methods: - - {% highlight python %} - class ForeachWriter: - def open(self, partition_id, epoch_id): - # Open connection. This method is optional in Python. - pass - - def process(self, row): - # Write row to connection. This method is NOT optional in Python. - pass - - def close(self, error): - # Close the connection. This method in optional in Python. - pass - - query = streamingDF.writeStream.foreach(ForeachWriter()).start() - {% endhighlight %} - -
    -
    -R is not yet supported. -
    -
    - - -**Execution semantics** -When the streaming query is started, Spark calls the function or the object’s methods in the following way: - -- A single copy of this object is responsible for all the data generated by a single task in a query. - In other words, one instance is responsible for processing one partition of the data generated in a distributed manner. - -- This object must be serializable, because each task will get a fresh serialized-deserialized copy - of the provided object. Hence, it is strongly recommended that any initialization for writing data - (for example. opening a connection or starting a transaction) is done after the open() method has - been called, which signifies that the task is ready to generate data. - -- The lifecycle of the methods are as follows: +##### Using Foreach +The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.1, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` +([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), +which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. - - For each partition with partition_id: +- The writer must be serializable, as it will be serialized and sent to the executors for execution. - - For each batch/epoch of streaming data with epoch_id: +- All the three methods, `open`, `process` and `close` will be called on the executors. - - Method open(partitionId, epochId) is called. +- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. - - If open(...) returns true, for each row in the partition and batch/epoch, method process(row) is called. +- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. - - Method close(error) is called with error (if any) seen while processing rows. +- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. -- The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM or Python process crashes in the middle. - -- **Note:** The partitionId and epochId in the open() method can be used to deduplicate generated data - when failures cause reprocessing of some input data. This depends on the execution mode of the query. - If the streaming query is being executed in the micro-batch mode, then every partition represented - by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data. - Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit - data and achieve exactly-once guarantees. However, if the streaming query is being executed - in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. +- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. #### Triggers The trigger settings of a streaming query defines the timing of streaming data processing, whether @@ -2938,78 +2709,6 @@ write.stream(aggDF, "memory", outputMode = "complete", checkpointLocation = "pat - -## Recovery Semantics after Changes in a Streaming Query -There are limitations on what changes in a streaming query are allowed between restarts from the -same checkpoint location. Here are a few kinds of changes that are either not allowed, or -the effect of the change is not well-defined. For all of them: - -- The term *allowed* means you can do the specified change but whether the semantics of its effect - is well-defined depends on the query and the change. - -- The term *not allowed* means you should not do the specified change as the restarted query is likely - to fail with unpredictable errors. `sdf` represents a streaming DataFrame/Dataset - generated with sparkSession.readStream. - -**Types of changes** - -- *Changes in the number or type (i.e. different source) of input sources*: This is not allowed. - -- *Changes in the parameters of input sources*: Whether this is allowed and whether the semantics - of the change are well-defined depends on the source and the query. Here are a few examples. - - - Addition/deletion/modification of rate limits is allowed: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)` - - - Changes to subscribed topics/files is generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` - -- *Changes in the type of output sink*: Changes between a few specific combinations of sinks - are allowed. This needs to be verified on a case-by-case basis. Here are a few examples. - - - File sink to Kafka sink is allowed. Kafka will see only the new data. - - - Kafka sink to file sink is not allowed. - - - Kafka sink changed to foreach, or vice versa is allowed. - -- *Changes in the parameters of output sink*: Whether this is allowed and whether the semantics of - the change are well-defined depends on the sink and the query. Here are a few examples. - - - Changes to output directory of a file sink is not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` - - - Changes to output topic is allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` - - - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) is allowed, but the semantics of the change depends on the code. - -- *Changes in projection / filter / map-like operations**: Some cases are allowed. For example: - - - Addition / deletion of filters is allowed: `sdf.selectExpr("a")` to `sdf.where(...).selectExpr("a").filter(...)`. - - - Changes in projections with same output schema is allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` - - - Changes in projections with different output schema are conditionally allowed: `sdf.selectExpr("a").writeStream` to `sdf.selectExpr("b").writeStream` is allowed only if the output sink allows the schema change from `"a"` to `"b"`. - -- *Changes in stateful operations*: Some operations in streaming queries need to maintain - state data in order to continuously update the result. Structured Streaming automatically checkpoints - the state data to fault-tolerant storage (for example, HDFS, AWS S3, Azure Blob storage) and restores it after restart. - However, this assumes that the schema of the state data remains same across restarts. This means that - *any changes (that is, additions, deletions, or schema modifications) to the stateful operations of a streaming query are not allowed between restarts*. - Here is the list of stateful operations whose schema should not be changed between restarts in order to ensure state recovery: - - - *Streaming aggregation*: For example, `sdf.groupBy("a").agg(...)`. Any change in number or type of grouping keys or aggregates is not allowed. - - - *Streaming deduplication*: For example, `sdf.dropDuplicates("a")`. Any change in number or type of grouping keys or aggregates is not allowed. - - - *Stream-stream join*: For example, `sdf1.join(sdf2, ...)` (i.e. both inputs are generated with `sparkSession.readStream`). Changes - in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) not allowed. Other changes in the join condition are ill-defined. - - - *Arbitrary stateful operation*: For example, `sdf.groupByKey(...).mapGroupsWithState(...)` or `sdf.groupByKey(...).flatMapGroupsWithState(...)`. - Any change to the schema of the user-defined state and the type of timeout is not allowed. - Any change within the user-defined state-mapping function are allowed, but the semantic effect of the change depends on the user-defined logic. - If you really want to support state schema changes, then you can explicitly encode/decode your complex state data - structures into bytes using an encoding/decoding scheme that supports schema migration. For example, - if you save your state as Avro-encoded bytes, then you are free to change the Avro-state-schema between query - restarts as the binary state will always be restored successfully. - # Continuous Processing ## [Experimental] {:.no_toc} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java deleted file mode 100644 index 891f306cbf9c7..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.ml; - -// $example on$ -import java.util.Arrays; -import java.util.List; - -import org.apache.spark.ml.fpm.PrefixSpan; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.types.*; -// $example off$ - -/** - * An example demonstrating PrefixSpan. - * Run with - *
    - * bin/run-example ml.JavaPrefixSpanExample
    - * 
    - */ -public class JavaPrefixSpanExample { - public static void main(String[] args) { - SparkSession spark = SparkSession - .builder() - .appName("JavaPrefixSpanExample") - .getOrCreate(); - - // $example on$ - List data = Arrays.asList( - RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3))), - RowFactory.create(Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1,2))), - RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5))), - RowFactory.create(Arrays.asList(Arrays.asList(6))) - ); - StructType schema = new StructType(new StructField[]{ new StructField( - "sequence", new ArrayType(new ArrayType(DataTypes.IntegerType, true), true), - false, Metadata.empty()) - }); - Dataset sequenceDF = spark.createDataFrame(data, schema); - - PrefixSpan prefixSpan = new PrefixSpan().setMinSupport(0.5).setMaxPatternLength(5); - - // Finding frequent sequential patterns - prefixSpan.findFrequentSequentialPatterns(sequenceDF).show(); - // $example off$ - - spark.stop(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index cbe9dfdaa907b..ef3c904775697 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -123,13 +123,6 @@ private static void runBasicDataSourceExample(SparkSession spark) { .option("header", "true") .load("examples/src/main/resources/people.csv"); // $example off:manual_load_options_csv$ - // $example on:manual_save_options_orc$ - usersDF.write().format("orc") - .option("orc.bloom.filter.columns", "favorite_color") - .option("orc.dictionary.key.threshold", "1.0") - .option("orc.column.encoding.direct", "name") - .save("users_with_options.orc"); - // $example off:manual_save_options_orc$ // $example on:direct_sql$ Dataset sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); diff --git a/examples/src/main/python/ml/prefixspan_example.py b/examples/src/main/python/ml/prefixspan_example.py deleted file mode 100644 index 88d1d4197341b..0000000000000 --- a/examples/src/main/python/ml/prefixspan_example.py +++ /dev/null @@ -1,48 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -""" -An example demonstrating PrefixSpan. -Run with: - bin/spark-submit examples/src/main/python/ml/prefixspan_example.py -""" -# $example on$ -from pyspark.ml.fpm import PrefixSpan -# $example off$ -from pyspark.sql import Row, SparkSession - -if __name__ == "__main__": - spark = SparkSession\ - .builder\ - .appName("PrefixSpanExample")\ - .getOrCreate() - sc = spark.sparkContext - - # $example on$ - df = sc.parallelize([Row(sequence=[[1, 2], [3]]), - Row(sequence=[[1], [3, 2], [1, 2]]), - Row(sequence=[[1, 2], [5]]), - Row(sequence=[[6]])]).toDF() - - prefixSpan = PrefixSpan(minSupport=0.5, maxPatternLength=5, - maxLocalProjDBSize=32000000) - - # Find frequent sequential patterns. - prefixSpan.findFrequentSequentialPatterns(df).show() - # $example off$ - - spark.stop() diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 04660724b308d..d8c879dfe02ed 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -57,15 +57,6 @@ def basic_datasource_example(spark): format="csv", sep=":", inferSchema="true", header="true") # $example off:manual_load_options_csv$ - # $example on:manual_save_options_orc$ - df = spark.read.orc("examples/src/main/resources/users.orc") - (df.write.format("orc") - .option("orc.bloom.filter.columns", "favorite_color") - .option("orc.dictionary.key.threshold", "1.0") - .option("orc.column.encoding.direct", "name") - .save("users_with_options.orc")) - # $example off:manual_save_options_orc$ - # $example on:write_sorting_and_bucketing$ df.write.bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed") # $example off:write_sorting_and_bucketing$ diff --git a/examples/src/main/python/streaming/direct_kafka_wordcount.py b/examples/src/main/python/streaming/direct_kafka_wordcount.py new file mode 100644 index 0000000000000..c5c186c11f79a --- /dev/null +++ b/examples/src/main/python/streaming/direct_kafka_wordcount.py @@ -0,0 +1,56 @@ +# +# 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. +# + +r""" + Counts words in UTF8 encoded, '\n' delimited text directly received from Kafka in every 2 seconds. + Usage: direct_kafka_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first, see + http://kafka.apache.org/documentation.html#quickstart + + and then run the example + `$ bin/spark-submit --jars \ + external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ + examples/src/main/python/streaming/direct_kafka_wordcount.py \ + localhost:9092 test` +""" +from __future__ import print_function + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: direct_kafka_wordcount.py ", file=sys.stderr) + sys.exit(-1) + + sc = SparkContext(appName="PythonStreamingDirectKafkaWordCount") + ssc = StreamingContext(sc, 2) + + brokers, topic = sys.argv[1:] + kvs = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/flume_wordcount.py b/examples/src/main/python/streaming/flume_wordcount.py new file mode 100644 index 0000000000000..c8ea92b61ca6e --- /dev/null +++ b/examples/src/main/python/streaming/flume_wordcount.py @@ -0,0 +1,56 @@ +# +# 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. +# + +r""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: flume_wordcount.py + + To run this on your local machine, you need to setup Flume first, see + https://flume.apache.org/documentation.html + + and then run the example + `$ bin/spark-submit --jars \ + external/flume-assembly/target/scala-*/spark-streaming-flume-assembly-*.jar \ + examples/src/main/python/streaming/flume_wordcount.py \ + localhost 12345 +""" +from __future__ import print_function + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.flume import FlumeUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: flume_wordcount.py ", file=sys.stderr) + sys.exit(-1) + + sc = SparkContext(appName="PythonStreamingFlumeWordCount") + ssc = StreamingContext(sc, 1) + + hostname, port = sys.argv[1:] + kvs = FlumeUtils.createStream(ssc, hostname, int(port)) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py new file mode 100644 index 0000000000000..e9ee08b9fd228 --- /dev/null +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -0,0 +1,56 @@ +# +# 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. +# + +r""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: kafka_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first, see + http://kafka.apache.org/documentation.html#quickstart + + and then run the example + `$ bin/spark-submit --jars \ + external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ + examples/src/main/python/streaming/kafka_wordcount.py \ + localhost:2181 test` +""" +from __future__ import print_function + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: kafka_wordcount.py ", file=sys.stderr) + sys.exit(-1) + + sc = SparkContext(appName="PythonStreamingKafkaWordCount") + ssc = StreamingContext(sc, 1) + + zkQuorum, topic = sys.argv[1:] + kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index 196a110f351ce..a5ed723da47ca 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -114,14 +114,10 @@ write.df(namesAndAges, "namesAndAges.parquet", "parquet") # $example on:manual_load_options_csv$ -df <- read.df("examples/src/main/resources/people.csv", "csv", sep = ";", inferSchema = TRUE, header = TRUE) +df <- read.df("examples/src/main/resources/people.csv", "csv") namesAndAges <- select(df, "name", "age") # $example off:manual_load_options_csv$ -# $example on:manual_save_options_orc$ -df <- read.df("examples/src/main/resources/users.orc", "orc") -write.orc(df, "users_with_options.orc", orc.bloom.filter.columns = "favorite_color", orc.dictionary.key.threshold = 1.0, orc.column.encoding.direct = "name") -# $example off:manual_save_options_orc$ # $example on:direct_sql$ df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") diff --git a/examples/src/main/r/ml/prefixSpan.R b/examples/src/main/r/ml/prefixSpan.R deleted file mode 100644 index 9b70573ffb787..0000000000000 --- a/examples/src/main/r/ml/prefixSpan.R +++ /dev/null @@ -1,42 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# To run this example use -# ./bin/spark-submit examples/src/main/r/ml/prefixSpan.R - -# Load SparkR library into your R session -library(SparkR) - -# Initialize SparkSession -sparkR.session(appName = "SparkR-ML-prefixSpan-example") - -# $example on$ -# Load training data - -df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), - list(list(list(1L), list(3L, 2L), list(1L, 2L))), - list(list(list(1L, 2L), list(5L))), - list(list(list(6L)))), schema = c("sequence")) - -# Finding frequent sequential patterns -frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, - maxLocalProjDBSize = 32000000L) -showDF(frequency) - -# $example off$ - -sparkR.session.stop() \ No newline at end of file diff --git a/examples/src/main/resources/users.orc b/examples/src/main/resources/users.orc deleted file mode 100644 index 12478a5d03c26cb30b35af232a5764e076eaab1f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 547 zcmZ`#Jxc>Y5S`t<-He+fZZ-y&D1Mwxz(Or-4vQp$h^RSIU8P1nQP2b~QLz($LH>cQ z{tF8ce~yK{&c!BZT$uM}cG)*?rrFvo0%&DDwa5Zri!?d488{WO8PdpWktqx{m#HrO)INGvp)yr>5J3sxN1B9l z09)*Y`hL|YB_YBFybP~vd4M;e{Nxp&KdZC<;PMiYxg|pG772wj63;#7=$#qnd}2(&;MDi2oBw}Np|@jC6Rq*6F*-*nT9esXxyz3iqHb5=Cf&h^!ClJ)|Q zIq7{gBx=h%s>CV}haSWKJceUEhKjAnu?l}#tPS?J0iPHx>i*sY9Qt^a{vGU diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 08af3306a96f1..e1f985ece8c06 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -41,8 +41,6 @@ object HdfsTest { val end = System.currentTimeMillis() println(s"Iteration $iter took ${end-start} ms") } - println(s"File contents: ${file.map(_.toString).take(1).mkString(",").slice(0, 10)}") - println(s"Returned length(s) of: ${file.map(_.length).sum().toString}") spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala deleted file mode 100644 index 0a2d31097a024..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.ml - -// scalastyle:off println - -// $example on$ -import org.apache.spark.ml.fpm.PrefixSpan -// $example off$ -import org.apache.spark.sql.SparkSession - -/** - * An example demonstrating PrefixSpan. - * Run with - * {{{ - * bin/run-example ml.PrefixSpanExample - * }}} - */ -object PrefixSpanExample { - - def main(args: Array[String]): Unit = { - val spark = SparkSession - .builder - .appName(s"${this.getClass.getSimpleName}") - .getOrCreate() - import spark.implicits._ - - // $example on$ - val smallTestData = Seq( - Seq(Seq(1, 2), Seq(3)), - Seq(Seq(1), Seq(3, 2), Seq(1, 2)), - Seq(Seq(1, 2), Seq(5)), - Seq(Seq(6))) - - val df = smallTestData.toDF("sequence") - val result = new PrefixSpan() - .setMinSupport(0.5) - .setMaxPatternLength(5) - .setMaxLocalProjDBSize(32000000) - .findFrequentSequentialPatterns(df) - .show() - // $example off$ - - spark.stop() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 18615d9b9b908..7d83aacb11548 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -56,13 +56,6 @@ object SQLDataSourceExample { .option("header", "true") .load("examples/src/main/resources/people.csv") // $example off:manual_load_options_csv$ - // $example on:manual_save_options_orc$ - usersDF.write.format("orc") - .option("orc.bloom.filter.columns", "favorite_color") - .option("orc.dictionary.key.threshold", "1.0") - .option("orc.column.encoding.direct", "name") - .save("users_with_options.orc") - // $example off:manual_save_options_orc$ // $example on:direct_sql$ val sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") diff --git a/external/avro/benchmarks/AvroWriteBenchmark-results.txt b/external/avro/benchmarks/AvroWriteBenchmark-results.txt deleted file mode 100644 index fb2a77333eec5..0000000000000 --- a/external/avro/benchmarks/AvroWriteBenchmark-results.txt +++ /dev/null @@ -1,10 +0,0 @@ -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Avro writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Output Single Int Column 3213 / 3373 4.9 204.3 1.0X -Output Single Double Column 3313 / 3345 4.7 210.7 1.0X -Output Int and String Column 7303 / 7316 2.2 464.3 0.4X -Output Partitions 5309 / 5691 3.0 337.5 0.6X -Output Buckets 7031 / 7557 2.2 447.0 0.5X - diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 5656ac7f38e1b..915769fa708b0 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -17,37 +17,20 @@ package org.apache.spark.sql.avro -import scala.util.control.NonFatal - import org.apache.avro.Schema import org.apache.avro.generic.GenericDatumReader import org.apache.avro.io.{BinaryDecoder, DecoderFactory} -import org.apache.spark.SparkException -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, SpecificInternalRow, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} -import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} -case class AvroDataToCatalyst( - child: Expression, - jsonFormatSchema: String, - options: Map[String, String]) +case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) - override lazy val dataType: DataType = { - val dt = SchemaConverters.toSqlType(avroSchema).dataType - parseMode match { - // With PermissiveMode, the output Catalyst row might contain columns of null values for - // corrupt records, even if some of the columns are not nullable in the user-provided schema. - // Therefore we force the schema to be all nullable here. - case PermissiveMode => dt.asNullable - case _ => dt - } - } + override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType override def nullable: Boolean = true @@ -61,69 +44,24 @@ case class AvroDataToCatalyst( @transient private var result: Any = _ - @transient private lazy val parseMode: ParseMode = { - val mode = AvroOptions(options).parseMode - if (mode != PermissiveMode && mode != FailFastMode) { - throw new AnalysisException(unacceptableModeMessage(mode.name)) - } - mode + override def nullSafeEval(input: Any): Any = { + val binary = input.asInstanceOf[Array[Byte]] + decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) + result = reader.read(result, decoder) + deserializer.deserialize(result) } - private def unacceptableModeMessage(name: String): String = { - s"from_avro() doesn't support the $name mode. " + - s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}." + override def simpleString: String = { + s"from_avro(${child.sql}, ${dataType.simpleString})" } - @transient private lazy val nullResultRow: Any = dataType match { - case st: StructType => - val resultRow = new SpecificInternalRow(st.map(_.dataType)) - for(i <- 0 until st.length) { - resultRow.setNullAt(i) - } - resultRow - - case _ => - null - } - - - override def nullSafeEval(input: Any): Any = { - val binary = input.asInstanceOf[Array[Byte]] - try { - decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) - result = reader.read(result, decoder) - deserializer.deserialize(result) - } catch { - // There could be multiple possible exceptions here, e.g. java.io.IOException, - // AvroRuntimeException, ArrayIndexOutOfBoundsException, etc. - // To make it simple, catch all the exceptions here. - case NonFatal(e) => parseMode match { - case PermissiveMode => nullResultRow - case FailFastMode => - throw new SparkException("Malformed records are detected in record parsing. " + - s"Current parse Mode: ${FailFastMode.name}. To process malformed records as null " + - "result, try setting the option 'mode' as 'PERMISSIVE'.", e) - case _ => - throw new AnalysisException(unacceptableModeMessage(parseMode.name)) - } - } + override def sql: String = { + s"from_avro(${child.sql}, ${dataType.catalogString})" } - override def prettyName: String = "from_avro" - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) - nullSafeCodeGen(ctx, ev, eval => { - val result = ctx.freshName("result") - val dt = CodeGenerator.boxedType(dataType) - s""" - $dt $result = ($dt) $expr.nullSafeEval($eval); - if ($result == null) { - ${ev.isNull} = true; - } else { - ${ev.value} = $result; - } - """ - }) + defineCodeGen(ctx, ev, input => + s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index fec17bfff5424..67f56343b4524 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.avro import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, FailFastMode, ParseMode} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.internal.SQLConf /** @@ -80,17 +79,4 @@ class AvroOptions( val compression: String = { parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec) } - - val parseMode: ParseMode = - parameters.get("mode").map(ParseMode.fromString).getOrElse(FailFastMode) -} - -object AvroOptions { - def apply(parameters: Map[String, String]): AvroOptions = { - val hadoopConf = SparkSession - .getActiveSession - .map(_.sessionState.newHadoopConf()) - .getOrElse(new Configuration()) - new AvroOptions(CaseInsensitiveMap(parameters), hadoopConf) - } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 6ed330d92f5e6..141ff3782adfb 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -52,7 +52,13 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { out.toByteArray } - override def prettyName: String = "to_avro" + override def simpleString: String = { + s"to_avro(${child.sql}, ${child.dataType.simpleString})" + } + + override def sql: String = { + s"to_avro(${child.sql}, ${child.dataType.catalogString})" + } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index 64127af73881b..bd1576587d7fa 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -43,10 +43,6 @@ object SchemaConverters { * This function takes an avro schema and returns a sql schema. */ def toSqlType(avroSchema: Schema): SchemaType = { - toSqlTypeHelper(avroSchema, Set.empty) - } - - def toSqlTypeHelper(avroSchema: Schema, existingRecordNames: Set[String]): SchemaType = { avroSchema.getType match { case INT => avroSchema.getLogicalType match { case _: Date => SchemaType(DateType, nullable = false) @@ -71,28 +67,21 @@ object SchemaConverters { case ENUM => SchemaType(StringType, nullable = false) case RECORD => - if (existingRecordNames.contains(avroSchema.getFullName)) { - throw new IncompatibleSchemaException(s""" - |Found recursive reference in Avro schema, which can not be processed by Spark: - |${avroSchema.toString(true)} - """.stripMargin) - } - val newRecordNames = existingRecordNames + avroSchema.getFullName val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlTypeHelper(f.schema(), newRecordNames) + val schemaType = toSqlType(f.schema()) StructField(f.name, schemaType.dataType, schemaType.nullable) } SchemaType(StructType(fields), nullable = false) case ARRAY => - val schemaType = toSqlTypeHelper(avroSchema.getElementType, existingRecordNames) + val schemaType = toSqlType(avroSchema.getElementType) SchemaType( ArrayType(schemaType.dataType, containsNull = schemaType.nullable), nullable = false) case MAP => - val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames) + val schemaType = toSqlType(avroSchema.getValueType) SchemaType( MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), nullable = false) @@ -102,14 +91,13 @@ object SchemaConverters { // In case of a union with null, eliminate it and make a recursive call val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL) if (remainingUnionTypes.size == 1) { - toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames).copy(nullable = true) + toSqlType(remainingUnionTypes.head).copy(nullable = true) } else { - toSqlTypeHelper(Schema.createUnion(remainingUnionTypes.asJava), existingRecordNames) - .copy(nullable = true) + toSqlType(Schema.createUnion(remainingUnionTypes.asJava)).copy(nullable = true) } } else avroSchema.getTypes.asScala.map(_.getType) match { case Seq(t1) => - toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames) + toSqlType(avroSchema.getTypes.get(0)) case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => SchemaType(LongType, nullable = false) case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => @@ -119,7 +107,7 @@ object SchemaConverters { // This is consistent with the behavior when converting between Avro and Parquet. val fields = avroSchema.getTypes.asScala.zipWithIndex.map { case (s, i) => - val schemaType = toSqlTypeHelper(s, existingRecordNames) + val schemaType = toSqlType(s) // All fields are nullable because only one of them is set at a time StructField(s"member$i", schemaType.dataType, nullable = true) } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala index dee8575c621c8..97f9427f96c55 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala @@ -17,12 +17,9 @@ package org.apache.spark.sql -import scala.collection.JavaConverters._ - import org.apache.spark.annotation.Experimental package object avro { - /** * Converts a binary column of avro format into its corresponding catalyst value. The specified * schema must match the read data, otherwise the behavior is undefined: it may fail or return @@ -34,29 +31,8 @@ package object avro { * @since 2.4.0 */ @Experimental - def from_avro( - data: Column, - jsonFormatSchema: String): Column = { - new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema, Map.empty)) - } - - /** - * Converts a binary column of avro format into its corresponding catalyst value. The specified - * schema must match the read data, otherwise the behavior is undefined: it may fail or return - * arbitrary result. - * - * @param data the binary column. - * @param jsonFormatSchema the avro schema in JSON string format. - * @param options options to control how the Avro record is parsed. - * - * @since 3.0.0 - */ - @Experimental - def from_avro( - data: Column, - jsonFormatSchema: String, - options: java.util.Map[String, String]): Column = { - new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema, options.asScala.toMap)) + def from_avro(data: Column, jsonFormatSchema: String): Column = { + new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema)) } /** diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 80dd4c535ad9c..8334cca6cd8f1 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.avro -import org.apache.avro.Schema - import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{RandomDataGenerator, Row} +import org.apache.spark.sql.RandomDataGenerator import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} -import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -class AvroCatalystDataConversionSuite extends SparkFunSuite - with SharedSQLContext - with ExpressionEvalHelper { +class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalHelper { private def roundTripTest(data: Literal): Unit = { val avroType = SchemaConverters.toAvroType(data.dataType, data.nullable) @@ -38,26 +33,14 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite private def checkResult(data: Literal, schema: String, expected: Any): Unit = { checkEvaluation( - AvroDataToCatalyst(CatalystDataToAvro(data), schema, Map.empty), + AvroDataToCatalyst(CatalystDataToAvro(data), schema), prepareExpectedResult(expected)) } - protected def checkUnsupportedRead(data: Literal, schema: String): Unit = { - val binary = CatalystDataToAvro(data) - intercept[Exception] { - AvroDataToCatalyst(binary, schema, Map("mode" -> "FAILFAST")).eval() - } - - val expected = { - val avroSchema = new Schema.Parser().parse(schema) - SchemaConverters.toSqlType(avroSchema).dataType match { - case st: StructType => Row.fromSeq((0 until st.length).map(_ => null)) - case _ => null - } + private def assertFail(data: Literal, schema: String): Unit = { + intercept[java.io.EOFException] { + AvroDataToCatalyst(CatalystDataToAvro(data), schema).eval() } - - checkEvaluation(AvroDataToCatalyst(binary, schema, Map("mode" -> "PERMISSIVE")), - expected) } private val testingTypes = Seq( @@ -138,7 +121,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite """.stripMargin // When read int as string, avro reader is not able to parse the binary and fail. - checkUnsupportedRead(data, avroTypeJson) + assertFail(data, avroTypeJson) } test("read string as int") { @@ -168,7 +151,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite // When read float data as double, avro reader fails(trying to read 8 bytes while the data have // only 4 bytes). - checkUnsupportedRead(data, avroTypeJson) + assertFail(data, avroTypeJson) } test("read double as float") { @@ -184,29 +167,4 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite // avro reader reads the first 4 bytes of a double as a float, the result is totally undefined. checkResult(data, avroTypeJson, 5.848603E35f) } - - test("Handle unsupported input of record type") { - val actualSchema = StructType(Seq( - StructField("col_0", StringType, false), - StructField("col_1", ShortType, false), - StructField("col_2", DecimalType(8, 4), false), - StructField("col_3", BooleanType, true), - StructField("col_4", DecimalType(38, 38), false))) - - val expectedSchema = StructType(Seq( - StructField("col_0", BinaryType, false), - StructField("col_1", DoubleType, false), - StructField("col_2", DecimalType(18, 4), false), - StructField("col_3", StringType, true), - StructField("col_4", DecimalType(38, 38), false))) - - val seed = scala.util.Random.nextLong() - withClue(s"create random record with seed $seed") { - val data = RandomDataGenerator.randomRow(new scala.util.Random(seed), actualSchema) - val converter = CatalystTypeConverters.createToCatalystConverter(actualSchema) - val input = Literal.create(converter(data), actualSchema) - val avroSchema = SchemaConverters.toAvroType(expectedSchema).toString - checkUnsupportedRead(input, avroSchema) - } - } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index 46a37d8759da1..90a4cd6ccf9dd 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.avro -import scala.collection.JavaConverters._ +import org.apache.avro.Schema -import org.apache.spark.SparkException -import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.functions.struct -import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.test.SharedSQLContext -class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +class AvroFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("roundtrip in to_avro and from_avro - int and string") { @@ -62,33 +61,6 @@ class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUti checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df) } - test("handle invalid input in from_avro") { - val count = 10 - val df = spark.range(count).select(struct('id, 'id.as("id2")).as("struct")) - val avroStructDF = df.select(to_avro('struct).as("avro")) - val avroTypeStruct = s""" - |{ - | "type": "record", - | "name": "struct", - | "fields": [ - | {"name": "col1", "type": "long"}, - | {"name": "col2", "type": "double"} - | ] - |} - """.stripMargin - - intercept[SparkException] { - avroStructDF.select( - from_avro('avro, avroTypeStruct, Map("mode" -> "FAILFAST").asJava)).collect() - } - - // For PERMISSIVE mode, the result should be row of null columns. - val expected = (0 until count).map(_ => Row(Row(null, null))) - checkAnswer( - avroStructDF.select(from_avro('avro, avroTypeStruct, Map("mode" -> "PERMISSIVE").asJava)), - expected) - } - test("roundtrip in to_avro and from_avro - array with null") { val dfOne = Seq(Tuple1(Tuple1(1) :: Nil), Tuple1(null :: Nil)).toDF("array") val avroTypeArrStruct = s""" diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 4fea2cb969446..1e08f7b50b115 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1309,69 +1309,4 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { checkCodec(df, path, "xz") } } - - private def checkSchemaWithRecursiveLoop(avroSchema: String): Unit = { - val message = intercept[IncompatibleSchemaException] { - SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema)) - }.getMessage - - assert(message.contains("Found recursive reference in Avro schema")) - } - - test("Detect recursive loop") { - checkSchemaWithRecursiveLoop(""" - |{ - | "type": "record", - | "name": "LongList", - | "fields" : [ - | {"name": "value", "type": "long"}, // each element has a long - | {"name": "next", "type": ["null", "LongList"]} // optional next element - | ] - |} - """.stripMargin) - - checkSchemaWithRecursiveLoop(""" - |{ - | "type": "record", - | "name": "LongList", - | "fields": [ - | { - | "name": "value", - | "type": { - | "type": "record", - | "name": "foo", - | "fields": [ - | { - | "name": "parent", - | "type": "LongList" - | } - | ] - | } - | } - | ] - |} - """.stripMargin) - - checkSchemaWithRecursiveLoop(""" - |{ - | "type": "record", - | "name": "LongList", - | "fields" : [ - | {"name": "value", "type": "long"}, - | {"name": "array", "type": {"type": "array", "items": "LongList"}} - | ] - |} - """.stripMargin) - - checkSchemaWithRecursiveLoop(""" - |{ - | "type": "record", - | "name": "LongList", - | "fields" : [ - | {"name": "value", "type": "long"}, - | {"name": "map", "type": {"type": "map", "values": "LongList"}} - | ] - |} - """.stripMargin) - } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala index 0b11434757c93..df13b4a1c2d3a 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala @@ -19,19 +19,22 @@ package org.apache.spark.sql.execution.benchmark /** * Benchmark to measure Avro data sources write performance. - * To run this benchmark: - * {{{ - * 1. without sbt: bin/spark-submit --class - * --jars ,, - * , - * - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "avro/test:runMain " - * Results will be written to "benchmarks/AvroWriteBenchmark-results.txt". - * }}} + * Usage: + * 1. with spark-submit: bin/spark-submit --class + * 2. with sbt: build/sbt "avro/test:runMain " */ object AvroWriteBenchmark extends DataSourceWriteBenchmark { - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runDataSourceBenchmark("Avro") + def main(args: Array[String]): Unit = { + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + Avro writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 2481 / 2499 6.3 157.8 1.0X + Output Single Double Column 2705 / 2710 5.8 172.0 0.9X + Output Int and String Column 5539 / 5639 2.8 352.2 0.4X + Output Partitions 4613 / 5004 3.4 293.3 0.5X + Output Buckets 5554 / 5561 2.8 353.1 0.4X + */ + runBenchmark("Avro") } } diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml new file mode 100644 index 0000000000000..002bd6fb7f294 --- /dev/null +++ b/external/flume-assembly/pom.xml @@ -0,0 +1,167 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume-assembly_2.11 + jar + Spark Project External Flume Assembly + http://spark.apache.org/ + + + provided + streaming-flume-assembly + + + + + org.apache.spark + spark-streaming-flume_${scala.binary.version} + ${project.version} + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + commons-net + commons-net + provided + + + com.google.protobuf + protobuf-java + provided + + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.scala-lang + scala-library + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + + + flume-provided + + provided + + + + + diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml new file mode 100644 index 0000000000000..168d9d3b2ae0a --- /dev/null +++ b/external/flume-sink/pom.xml @@ -0,0 +1,140 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume-sink_2.11 + + streaming-flume-sink + + jar + Spark Project External Flume Sink + http://spark.apache.org/ + + + + org.apache.flume + flume-ng-sdk + + + + com.google.guava + guava + + + + org.apache.thrift + libthrift + + + + + org.apache.flume + flume-ng-core + + + com.google.guava + guava + + + org.apache.thrift + libthrift + + + + + org.scala-lang + scala-library + + + + com.google.guava + guava + test + + + + io.netty + netty + 3.4.0.Final + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro + + + + generate-sources + + idl-protocol + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + + + + diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl new file mode 100644 index 0000000000000..8806e863ac7c6 --- /dev/null +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -0,0 +1,40 @@ +/** + * 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. + */ + +@namespace("org.apache.spark.streaming.flume.sink") + +protocol SparkFlumeProtocol { + + record SparkSinkEvent { + map headers; + bytes body; + } + + record EventBatch { + string errorMsg = ""; // If this is empty it is a valid message, else it represents an error + string sequenceNumber; + array events; + } + + EventBatch getEventBatch (int n); + + void ack (string sequenceNumber); + + void nack (string sequenceNumber); +} 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 new file mode 100644 index 0000000000000..09d3fe91e42c8 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Copy of the org.apache.spark.Logging for being used in the Spark Sink. + * The org.apache.spark.Logging is not used so that all of Spark is not brought + * in as a dependency. + */ +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 + + // Method to get or create the logger for this object + protected def log: Logger = { + 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 + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + private def initializeIfNecessary() { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } + } + } + } + + private def initializeLogging() { + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +private[sink] object Logging { + @volatile private var initialized = false + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + // scalastyle:off classforname + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + // scalastyle:on classforname + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala new file mode 100644 index 0000000000000..8050ec357e261 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.util.UUID +import java.util.concurrent.{CountDownLatch, Executors} +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable + +import org.apache.flume.Channel + +/** + * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process + * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. + * @param threads Number of threads to use to process requests. + * @param channel The channel that the sink pulls events from + * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark + * is rolled back. + */ +// Flume forces transactions to be thread-local. So each transaction *must* be committed, or +// rolled back from the thread it was originally created in. So each getEvents call from Spark +// creates a TransactionProcessor which runs in a new thread, in which the transaction is created +// and events are pulled off the channel. Once the events are sent to spark, +// that thread is blocked and the TransactionProcessor is saved in a map, +// until an ACK or NACK comes back or the transaction times out (after the specified timeout). +// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then +// unblocked, at which point the transaction is committed or rolled back. + +private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, + val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { + val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, + new SparkSinkThreadFactory("Spark Sink Processor Thread - %d"))) + // Protected by `sequenceNumberToProcessor` + private val sequenceNumberToProcessor = mutable.HashMap[CharSequence, TransactionProcessor]() + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing a new transaction. To reduce the probability of that happening a random string is + // prepended to the sequence number. Does not change for life of sink + private val seqBase = UUID.randomUUID().toString.substring(0, 8) + private val seqCounter = new AtomicLong(0) + + // Protected by `sequenceNumberToProcessor` + private var stopped = false + + @volatile private var isTest = false + private var testLatch: CountDownLatch = null + + /** + * Returns a bunch of events to Spark over Avro RPC. + * @param n Maximum number of events to return in a batch + * @return [[EventBatch]] instance that has a sequence number and an array of at most n events + */ + override def getEventBatch(n: Int): EventBatch = { + logDebug("Got getEventBatch call from Spark.") + val sequenceNumber = seqBase + seqCounter.incrementAndGet() + createProcessor(sequenceNumber, n) match { + case Some(processor) => + transactionExecutorOpt.foreach(_.submit(processor)) + // Wait until a batch is available - will be an error if error message is non-empty + val batch = processor.getEventBatch + if (SparkSinkUtils.isErrorBatch(batch)) { + // Remove the processor if it is an error batch since no ACK is sent. + removeAndGetProcessor(sequenceNumber) + logWarning("Received an error batch - no events were received from channel! ") + } + batch + case None => + new EventBatch("Spark sink has been stopped!", "", java.util.Collections.emptyList()) + } + } + + private def createProcessor(seq: String, n: Int): Option[TransactionProcessor] = { + sequenceNumberToProcessor.synchronized { + if (!stopped) { + val processor = new TransactionProcessor( + channel, seq, n, transactionTimeout, backOffInterval, this) + sequenceNumberToProcessor.put(seq, processor) + if (isTest) { + processor.countDownWhenBatchAcked(testLatch) + } + Some(processor) + } else { + None + } + } + } + + /** + * Called by Spark to indicate successful commit of a batch + * @param sequenceNumber The sequence number of the event batch that was successful + */ + override def ack(sequenceNumber: CharSequence): Void = { + logDebug("Received Ack for batch with sequence number: " + sequenceNumber) + completeTransaction(sequenceNumber, success = true) + null + } + + /** + * Called by Spark to indicate failed commit of a batch + * @param sequenceNumber The sequence number of the event batch that failed + * @return + */ + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + logInfo("Spark failed to commit transaction. Will reattempt events.") + null + } + + /** + * Helper method to commit or rollback a transaction. + * @param sequenceNumber The sequence number of the batch that was completed + * @param success Whether the batch was successful or not. + */ + private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + removeAndGetProcessor(sequenceNumber).foreach { processor => + processor.batchProcessed(success) + } + } + + /** + * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. + * @param sequenceNumber + * @return An `Option` of the transaction processor for the corresponding batch. Note that this + * instance is no longer tracked and the caller is responsible for that txn processor. + */ + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): + Option[TransactionProcessor] = { + sequenceNumberToProcessor.synchronized { + sequenceNumberToProcessor.remove(sequenceNumber.toString) + } + } + + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } + + /** + * Shuts down the executor used to process transactions. + */ + def shutdown() { + logInfo("Shutting down Spark Avro Callback Handler") + sequenceNumberToProcessor.synchronized { + stopped = true + sequenceNumberToProcessor.values.foreach(_.shutdown()) + } + transactionExecutorOpt.foreach(_.shutdownNow()) + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala new file mode 100644 index 0000000000000..e5b63aa1a77ef --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent._ + +import org.apache.avro.ipc.NettyServer +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.Context +import org.apache.flume.Sink.Status +import org.apache.flume.conf.{Configurable, ConfigurationException} +import org.apache.flume.sink.AbstractSink + +/** + * A sink that uses Avro RPC to run a server that can be polled by Spark's + * FlumePollingInputDStream. This sink has the following configuration parameters: + * + * hostname - The hostname to bind to. Default: 0.0.0.0 + * port - The port to bind to. (No default - mandatory) + * timeout - Time in seconds after which a transaction is rolled back, + * if an ACK is not received from Spark within that time + * threads - Number of threads to use to receive requests from Spark (Default: 10) + * + * This sink is unlike other Flume sinks in the sense that it does not push data, + * instead the process method in this sink simply blocks the SinkRunner the first time it is + * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. + * + * Each time a getEventBatch call comes, creates a transaction and reads events + * from the channel. When enough events are read, the events are sent to the Spark receiver and + * the thread itself is blocked and a reference to it saved off. + * + * When the ack for that batch is received, + * the thread which created the transaction is retrieved and it commits the transaction with the + * channel from the same thread it was originally created in (since Flume transactions are + * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack + * is received within the specified timeout, the transaction is rolled back too. If an ack comes + * after that, it is simply ignored and the events get re-sent. + * + */ + +class SparkSink extends AbstractSink with Logging with Configurable { + + // Size of the pool to use for holding transaction processors. + private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS + + // Timeout for each transaction. If spark does not respond in this much time, + // rollback the transaction + private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT + + // Address info to bind on + private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME + private var port: Int = 0 + + private var backOffInterval: Int = 200 + + // Handle to the server + private var serverOpt: Option[NettyServer] = None + + // The handler that handles the callback from Avro + private var handler: Option[SparkAvroCallbackHandler] = None + + // Latch that blocks off the Flume framework from wasting 1 thread. + private val blockingLatch = new CountDownLatch(1) + + override def start() { + logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + + hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + + transactionTimeout + ".") + handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, + backOffInterval)) + val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) + // Using the constructor that takes specific thread-pools requires bringing in netty + // dependencies which are being excluded in the build. In practice, + // Netty dependencies are already available on the JVM as Flume would have pulled them in. + serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) + serverOpt.foreach { server => + logInfo("Starting Avro server for sink: " + getName) + server.start() + } + super.start() + } + + override def stop() { + logInfo("Stopping Spark Sink: " + getName) + handler.foreach { callbackHandler => + callbackHandler.shutdown() + } + serverOpt.foreach { server => + logInfo("Stopping Avro Server for sink: " + getName) + server.close() + server.join() + } + blockingLatch.countDown() + super.stop() + } + + override def configure(ctx: Context) { + import SparkSinkConfig._ + hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) + port = Option(ctx.getInteger(CONF_PORT)). + getOrElse(throw new ConfigurationException("The port to bind to must be specified")) + poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) + transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) + backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) + logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + + "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + + "backoffInterval: " + backOffInterval) + } + + override def process(): Status = { + // This method is called in a loop by the Flume framework - block it until the sink is + // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is + // being shut down. + logInfo("Blocking Sink Runner, sink will continue to run..") + blockingLatch.await() + Status.BACKOFF + } + + private[flume] def getPort(): Int = { + serverOpt + .map(_.getPort) + .getOrElse( + throw new RuntimeException("Server was not started!") + ) + } + + /** + * Pass in a [[CountDownLatch]] for testing purposes. This batch is counted down when each + * batch is received. The test can simply call await on this latch till the expected number of + * batches are received. + * @param latch + */ + private[flume] def countdownWhenBatchReceived(latch: CountDownLatch) { + handler.foreach(_.countDownWhenBatchAcked(latch)) + } +} + +/** + * Configuration parameters and their defaults. + */ +private[flume] +object SparkSinkConfig { + val THREADS = "threads" + val DEFAULT_THREADS = 10 + + val CONF_TRANSACTION_TIMEOUT = "timeout" + val DEFAULT_TRANSACTION_TIMEOUT = 60 + + val CONF_HOSTNAME = "hostname" + val DEFAULT_HOSTNAME = "0.0.0.0" + + val CONF_PORT = "port" + + val CONF_BACKOFF_INTERVAL = "backoffInterval" + val DEFAULT_BACKOFF_INTERVAL = 200 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala similarity index 57% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala index 7f7ef216cf485..845fc8debda75 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala @@ -14,20 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.features.hadooputils +package org.apache.spark.streaming.flume.sink -import io.fabric8.kubernetes.api.model.Secret +import java.util.concurrent.ThreadFactory +import java.util.concurrent.atomic.AtomicLong /** - * Represents a given configuration of the Kerberos Configuration logic - *

    - * - The secret containing a DT, either previously specified or built on the fly - * - The name of the secret where the DT will be stored - * - The data item-key on the secret which correlates with where the current DT data is stored - * - The Job User's username + * Thread factory that generates daemon threads with a specified name format. */ -private[spark] case class KerberosConfigSpec( - dtSecret: Option[Secret], - dtSecretName: String, - dtSecretItemKey: String, - jobUserName: String) +private[sink] class SparkSinkThreadFactory(nameFormat: String) extends ThreadFactory { + + private val threadId = new AtomicLong() + + override def newThread(r: Runnable): Thread = { + val t = new Thread(r, nameFormat.format(threadId.incrementAndGet())) + t.setDaemon(true) + t + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala similarity index 63% rename from mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala index 268d596fafc54..47c0e294d6b52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala @@ -14,21 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.streaming.flume.sink -package org.apache.spark.ml.r - -import org.apache.spark.ml.fpm.PrefixSpan - -private[r] object PrefixSpanWrapper { - def getPrefixSpan( - minSupport: Double, - maxPatternLength: Int, - maxLocalProjDBSize: Double, - sequenceCol: String): PrefixSpan = { - new PrefixSpan() - .setMinSupport(minSupport) - .setMaxPatternLength(maxPatternLength) - .setMaxLocalProjDBSize(maxLocalProjDBSize.toLong) - .setSequenceCol(sequenceCol) +private[flume] object SparkSinkUtils { + /** + * This method determines if this batch represents an error or not. + * @param batch - The batch to check + * @return - true if the batch represents an error + */ + def isErrorBatch(batch: EventBatch): Boolean = { + !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. } } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala new file mode 100644 index 0000000000000..19e736f016977 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.nio.ByteBuffer +import java.util +import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} + +import scala.util.control.Breaks + +import org.apache.flume.{Channel, Transaction} + +// Flume forces transactions to be thread-local (horrible, I know!) +// So the sink basically spawns a new thread to pull the events out within a transaction. +// The thread fills in the event batch object that is set before the thread is scheduled. +// After filling it in, the thread waits on a condition - which is released only +// when the success message comes back for the specific sequence number for that event batch. +/** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. The thread is blocked until the success call for that transaction + * comes back with an ACK or NACK. + * @param channel The channel from which to pull events + * @param seqNum The sequence number to use for the transaction. Must be unique + * @param maxBatchSize The maximum number of events to process per batch + * @param transactionTimeout Time in seconds after which a transaction must be rolled back + * without waiting for an ACK from Spark + * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts + */ +private class TransactionProcessor(val channel: Channel, val seqNum: String, + var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, + val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { + + // If a real batch is not returned, we always have to return an error batch. + @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", + util.Collections.emptyList()) + + // Synchronization primitives + val batchGeneratedLatch = new CountDownLatch(1) + val batchAckLatch = new CountDownLatch(1) + + // Sanity check to ensure we don't loop like crazy + val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 + + // OK to use volatile, since the change would only make this true (otherwise it will be + // changed to false - we never apply a negation operation to this) - which means the transaction + // succeeded. + @volatile private var batchSuccess = false + + @volatile private var stopped = false + + @volatile private var isTest = false + + private var testLatch: CountDownLatch = null + + // The transaction that this processor would handle + var txOpt: Option[Transaction] = None + + /** + * Get an event batch from the channel. This method will block until a batch of events is + * available from the channel. If no events are available after a large number of attempts of + * polling the channel, this method will return an [[EventBatch]] with a non-empty error message + * + * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a + * maximum of maxBatchSize events + */ + def getEventBatch: EventBatch = { + batchGeneratedLatch.await() + eventBatch + } + + /** + * This method is to be called by the sink when it receives an ACK or NACK from Spark. This + * method is a no-op if it is called after transactionTimeout has expired since + * getEventBatch returned a batch of events. + * @param success True if an ACK was received and the transaction should be committed, else false. + */ + def batchProcessed(success: Boolean) { + logDebug("Batch processed for sequence number: " + seqNum) + batchSuccess = success + batchAckLatch.countDown() + } + + private[flume] def shutdown(): Unit = { + logDebug("Shutting down transaction processor") + stopped = true + } + + /** + * Populates events into the event batch. If the batch cannot be populated, + * this method will not set the events into the event batch, but it sets an error message. + */ + private def populateEvents() { + try { + txOpt = Option(channel.getTransaction) + if(txOpt.isEmpty) { + eventBatch.setErrorMsg("Something went wrong. Channel was " + + "unable to create a transaction!") + } + txOpt.foreach { tx => + tx.begin() + val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) + val loop = new Breaks + var gotEventsInThisTxn = false + var loopCounter: Int = 0 + loop.breakable { + while (!stopped && events.size() < maxBatchSize + && loopCounter < totalAttemptsToRemoveFromChannel) { + loopCounter += 1 + Option(channel.take()) match { + case Some(event) => + events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), + ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true + case None => + if (!gotEventsInThisTxn && !stopped) { + logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + + " the current transaction") + TimeUnit.MILLISECONDS.sleep(backOffInterval) + } else { + loop.break() + } + } + } + } + if (!gotEventsInThisTxn && !stopped) { + val msg = "Tried several times, " + + "but did not get any events from the channel!" + logWarning(msg) + eventBatch.setErrorMsg(msg) + } else { + // At this point, the events are available, so fill them into the event batch + eventBatch = new EventBatch("", seqNum, events) + } + } + } catch { + case interrupted: InterruptedException => + // Don't pollute logs if the InterruptedException came from this being stopped + if (!stopped) { + logWarning("Error while processing transaction.", interrupted) + } + case e: Exception => + logWarning("Error while processing transaction.", e) + eventBatch.setErrorMsg(e.getMessage) + try { + txOpt.foreach { tx => + rollbackAndClose(tx, close = true) + } + } finally { + txOpt = None + } + } finally { + batchGeneratedLatch.countDown() + } + } + + /** + * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in + * this method commits the transaction with the channel. If the ACK does not come in within + * that time or a NACK comes in, this method rolls back the transaction. + */ + private def processAckOrNack() { + batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) + txOpt.foreach { tx => + if (batchSuccess) { + try { + logDebug("Committing transaction") + tx.commit() + } catch { + case e: Exception => + logWarning("Error while attempting to commit transaction. Transaction will be rolled " + + "back", e) + rollbackAndClose(tx, close = false) // tx will be closed later anyway + } finally { + tx.close() + if (isTest) { + testLatch.countDown() + } + } + } else { + logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") + rollbackAndClose(tx, close = true) + // This might have been due to timeout or a NACK. Either way the following call does not + // cause issues. This is required to ensure the TransactionProcessor instance is not leaked + parent.removeAndGetProcessor(seqNum) + } + } + } + + /** + * Helper method to rollback and optionally close a transaction + * @param tx The transaction to rollback + * @param close Whether the transaction should be closed or not after rolling back + */ + private def rollbackAndClose(tx: Transaction, close: Boolean) { + try { + logWarning("Spark was unable to successfully process the events. Transaction is being " + + "rolled back.") + tx.rollback() + } catch { + case e: Exception => + logError("Error rolling back transaction. Rollback may have failed!", e) + } finally { + if (close) { + tx.close() + } + } + } + + /** + * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] + * @param inMap The map to be converted + * @return The converted map + */ + private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + + /** + * When the thread is started it sets as many events as the batch size or less (if enough + * events aren't available) into the eventBatch and object and lets any threads waiting on the + * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, + * or for a specified timeout and commits or rolls back the transaction. + * @return + */ + override def call(): Void = { + populateEvents() + processAckOrNack() + null + } + + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } +} diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..1e3f163f95c09 --- /dev/null +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN + diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala new file mode 100644 index 0000000000000..e8ca1e716394d --- /dev/null +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.net.InetSocketAddress +import java.nio.charset.StandardCharsets +import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} + +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.event.EventBuilder +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +// Due to MNG-1378, there is not a way to include test dependencies transitively. +// We cannot include Spark core tests as a dependency here because it depends on +// Spark core main, which has too many dependencies to require here manually. +// For this reason, we continue to use FunSuite and ignore the scalastyle checks +// that fail if this is detected. +// scalastyle:off +import org.scalatest.FunSuite + +class SparkSinkSuite extends FunSuite { +// scalastyle:on + + val eventsPerBatch = 1000 + val channelCapacity = 5000 + + test("Success with ack") { + val (channel, sink, latch) = initializeChannelAndSink() + channel.start() + sink.start() + + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + client.ack(events.getSequenceNumber) + assert(events.getEvents.size() === 1000) + latch.await(1, TimeUnit.SECONDS) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Failure with nack") { + val (channel, sink, latch) = initializeChannelAndSink() + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + client.nack(events.getSequenceNumber) + latch.await(1, TimeUnit.SECONDS) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Failure with timeout") { + val (channel, sink, latch) = initializeChannelAndSink(Map(SparkSinkConfig + .CONF_TRANSACTION_TIMEOUT -> 1.toString)) + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + latch.await(1, TimeUnit.SECONDS) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Multiple consumers") { + testMultipleConsumers(failSome = false) + } + + test("Multiple consumers with some failures") { + testMultipleConsumers(failSome = true) + } + + def testMultipleConsumers(failSome: Boolean): Unit = { + implicit val executorContext = ExecutionContext + .fromExecutorService(Executors.newFixedThreadPool(5)) + val (channel, sink, latch) = initializeChannelAndSink(Map.empty, 5) + channel.start() + sink.start() + (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + val transceiversAndClients = getTransceiverAndClient(address, 5) + val batchCounter = new CountDownLatch(5) + val counter = new AtomicInteger(0) + transceiversAndClients.foreach(x => { + Future { + val client = x._2 + val events = client.getEventBatch(1000) + if (!failSome || counter.getAndIncrement() % 2 == 0) { + client.ack(events.getSequenceNumber) + } else { + client.nack(events.getSequenceNumber) + throw new RuntimeException("Sending NACK for failure!") + } + events + }.onComplete { + case Success(events) => + assert(events.getEvents.size() === 1000) + batchCounter.countDown() + case Failure(t) => + // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout + batchCounter.countDown() + } + }) + batchCounter.await() + latch.await(1, TimeUnit.SECONDS) + executorContext.shutdown() + if(failSome) { + assert(availableChannelSlots(channel) === 3000) + } else { + assertChannelIsEmpty(channel) + } + sink.stop() + channel.stop() + transceiversAndClients.foreach(x => x._1.close()) + } + + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty, + batchCounter: Int = 1): (MemoryChannel, SparkSink, CountDownLatch) = { + val channel = new MemoryChannel() + val channelContext = new Context() + + channelContext.put("capacity", channelCapacity.toString) + channelContext.put("transactionCapacity", 1000.toString) + channelContext.put("keep-alive", 0.toString) + channelContext.putAll(overrides.asJava) + channel.setName(scala.util.Random.nextString(10)) + channel.configure(channelContext) + + val sink = new SparkSink() + val sinkContext = new Context() + sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") + sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) + sink.configure(sinkContext) + sink.setChannel(channel) + val latch = new CountDownLatch(batchCounter) + sink.countdownWhenBatchReceived(latch) + (channel, sink, latch) + } + + private def putEvents(ch: MemoryChannel, count: Int): Unit = { + val tx = ch.getTransaction + tx.begin() + (1 to count).foreach(x => + ch.put(EventBuilder.withBody(x.toString.getBytes(StandardCharsets.UTF_8)))) + tx.commit() + tx.close() + } + + private def getTransceiverAndClient(address: InetSocketAddress, + count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { + + (1 to count).map(_ => { + lazy val channelFactoryExecutor = Executors.newCachedThreadPool( + new SparkSinkThreadFactory("Flume Receiver Channel Thread - %d")) + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + val transceiver = new NettyTransceiver(address, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + (transceiver, client) + }) + } + + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { + assert(availableChannelSlots(channel) === channelCapacity) + } + + private def availableChannelSlots(channel: MemoryChannel): Int = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] + } +} diff --git a/external/flume/pom.xml b/external/flume/pom.xml new file mode 100644 index 0000000000000..1410ef7f4702d --- /dev/null +++ b/external/flume/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume_2.11 + + streaming-flume + + jar + Spark Project External Flume + http://spark.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-streaming-flume-sink_${scala.binary.version} + ${project.version} + + + org.apache.flume + flume-ng-core + + + org.apache.flume + flume-ng-sdk + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java b/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java new file mode 100644 index 0000000000000..4e3420d9c3b06 --- /dev/null +++ b/external/flume/src/main/java/org/apache/spark/examples/JavaFlumeEventCount.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import org.apache.spark.SparkConf; +import org.apache.spark.streaming.*; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.streaming.flume.FlumeUtils; +import org.apache.spark.streaming.flume.SparkFlumeEvent; + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with an AvroSink in Flume. It will start + * an Avro server on at the request host:port address and listen for requests. + * Your Flume AvroSink should be pointed to this address. + * + * Usage: JavaFlumeEventCount + * is the host the Flume receiver will be started on - a receiver + * creates a server and listens for flume events. + * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount ` + */ +public final class JavaFlumeEventCount { + private JavaFlumeEventCount() { + } + + public static void main(String[] args) throws Exception { + if (args.length != 2) { + System.err.println("Usage: JavaFlumeEventCount "); + System.exit(1); + } + + String host = args[0]; + int port = Integer.parseInt(args[1]); + + Duration batchInterval = new Duration(2000); + SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); + JavaReceiverInputDStream flumeStream = + FlumeUtils.createStream(ssc, host, port); + + flumeStream.count(); + + flumeStream.count().map(in -> "Received " + in + " flume events.").print(); + + ssc.start(); + ssc.awaitTermination(); + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala b/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala new file mode 100644 index 0000000000000..f877f79391b37 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/examples/FlumeEventCount.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with an AvroSink in Flume. It will start + * an Avro server on at the request host:port address and listen for requests. + * Your Flume AvroSink should be pointed to this address. + * + * Usage: FlumeEventCount + * is the host the Flume receiver will be started on - a receiver + * creates a server and listens for flume events. + * is the port the Flume receiver will listen on. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount ` + */ +object FlumeEventCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println( + "Usage: FlumeEventCount ") + System.exit(1) + } + + val Array(host, IntParam(port)) = args + + val batchInterval = Milliseconds(2000) + + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumeEventCount") + val ssc = new StreamingContext(sparkConf, batchInterval) + + // Create a flume stream + val stream = FlumeUtils.createStream(ssc, host, port, StorageLevel.MEMORY_ONLY_SER_2) + + // Print out the count of events received from this server in each batch + stream.count().map(cnt => "Received " + cnt + " flume events." ).print() + + ssc.start() + ssc.awaitTermination() + } +} +// scalastyle:on println diff --git a/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala b/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala new file mode 100644 index 0000000000000..79a4027ca5bde --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/examples/FlumePollingEventCount.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// scalastyle:off println +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with the Spark Sink running in a Flume agent. See + * the Spark Streaming programming guide for more details. + * + * Usage: FlumePollingEventCount + * `host` is the host on which the Spark Sink is running. + * `port` is the port at which the Spark Sink is listening. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` + */ +object FlumePollingEventCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println( + "Usage: FlumePollingEventCount ") + System.exit(1) + } + + val Array(host, IntParam(port)) = args + + val batchInterval = Milliseconds(2000) + + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") + val ssc = new StreamingContext(sparkConf, batchInterval) + + // Create a flume stream that polls the Spark Sink running in a Flume agent + val stream = FlumeUtils.createPollingStream(ssc, host, port) + + // Print out the count of events received from this server in each batch + stream.count().map(cnt => "Received " + cnt + " flume events." ).print() + + ssc.start() + ssc.awaitTermination() + } +} +// scalastyle:on println diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala new file mode 100644 index 0000000000000..07c5286477737 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.io.{ObjectInput, ObjectOutput} + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * A simple object that provides the implementation of readExternal and writeExternal for both + * the wrapper classes for Flume-style Events. + */ +private[streaming] object EventTransformer extends Logging { + def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], + Array[Byte]) = { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.readFully(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.readFully(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) + + headers.put(key, value) + } + (headers, bodyBuff) + } + + def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], + body: Array[Byte]) { + out.writeInt(body.length) + out.write(body) + val numHeaders = headers.size() + out.writeInt(numHeaders) + for ((k, v) <- headers.asScala) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala new file mode 100644 index 0000000000000..8af7c23431063 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume + +import scala.collection.mutable.ArrayBuffer + +import com.google.common.base.Throwables + +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.flume.sink._ + +/** + * This class implements the core functionality of [[FlumePollingReceiver]]. When started it + * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be + * run via a [[java.util.concurrent.Executor]] as this implements [[Runnable]] + * + * @param receiver The receiver that owns this instance. + */ + +private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends Runnable with + Logging { + + def run(): Unit = { + while (!receiver.isStopped()) { + val connection = receiver.getConnections.poll() + val client = connection.client + var batchReceived = false + var seq: CharSequence = null + try { + getBatch(client) match { + case Some(eventBatch) => + batchReceived = true + seq = eventBatch.getSequenceNumber + val events = toSparkFlumeEvents(eventBatch.getEvents) + if (store(events)) { + sendAck(client, seq) + } else { + sendNack(batchReceived, client, seq) + } + case None => + } + } catch { + case e: Exception => + Throwables.getRootCause(e) match { + // If the cause was an InterruptedException, then check if the receiver is stopped - + // if yes, just break out of the loop. Else send a Nack and log a warning. + // In the unlikely case, the cause was not an Exception, + // then just throw it out and exit. + case interrupted: InterruptedException => + if (!receiver.isStopped()) { + logWarning("Interrupted while receiving data from Flume", interrupted) + sendNack(batchReceived, client, seq) + } + case exception: Exception => + logWarning("Error while receiving data from Flume", exception) + sendNack(batchReceived, client, seq) + } + } finally { + receiver.getConnections.add(connection) + } + } + } + + /** + * Gets a batch of events from the specified client. This method does not handle any exceptions + * which will be propagated to the caller. + * @param client Client to get events from + * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] + */ + private def getBatch(client: SparkFlumeProtocol.Callback): Option[EventBatch] = { + val eventBatch = client.getEventBatch(receiver.getMaxBatchSize) + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { + // No error, proceed with processing data + logDebug(s"Received batch of ${eventBatch.getEvents.size} events with sequence " + + s"number: ${eventBatch.getSequenceNumber}") + Some(eventBatch) + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + + eventBatch.getErrorMsg) + None + } + } + + /** + * Store the events in the buffer to Spark. This method will not propagate any exceptions, + * but will propagate any other errors. + * @param buffer The buffer to store + * @return true if the data was stored without any exception being thrown, else false + */ + private def store(buffer: ArrayBuffer[SparkFlumeEvent]): Boolean = { + try { + receiver.store(buffer) + true + } catch { + case e: Exception => + logWarning("Error while attempting to store data received from Flume", e) + false + } + } + + /** + * Send an ack to the client for the sequence number. This method does not handle any exceptions + * which will be propagated to the caller. + * @param client client to send the ack to + * @param seq sequence number of the batch to be ack-ed. + * @return + */ + private def sendAck(client: SparkFlumeProtocol.Callback, seq: CharSequence): Unit = { + logDebug("Sending ack for sequence number: " + seq) + client.ack(seq) + logDebug("Ack sent for sequence number: " + seq) + } + + /** + * This method sends a Nack if a batch was received to the client with the given sequence + * number. Any exceptions thrown by the RPC call is simply thrown out as is - no effort is made + * to handle it. + * @param batchReceived true if a batch was received. If this is false, no nack is sent + * @param client The client to which the nack should be sent + * @param seq The sequence number of the batch that is being nack-ed. + */ + private def sendNack(batchReceived: Boolean, client: SparkFlumeProtocol.Callback, + seq: CharSequence): Unit = { + if (batchReceived) { + // Let Flume know that the events need to be pushed back into the channel. + logDebug("Sending nack for sequence number: " + seq) + client.nack(seq) // If the agent is down, even this could fail and throw + logDebug("Nack sent for sequence number: " + seq) + } + } + + /** + * Utility method to convert [[SparkSinkEvent]]s to [[SparkFlumeEvent]]s + * @param events - Events to convert to SparkFlumeEvents + * @return - The SparkFlumeEvent generated from SparkSinkEvent + */ + private def toSparkFlumeEvents(events: java.util.List[SparkSinkEvent]): + ArrayBuffer[SparkFlumeEvent] = { + // Convert each Flume event to a serializable SparkFlumeEvent + val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) + var j = 0 + while (j < events.size()) { + val event = events.get(j) + val sparkFlumeEvent = new SparkFlumeEvent() + sparkFlumeEvent.event.setBody(event.getBody) + sparkFlumeEvent.event.setHeaders(event.getHeaders) + buffer += sparkFlumeEvent + j += 1 + } + buffer + } +} 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 new file mode 100644 index 0000000000000..13aa817492f7b --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.net.InetSocketAddress +import java.nio.ByteBuffer +import java.util.concurrent.Executors + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import org.apache.avro.ipc.NettyServer +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol, Status} +import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory +import org.jboss.netty.handler.codec.compression._ + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils + +private[streaming] +class FlumeInputDStream[T: ClassTag]( + _ssc: StreamingContext, + host: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean +) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { + + override def getReceiver(): Receiver[SparkFlumeEvent] = { + new FlumeReceiver(host, port, storageLevel, enableDecompression) + } +} + +/** + * A wrapper class for AvroFlumeEvent's with a custom serialization format. + * + * This is necessary because AvroFlumeEvent uses inner data structures + * which are not serializable. + */ +class SparkFlumeEvent() extends Externalizable { + var event: AvroFlumeEvent = new AvroFlumeEvent() + + /* De-serialize from bytes. */ + def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.readFully(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.readFully(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) + + headers.put(key, value) + } + + event.setBody(ByteBuffer.wrap(bodyBuff)) + event.setHeaders(headers) + } + + /* Serialize to bytes. */ + def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { + val body = event.getBody + out.writeInt(body.remaining()) + Utils.writeByteBuffer(body, out) + + val numHeaders = event.getHeaders.size() + out.writeInt(numHeaders) + for ((k, v) <- event.getHeaders.asScala) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} + +private[streaming] object SparkFlumeEvent { + def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { + val event = new SparkFlumeEvent + event.event = in + event + } +} + +/** A simple server that implements Flume's Avro protocol. */ +private[streaming] +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 = { + events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) + Status.OK + } +} + +/** + * A NetworkReceiver which listens for events using the + * Flume Avro interface. + */ +private[streaming] +class FlumeReceiver( + host: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { + + lazy val responder = new SpecificResponder( + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + var server: NettyServer = null + + private def initServer() = { + if (enableDecompression) { + val channelFactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), + Executors.newCachedThreadPool()) + val channelPipelineFactory = new CompressionChannelPipelineFactory() + + new NettyServer( + responder, + new InetSocketAddress(host, port), + channelFactory, + channelPipelineFactory, + null) + } else { + new NettyServer(responder, new InetSocketAddress(host, port)) + } + } + + def onStart() { + synchronized { + if (server == null) { + server = initServer() + server.start() + } else { + logWarning("Flume receiver being asked to start more then once with out close") + } + } + logInfo("Flume receiver started") + } + + def onStop() { + synchronized { + if (server != null) { + server.close() + server = null + } + } + logInfo("Flume receiver stopped") + } + + override def preferredLocation: Option[String] = Option(host) + + /** + * A Netty Pipeline factory that will decompress incoming data from + * and the Netty client and compress data going back to the client. + * + * The compression on the return is required because Flume requires + * a successful response to indicate it can remove the event/batch + * from the configured channel + */ + private[streaming] + class CompressionChannelPipelineFactory extends ChannelPipelineFactory { + def getPipeline(): ChannelPipeline = { + val pipeline = Channels.pipeline() + val encoder = new ZlibEncoder(6) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + pipeline + } + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala new file mode 100644 index 0000000000000..d84e289272c62 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume + + +import java.net.InetSocketAddress +import java.util.concurrent.{Executors, LinkedBlockingQueue, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.flume.sink._ +import org.apache.spark.streaming.receiver.Receiver + +/** + * A `ReceiverInputDStream` that can be used to read data from several Flume agents running + * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. + * @param _ssc Streaming context that will execute this input stream + * @param addresses List of addresses at which SparkSinks are listening + * @param maxBatchSize Maximum size of a batch + * @param parallelism Number of parallel connections to open + * @param storageLevel The storage level to use. + * @tparam T Class type of the object of this stream + */ +private[streaming] class FlumePollingInputDStream[T: ClassTag]( + _ssc: StreamingContext, + val addresses: Seq[InetSocketAddress], + val maxBatchSize: Int, + val parallelism: Int, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { + + override def getReceiver(): Receiver[SparkFlumeEvent] = { + new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) + } +} + +private[streaming] class FlumePollingReceiver( + addresses: Seq[InetSocketAddress], + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { + + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + + lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) + + private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() + + override def onStart(): Unit = { + // Create the connections to each Flume agent. + addresses.foreach { host => + val transceiver = new NettyTransceiver(host, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + connections.add(new FlumeConnection(transceiver, client)) + } + for (i <- 0 until parallelism) { + logInfo("Starting Flume Polling Receiver worker threads..") + // Threads that pull data from Flume. + receiverExecutor.submit(new FlumeBatchFetcher(this)) + } + } + + override def onStop(): Unit = { + logInfo("Shutting down Flume Polling Receiver") + receiverExecutor.shutdown() + // Wait upto a minute for the threads to die + if (!receiverExecutor.awaitTermination(60, TimeUnit.SECONDS)) { + receiverExecutor.shutdownNow() + } + connections.asScala.foreach(_.transceiver.close()) + channelFactory.releaseExternalResources() + } + + private[flume] def getConnections: LinkedBlockingQueue[FlumeConnection] = { + this.connections + } + + private[flume] def getMaxBatchSize: Int = { + this.maxBatchSize + } +} + +/** + * A wrapper around the transceiver and the Avro IPC API. + * @param transceiver The transceiver to use for communication with Flume + * @param client The client that the callbacks are received on. + */ +private[flume] class FlumeConnection(val transceiver: NettyTransceiver, + val client: SparkFlumeProtocol.Callback) + + + diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala new file mode 100644 index 0000000000000..e8623b4766aea --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.net.{InetSocketAddress, ServerSocket} +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets +import java.util.{List => JList} +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.commons.lang3.RandomUtils +import org.apache.flume.source.avro +import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} +import org.jboss.netty.channel.ChannelPipeline +import org.jboss.netty.channel.socket.SocketChannel +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.handler.codec.compression.{ZlibDecoder, ZlibEncoder} + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +/** + * Share codes for Scala and Python unit tests + */ +private[flume] class FlumeTestUtils { + + private var transceiver: NettyTransceiver = null + + private val testPort: Int = findFreePort() + + def getTestPort(): Int = testPort + + /** Find a free port */ + private def findFreePort(): Int = { + val candidatePort = RandomUtils.nextInt(1024, 65536) + Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { + val socket = new ServerSocket(trialPort) + socket.close() + (null, trialPort) + }, new SparkConf())._2 + } + + /** Send data to the flume receiver */ + def writeInput(input: JList[String], enableCompression: Boolean): Unit = { + val testAddress = new InetSocketAddress("localhost", testPort) + + val inputEvents = input.asScala.map { item => + val event = new AvroFlumeEvent + event.setBody(ByteBuffer.wrap(item.getBytes(StandardCharsets.UTF_8))) + event.setHeaders(Collections.singletonMap("test", "header")) + event + } + + // if last attempted transceiver had succeeded, close it + close() + + // Create transceiver + transceiver = { + if (enableCompression) { + new NettyTransceiver(testAddress, new CompressionChannelFactory(6)) + } else { + new NettyTransceiver(testAddress) + } + } + + // Create Avro client with the transceiver + val client = SpecificRequestor.getClient(classOf[AvroSourceProtocol], transceiver) + if (client == null) { + throw new AssertionError("Cannot create client") + } + + // Send data + val status = client.appendBatch(inputEvents.asJava) + if (status != avro.Status.OK) { + throw new AssertionError("Sent events unsuccessfully") + } + } + + def close(): Unit = { + if (transceiver != null) { + transceiver.close() + transceiver = null + } + } + + /** Class to create socket channel with compression */ + private class CompressionChannelFactory(compressionLevel: Int) + extends NioClientSocketChannelFactory { + + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { + val encoder = new ZlibEncoder(compressionLevel) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + super.newChannel(pipeline) + } + } + +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala new file mode 100644 index 0000000000000..707193a957700 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.net.InetSocketAddress +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConverters._ + +import org.apache.spark.api.java.function.PairFunction +import org.apache.spark.api.python.PythonRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +@deprecated("Deprecated without replacement", "2.3.0") +object FlumeUtils { + private val DEFAULT_POLLING_PARALLELISM = 5 + private val DEFAULT_POLLING_BATCH_SIZE = 1000 + + /** + * Create a input stream from a Flume source. + * @param ssc StreamingContext object + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream ( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[SparkFlumeEvent] = { + createStream(ssc, hostname, port, storageLevel, false) + } + + /** + * Create a input stream from a Flume source. + * @param ssc StreamingContext object + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + * @param enableDecompression should netty server decompress input stream + */ + def createStream ( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ): ReceiverInputDStream[SparkFlumeEvent] = { + val inputStream = new FlumeInputDStream[SparkFlumeEvent]( + ssc, hostname, port, storageLevel, enableDecompression) + + inputStream + } + + /** + * Creates a input stream from a Flume source. + * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port, storageLevel, false) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + * @param enableDecompression should netty server decompress input stream + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Address of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): ReceiverInputDStream[SparkFlumeEvent] = { + new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, + parallelism, storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + */ + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running. + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running + * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) + } +} + +/** + * This is a helper class that wraps the methods in FlumeUtils into more Python-friendly class and + * function so that it can be easily instantiated and called from Python's FlumeUtils. + */ +private[flume] class FlumeUtilsPythonHelper { + + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ): JavaPairDStream[Array[Byte], Array[Byte]] = { + val dstream = FlumeUtils.createStream(jssc, hostname, port, storageLevel, enableDecompression) + FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) + } + + def createPollingStream( + jssc: JavaStreamingContext, + hosts: JList[String], + ports: JList[Int], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): JavaPairDStream[Array[Byte], Array[Byte]] = { + assert(hosts.size() == ports.size()) + val addresses = hosts.asScala.zip(ports.asScala).map { + case (host, port) => new InetSocketAddress(host, port) + } + val dstream = FlumeUtils.createPollingStream( + jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) + FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) + } + +} + +private object FlumeUtilsPythonHelper { + + private def stringMapToByteArray(map: JMap[CharSequence, CharSequence]): Array[Byte] = { + val byteStream = new ByteArrayOutputStream() + val output = new DataOutputStream(byteStream) + try { + output.writeInt(map.size) + map.asScala.foreach { kv => + PythonRDD.writeUTF(kv._1.toString, output) + PythonRDD.writeUTF(kv._2.toString, output) + } + byteStream.toByteArray + } + finally { + output.close() + } + } + + private def toByteArrayPairDStream(dstream: JavaReceiverInputDStream[SparkFlumeEvent]): + JavaPairDStream[Array[Byte], Array[Byte]] = { + dstream.mapToPair(new PairFunction[SparkFlumeEvent, Array[Byte], Array[Byte]] { + override def call(sparkEvent: SparkFlumeEvent): (Array[Byte], Array[Byte]) = { + val event = sparkEvent.event + val byteBuffer = event.getBody + val body = new Array[Byte](byteBuffer.remaining()) + byteBuffer.get(body) + (stringMapToByteArray(event.getHeaders), body) + } + }) + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala new file mode 100644 index 0000000000000..a3e784a4f32ee --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.nio.charset.StandardCharsets +import java.util.{Collections, List => JList, Map => JMap} +import java.util.concurrent._ + +import scala.collection.mutable.ArrayBuffer + +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.conf.Configurables +import org.apache.flume.event.EventBuilder + +import org.apache.spark.streaming.flume.sink.{SparkSink, SparkSinkConfig} + +/** + * Share codes for Scala and Python unit tests + */ +private[flume] class PollingFlumeTestUtils { + + private val batchCount = 5 + val eventsPerBatch = 100 + private val totalEventsPerChannel = batchCount * eventsPerBatch + private val channelCapacity = 5000 + + def getTotalEvents: Int = totalEventsPerChannel * channels.size + + private val channels = new ArrayBuffer[MemoryChannel] + private val sinks = new ArrayBuffer[SparkSink] + + /** + * Start a sink and return the port of this sink + */ + def startSingleSink(): Int = { + channels.clear() + sinks.clear() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + + channels += (channel) + sinks += sink + + sink.getPort() + } + + /** + * Start 2 sinks and return the ports + */ + def startMultipleSinks(): Seq[Int] = { + channels.clear() + sinks.clear() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val channel2 = new MemoryChannel() + Configurables.configure(channel2, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + + val sink2 = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink2, context) + sink2.setChannel(channel2) + sink2.start() + + sinks += sink + sinks += sink2 + channels += channel + channels += channel2 + + sinks.map(_.getPort()) + } + + /** + * Send data and wait until all data has been received + */ + def sendDataAndEnsureAllDataHasBeenReceived(): Unit = { + val executor = Executors.newCachedThreadPool() + val executorCompletion = new ExecutorCompletionService[Void](executor) + + val latch = new CountDownLatch(batchCount * channels.size) + sinks.foreach(_.countdownWhenBatchReceived(latch)) + + channels.foreach { channel => + executorCompletion.submit(new TxnSubmitter(channel)) + } + + for (i <- 0 until channels.size) { + executorCompletion.take() + } + + latch.await(15, TimeUnit.SECONDS) // Ensure all data has been received. + } + + /** + * A Python-friendly method to assert the output + */ + def assertOutput( + outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { + require(outputHeaders.size == outputBodies.size) + val eventSize = outputHeaders.size + if (eventSize != totalEventsPerChannel * channels.size) { + throw new AssertionError( + s"Expected ${totalEventsPerChannel * channels.size} events, but was $eventSize") + } + var counter = 0 + for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { + val eventBodyToVerify = s"${channels(k).getName}-$i" + val eventHeaderToVerify: JMap[String, String] = Collections.singletonMap(s"test-$i", "header") + var found = false + var j = 0 + while (j < eventSize && !found) { + if (eventBodyToVerify == outputBodies.get(j) && + eventHeaderToVerify == outputHeaders.get(j)) { + found = true + counter += 1 + } + j += 1 + } + } + if (counter != totalEventsPerChannel * channels.size) { + throw new AssertionError( + s"111 Expected ${totalEventsPerChannel * channels.size} events, but was $counter") + } + } + + def assertChannelsAreEmpty(): Unit = { + channels.foreach(assertChannelIsEmpty) + } + + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != channelCapacity) { + throw new AssertionError(s"Channel ${channel.getName} is not empty") + } + } + + def close(): Unit = { + sinks.foreach(_.stop()) + sinks.clear() + channels.foreach(_.stop()) + channels.clear() + } + + private class TxnSubmitter(channel: MemoryChannel) extends Callable[Void] { + override def call(): Void = { + var t = 0 + for (i <- 0 until batchCount) { + val tx = channel.getTransaction + tx.begin() + for (j <- 0 until eventsPerBatch) { + channel.put(EventBuilder.withBody( + s"${channel.getName}-$t".getBytes(StandardCharsets.UTF_8), + Collections.singletonMap(s"test-$t", "header"))) + t += 1 + } + tx.commit() + tx.close() + Thread.sleep(500) // Allow some time for the events to reach + } + null + } + } + +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java new file mode 100644 index 0000000000000..4a5da226aded3 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ + +/** + * Spark streaming receiver for Flume. + */ +package org.apache.spark.streaming.flume; diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala similarity index 70% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala rename to external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala index 81a11ae9dcdc6..9bfab68c4b8b7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala @@ -14,12 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.k8s.integrationtest.backend.docker -import org.apache.spark.deploy.k8s.integrationtest.TestConstants -import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend +package org.apache.spark.streaming -private[spark] object DockerForDesktopBackend - extends KubeConfigBackend(TestConstants.BACKEND_DOCKER_FOR_DESKTOP) { - -} +/** + * Spark streaming receiver for Flume. + */ +package object flume diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java new file mode 100644 index 0000000000000..cfedb5a042a35 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import org.apache.spark.SparkConf; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.junit.After; +import org.junit.Before; + +public abstract class LocalJavaStreamingContext { + + protected transient JavaStreamingContext ssc; + + @Before + public void setUp() { + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); + ssc.checkpoint("checkpoint"); + } + + @After + public void tearDown() { + ssc.stop(); + ssc = null; + } +} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java new file mode 100644 index 0000000000000..79c5b91654b42 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume; + +import java.net.InetSocketAddress; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.junit.Test; + +public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + // tests the API, does not actually test data receiving + InetSocketAddress[] addresses = new InetSocketAddress[] { + new InetSocketAddress("localhost", 12345) + }; + JavaReceiverInputDStream test1 = + FlumeUtils.createPollingStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( + ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); + } +} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java new file mode 100644 index 0000000000000..ada05f203b6a8 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.junit.Test; + +public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + // tests the API, does not actually test data receiving + JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", + 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", + 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", + 12345, StorageLevel.MEMORY_AND_DISK_SER_2(), false); + } +} diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..fd51f8faf56b9 --- /dev/null +++ b/external/flume/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN + diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala new file mode 100644 index 0000000000000..c97a27ca7c7aa --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.util.Utils + +/** + * This is a output stream just for the testsuites. All the output is collected into a + * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. + * + * The buffer contains a sequence of RDD's, each containing a sequence of items + */ +class TestOutputStream[T: ClassTag](parent: DStream[T], + val output: ConcurrentLinkedQueue[Seq[T]] = new ConcurrentLinkedQueue[Seq[T]]()) + extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { + val collected = rdd.collect() + output.add(collected) + }, false) { + + // This is to clear the output buffer every it is read from a checkpoint + @throws(classOf[IOException]) + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { + ois.defaultReadObject() + output.clear() + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala new file mode 100644 index 0000000000000..9241b13c100f1 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.net.InetSocketAddress +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext, TestOutputStream} +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.util.{ManualClock, Utils} + +class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { + + val maxAttempts = 5 + val batchDuration = Seconds(1) + + @transient private var _sc: SparkContext = _ + + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.clock", "org.apache.spark.util.ManualClock") + + val utils = new PollingFlumeTestUtils + + override def beforeAll(): Unit = { + super.beforeAll() + _sc = new SparkContext(conf) + } + + override def afterAll(): Unit = { + try { + if (_sc != null) { + _sc.stop() + _sc = null + } + } finally { + super.afterAll() + } + } + + test("flume polling test") { + testMultipleTimes(() => testFlumePolling()) + } + + test("flume polling test multiple hosts") { + testMultipleTimes(() => testFlumePollingMultipleHost()) + } + + /** + * Run the given test until no more java.net.BindException's are thrown. + * Do this only up to a certain attempt limit. + */ + private def testMultipleTimes(test: () => Unit): Unit = { + var testPassed = false + var attempt = 0 + while (!testPassed && attempt < maxAttempts) { + try { + test() + testPassed = true + } catch { + case e: Exception if Utils.isBindCollision(e) => + logWarning("Exception when running flume polling test: " + e) + attempt += 1 + } + } + assert(testPassed, s"Test failed after $attempt attempts!") + } + + private def testFlumePolling(): Unit = { + try { + val port = utils.startSingleSink() + + writeAndVerify(Seq(port)) + utils.assertChannelsAreEmpty() + } finally { + utils.close() + } + } + + private def testFlumePollingMultipleHost(): Unit = { + try { + val ports = utils.startMultipleSinks() + writeAndVerify(ports) + utils.assertChannelsAreEmpty() + } finally { + utils.close() + } + } + + def writeAndVerify(sinkPorts: Seq[Int]): Unit = { + // Set up the streaming context and input streams + val ssc = new StreamingContext(_sc, batchDuration) + val addresses = sinkPorts.map(port => new InetSocketAddress("localhost", port)) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + utils.eventsPerBatch, 5) + val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputQueue) + outputStream.register() + + ssc.start() + try { + utils.sendDataAndEnsureAllDataHasBeenReceived() + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + clock.advance(batchDuration.milliseconds) + + // The eventually is required to ensure that all data in the batch has been processed. + eventually(timeout(10 seconds), interval(100 milliseconds)) { + val flattenOutput = outputQueue.asScala.toSeq.flatten + val headers = flattenOutput.map(_.event.getHeaders.asScala.map { + case (key, value) => (key.toString, value.toString) + }).map(_.asJava) + val bodies = flattenOutput.map(e => JavaUtils.bytesToString(e.event.getBody)) + utils.assertOutput(headers.asJava, bodies.asJava) + } + } finally { + // here stop ssc only, but not underlying sparkcontext + ssc.stop(false) + } + } + +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala new file mode 100644 index 0000000000000..97dc7c293cba1 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.jboss.netty.channel.ChannelPipeline +import org.jboss.netty.channel.socket.SocketChannel +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.handler.codec.compression._ +import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} + +class FlumeStreamSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { + val conf = new SparkConf().setMaster("local[4]").setAppName("FlumeStreamSuite") + var ssc: StreamingContext = null + + // ignore palantir/spark + ignore("flume input stream") { + testFlumeStream(testCompression = false) + } + + // ignore palantir/spark + ignore("flume input compressed stream") { + testFlumeStream(testCompression = true) + } + + /** Run test on flume stream */ + private def testFlumeStream(testCompression: Boolean): Unit = { + val input = (1 to 100).map { _.toString } + val utils = new FlumeTestUtils + try { + val outputQueue = startContext(utils.getTestPort(), testCompression) + + eventually(timeout(10 seconds), interval(100 milliseconds)) { + utils.writeInput(input.asJava, testCompression) + } + + eventually(timeout(10 seconds), interval(100 milliseconds)) { + val outputEvents = outputQueue.asScala.toSeq.flatten.map { _.event } + outputEvents.foreach { + event => + event.getHeaders.get("test") should be("header") + } + val output = outputEvents.map(event => JavaUtils.bytesToString(event.getBody)) + output should be (input) + } + } finally { + if (ssc != null) { + ssc.stop() + } + utils.close() + } + } + + /** Setup and start the streaming context */ + private def startContext( + testPort: Int, testCompression: Boolean): (ConcurrentLinkedQueue[Seq[SparkFlumeEvent]]) = { + ssc = new StreamingContext(conf, Milliseconds(200)) + val flumeStream = FlumeUtils.createStream( + ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, testCompression) + val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputQueue) + outputStream.register() + ssc.start() + outputQueue + } + + /** Class to create socket channel with compression */ + private class CompressionChannelFactory(compressionLevel: Int) + extends NioClientSocketChannelFactory { + + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { + val encoder = new ZlibEncoder(compressionLevel) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + super.newChannel(pipeline) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index e1a9191cc5a84..fc09938a43a8c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -52,7 +52,7 @@ private[kafka010] object KafkaWriter extends Logging { s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") } else { - Literal.create(topic.get, StringType) + Literal(topic.get, StringType) } ).dataType match { case StringType => // good diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index 491a9c669bdbe..39c4e3fda1a4b 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -221,7 +221,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .as[(String, String)] val query = startStream(kafka.map(kv => kv._2.toInt)) - val testTime = 20.seconds + val testTime = 1.minutes val startTime = System.currentTimeMillis() // Track the current existing topics val topics = mutable.ArrayBuffer[String]() @@ -252,7 +252,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM testUtils.createTopic(topic, partitions = 1, overwrite = true) logInfo(s"Create topic $topic") case 3 => - Thread.sleep(100) + Thread.sleep(1000) case _ => // Push random messages for (topic <- topics) { val size = Random.nextInt(10) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 5ee76990b54f4..39c2cde7de40d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} +import org.apache.spark.sql.{ForeachWriter, SparkSession} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ @@ -900,7 +900,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testUtils.waitUntilOffsetAppears(topicPartition, 5) - val q = ds.writeStream.foreachBatch { (ds: Dataset[String], epochId: Long) => + val q = ds.writeStream.foreachBatch { (ds, epochId) => if (epochId == 0) { // Send more message before the tasks of the current batch start reading the current batch // data, so that the executors will prefetch messages in the next batch and drop them. In diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index d46c4139011da..81832fbdcd7ec 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -427,7 +427,6 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { .format("kafka") .option("checkpointLocation", checkpointDir.getCanonicalPath) .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.max.block.ms", "5000") .queryName("kafkaStream") withTopic.foreach(stream.option("topic", _)) withOutputMode.foreach(stream.outputMode(_)) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index cf283a5c3e11e..d2100fc5a4aba 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -26,9 +26,11 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging /** + * :: Experimental :: * Choice of how to create and configure underlying Kafka Consumers on driver and executors. * See [[ConsumerStrategies]] to obtain instances. * Kafka 0.10 consumers can require additional, sometimes complex, setup after object @@ -36,6 +38,7 @@ import org.apache.spark.internal.Logging * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ +@Experimental abstract class ConsumerStrategy[K, V] { /** * Kafka @@ -205,10 +208,13 @@ private case class Assign[K, V]( } /** - * Object for obtaining instances of [[ConsumerStrategy]] + * :: Experimental :: + * object for obtaining instances of [[ConsumerStrategy]] */ +@Experimental object ConsumerStrategies { /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -221,6 +227,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def Subscribe[K, V]( topics: Iterable[jl.String], kafkaParams: collection.Map[String, Object], @@ -232,6 +239,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -241,6 +249,7 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def Subscribe[K, V]( topics: Iterable[jl.String], kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -251,6 +260,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -263,6 +273,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def Subscribe[K, V]( topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object], @@ -271,6 +282,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -280,13 +292,14 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def Subscribe[K, V]( topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { new Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, jl.Long]()) } - /** + /** :: Experimental :: * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -300,6 +313,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: collection.Map[String, Object], @@ -310,7 +324,7 @@ object ConsumerStrategies { new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) } - /** + /** :: Experimental :: * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -321,6 +335,7 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -330,7 +345,7 @@ object ConsumerStrategies { ju.Collections.emptyMap[TopicPartition, jl.Long]()) } - /** + /** :: Experimental :: * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -344,6 +359,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: ju.Map[String, Object], @@ -351,7 +367,7 @@ object ConsumerStrategies { new SubscribePattern[K, V](pattern, kafkaParams, offsets) } - /** + /** :: Experimental :: * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -362,6 +378,7 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -372,6 +389,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -384,6 +402,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def Assign[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object], @@ -395,6 +414,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -404,6 +424,7 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def Assign[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -414,6 +435,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -426,6 +448,7 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object], @@ -434,6 +457,7 @@ object ConsumerStrategies { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -443,6 +467,7 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index ba4009ef08856..0acc9b8d2a0cf 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -108,6 +108,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( } } + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") private[streaming] override def name: String = s"Kafka 0.10 direct stream [$id]" protected[streaming] override val checkpointData = diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index 64b6ef6c53b6d..e6bdef04512d5 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -31,10 +32,13 @@ import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingCont import org.apache.spark.streaming.dstream._ /** + * :: Experimental :: * object for constructing Kafka streams and RDDs */ +@Experimental object KafkaUtils extends Logging { /** + * :: Experimental :: * Scala constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -48,6 +52,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createRDD[K, V]( sc: SparkContext, kafkaParams: ju.Map[String, Object], @@ -70,6 +75,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Java constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -83,6 +89,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createRDD[K, V]( jsc: JavaSparkContext, kafkaParams: ju.Map[String, Object], @@ -94,6 +101,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Scala constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number @@ -106,6 +114,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createDirectStream[K, V]( ssc: StreamingContext, locationStrategy: LocationStrategy, @@ -116,6 +125,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Scala constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -127,6 +137,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createDirectStream[K, V]( ssc: StreamingContext, locationStrategy: LocationStrategy, @@ -137,6 +148,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Java constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -146,6 +158,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createDirectStream[K, V]( jssc: JavaStreamingContext, locationStrategy: LocationStrategy, @@ -157,6 +170,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Java constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -168,6 +182,7 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ + @Experimental def createDirectStream[K, V]( jssc: JavaStreamingContext, locationStrategy: LocationStrategy, diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala index b4d9669f08387..c9a8a13f51c32 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -23,14 +23,18 @@ import scala.collection.JavaConverters._ import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: * Choice of how to schedule consumers for a given TopicPartition on an executor. * See [[LocationStrategies]] to obtain instances. * Kafka 0.10 consumers prefetch messages, so it's important for performance * to keep cached consumers on appropriate executors, not recreate them for every partition. * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. */ +@Experimental sealed abstract class LocationStrategy private case object PreferBrokers extends LocationStrategy @@ -40,32 +44,42 @@ private case object PreferConsistent extends LocationStrategy private case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy /** - * Object to obtain instances of [[LocationStrategy]] + * :: Experimental :: object to obtain instances of [[LocationStrategy]] + * */ +@Experimental object LocationStrategies { /** + * :: Experimental :: * Use this only if your executors are on the same nodes as your Kafka brokers. */ + @Experimental def PreferBrokers: LocationStrategy = org.apache.spark.streaming.kafka010.PreferBrokers /** + * :: Experimental :: * Use this in most cases, it will consistently distribute partitions across all executors. */ + @Experimental def PreferConsistent: LocationStrategy = org.apache.spark.streaming.kafka010.PreferConsistent /** + * :: Experimental :: * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ + @Experimental def PreferFixed(hostMap: collection.Map[TopicPartition, String]): LocationStrategy = new PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) /** + * :: Experimental :: * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ + @Experimental def PreferFixed(hostMap: ju.Map[TopicPartition, String]): LocationStrategy = new PreferFixed(hostMap) } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala index 077f02ed28ef7..c66d3c9b8d229 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala @@ -20,6 +20,8 @@ package org.apache.spark.streaming.kafka010 import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental + /** * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the * offset ranges in RDDs generated by the direct Kafka DStream (see @@ -36,6 +38,7 @@ trait HasOffsetRanges { } /** + * :: Experimental :: * Represents any object that can commit a collection of [[OffsetRange]]s. * The direct Kafka DStream implements this interface (see * [[KafkaUtils.createDirectStream]]). @@ -53,20 +56,25 @@ trait HasOffsetRanges { * }) * }}} */ +@Experimental trait CanCommitOffsets { /** + * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. */ + @Experimental def commitAsync(offsetRanges: Array[OffsetRange]): Unit /** + * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. * @param callback Only the most recently provided callback will be used at commit. */ + @Experimental def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala index 77193e2a0c8c9..4017fdbcaf95e 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala @@ -20,11 +20,14 @@ package org.apache.spark.streaming.kafka010 import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkConf +import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * Interface for user-supplied configurations that can't otherwise be set via Spark properties, * because they need tweaking on a per-partition basis, */ +@Experimental abstract class PerPartitionConfig extends Serializable { /** * Maximum rate (number of records per second) at which data will be read diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 93d0d2fd06c78..1974bb1e12e15 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -52,9 +52,6 @@ class DirectKafkaStreamSuite val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) - // Set a timeout of 10 seconds that's going to be used to fetch topics/partitions from kafka. - // Othewise the poll timeout defaults to 2 minutes and causes test cases to run longer. - .set("spark.streaming.kafka.consumer.poll.ms", "10000") private var ssc: StreamingContext = _ private var testDir: File = _ diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 47bc8fec2c80c..561bca5f55370 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -41,10 +41,6 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private val sparkConf = new SparkConf().setMaster("local[4]") .setAppName(this.getClass.getSimpleName) - // Set a timeout of 10 seconds that's going to be used to fetch topics/partitions from kafka. - // Othewise the poll timeout defaults to 2 minutes and causes test cases to run longer. - .set("spark.streaming.kafka.consumer.poll.ms", "10000") - private var sc: SparkContext = _ override def beforeAll { diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml new file mode 100644 index 0000000000000..83edb11f296ab --- /dev/null +++ b/external/kafka-0-8-assembly/pom.xml @@ -0,0 +1,170 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-8-assembly_2.11 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-0-8-assembly + + + + + org.apache.spark + spark-streaming-kafka-0-8_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + provided + + + org.lz4 + lz4-java + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + log4j + log4j + provided + + + org.scala-lang + scala-library + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml new file mode 100644 index 0000000000000..4545877a9d83f --- /dev/null +++ b/external/kafka-0-8/pom.xml @@ -0,0 +1,109 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-kafka-0-8_2.11 + + streaming-kafka-0-8 + + jar + Spark Integration for Kafka 0.8 + http://spark.apache.org/ + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka_${scala.binary.version} + 0.8.2.1 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + org.slf4j + slf4j-simple + + + org.apache.zookeeper + zookeeper + + + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-tags_${scala.binary.version} + + + + + org.apache.spark + spark-tags_${scala.binary.version} + test-jar + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala new file mode 100644 index 0000000000000..89ccbe219cecd --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import org.apache.spark.annotation.Experimental + +/** + * Represents the host and port info for a Kafka broker. + * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID. + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +final class Broker private( + /** Broker's hostname */ + val host: String, + /** Broker's port */ + val port: Int) extends Serializable { + override def equals(obj: Any): Boolean = obj match { + case that: Broker => + this.host == that.host && + this.port == that.port + case _ => false + } + + override def hashCode: Int = { + 41 * (41 + host.hashCode) + port + } + + override def toString(): String = { + s"Broker($host, $port)" + } +} + +/** + * :: Experimental :: + * Companion object that provides methods to create instances of [[Broker]]. + */ +@Experimental +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object Broker { + def create(host: String, port: Int): Broker = + new Broker(host, port) + + def apply(host: String, port: Int): Broker = + new Broker(host, port) + + def unapply(broker: Broker): Option[(String, Int)] = { + if (broker == null) { + None + } else { + Some((broker.host, broker.port)) + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala new file mode 100644 index 0000000000000..9297c39d170c4 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.annotation.tailrec +import scala.collection.mutable +import scala.reflect.ClassTag + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.Decoder + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.scheduler.rate.RateEstimator + +/** + * A stream of [[KafkaRDD]] where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * Starting offsets are specified in advance, + * and this DStream is not responsible for committing offsets, + * so that you can control exactly-once semantics. + * For an easy interface to Kafka-managed offsets, + * see [[KafkaCluster]] + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler function for translating each message into the desired type + */ +private[streaming] +class DirectKafkaInputDStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[K]: ClassTag, + T <: Decoder[V]: ClassTag, + R: ClassTag]( + _ssc: StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R + ) extends InputDStream[R](_ssc) with Logging { + val maxRetries = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRetries", 1) + + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") + private[streaming] override def name: String = s"Kafka direct stream [$id]" + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + + /** + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ + override protected[streaming] val rateController: Option[RateController] = { + if (RateController.isBackPressureEnabled(ssc.conf)) { + Some(new DirectKafkaRateController(id, + RateEstimator.create(ssc.conf, context.graph.batchDuration))) + } else { + None + } + } + + protected val kc = new KafkaCluster(kafkaParams) + + private val maxRateLimitPerPartition: Long = context.sparkContext.getConf.getLong( + "spark.streaming.kafka.maxRatePerPartition", 0) + + private val initialRate = context.sparkContext.getConf.getLong( + "spark.streaming.backpressure.initialRate", 0) + + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicAndPartition, Long]): Option[Map[TopicAndPartition, Long]] = { + + val estimatedRateLimit = rateController.map { x => { + val lr = x.getLatestRate() + if (lr > 0) lr else initialRate + }} + + // calculate a per-partition rate limit based on current lag + val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { + case Some(rate) => + val lagPerPartition = offsets.map { case (tp, offset) => + tp -> Math.max(offset - currentOffsets(tp), 0) + } + val totalLag = lagPerPartition.values.sum + + lagPerPartition.map { case (tp, lag) => + val backpressureRate = lag / totalLag.toDouble * rate + tp -> (if (maxRateLimitPerPartition > 0) { + Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) + } + case None => offsets.map { case (tp, offset) => tp -> maxRateLimitPerPartition.toDouble } + } + + if (effectiveRateLimitPerPartition.values.sum > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some(effectiveRateLimitPerPartition.map { + case (tp, limit) => tp -> Math.max((secsPerBatch * limit).toLong, 1L) + }) + } else { + None + } + } + + protected var currentOffsets = fromOffsets + + @tailrec + protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { + val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) + // Either.fold would confuse @tailrec, do it manually + if (o.isLeft) { + val err = o.left.get.toString + if (retries <= 0) { + throw new SparkException(err) + } else { + logError(err) + Thread.sleep(kc.config.refreshLeaderBackoffMs) + latestLeaderOffsets(retries - 1) + } + } else { + o.right.get + } + } + + // limits the maximum number of messages per partition + protected def clamp( + leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { + val offsets = leaderOffsets.mapValues(lo => lo.offset) + + maxMessagesPerPartition(offsets).map { mmp => + mmp.map { case (tp, messages) => + val lo = leaderOffsets(tp) + tp -> lo.copy(offset = Math.min(currentOffsets(tp) + messages, lo.offset)) + } + }.getOrElse(leaderOffsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { + val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + val rdd = KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + + // Report the record number and metadata of this batch interval to InputInfoTracker. + val offsetRanges = currentOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + } + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) + Some(rdd) + } + + override def start(): Unit = { + } + + def stop(): Unit = { + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } + + override def update(time: Time): Unit = { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time): Unit = { } + + override def restore(): Unit = { + // this is assuming that the topics don't change during execution, which is true currently + val topics = fromOffsets.keySet + val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) + + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) + } + } + } + + /** + * A RateController to retrieve the rate from RateEstimator. + */ + private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala new file mode 100644 index 0000000000000..570affab11853 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -0,0 +1,439 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.util.Properties + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.Random +import scala.util.control.NonFatal + +import kafka.api._ +import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} +import kafka.consumer.{ConsumerConfig, SimpleConsumer} + +import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Convenience methods for interacting with a Kafka cluster. + * See + * A Guide To The Kafka Protocol for more details on individual api calls. + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form + */ +@DeveloperApi +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { + import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} + + // ConsumerConfig isn't serializable + @transient private var _config: SimpleConsumerConfig = null + + def config: SimpleConsumerConfig = this.synchronized { + if (_config == null) { + _config = SimpleConsumerConfig(kafkaParams) + } + _config + } + + def connect(host: String, port: Int): SimpleConsumer = + new SimpleConsumer(host, port, config.socketTimeoutMs, + config.socketReceiveBufferBytes, config.clientId) + + def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] = + findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2)) + + // Metadata api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI + // scalastyle:on + + def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = { + val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, + 0, config.clientId, Seq(topic)) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp: TopicMetadataResponse = consumer.send(req) + resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata => + tm.partitionsMetadata.find(_.partitionId == partition) + }.foreach { pm: PartitionMetadata => + pm.leader.foreach { leader => + return Right((leader.host, leader.port)) + } + } + } + Left(errs) + } + + def findLeaders( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, (String, Int)]] = { + val topics = topicAndPartitions.map(_.topic) + val response = getPartitionMetadata(topics).right + val answer = response.flatMap { tms: Set[TopicMetadata] => + val leaderMap = tms.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.flatMap { pm: PartitionMetadata => + val tp = TopicAndPartition(tm.topic, pm.partitionId) + if (topicAndPartitions(tp)) { + pm.leader.map { l => + tp -> (l.host -> l.port) + } + } else { + None + } + } + }.toMap + + if (leaderMap.keys.size == topicAndPartitions.size) { + Right(leaderMap) + } else { + val missing = topicAndPartitions.diff(leaderMap.keySet) + val err = new Err + err += new SparkException(s"Couldn't find leaders for ${missing}") + Left(err) + } + } + answer + } + + def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { + getPartitionMetadata(topics).right.map { r => + r.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.map { pm: PartitionMetadata => + TopicAndPartition(tm.topic, pm.partitionId) + } + } + } + } + + def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = { + val req = TopicMetadataRequest( + TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp: TopicMetadataResponse = consumer.send(req) + val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) + + if (respErrs.isEmpty) { + return Right(resp.topicsMetadata.toSet) + } else { + respErrs.foreach { m => + val cause = ErrorMapping.exceptionFor(m.errorCode) + val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" + errs += new SparkException(msg, cause) + } + } + } + Left(errs) + } + + // Leader offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI + // scalastyle:on + + def getLatestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime) + + def getEarliestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime) + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { + getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => + r.map { kv => + // mapValues isn't serializable, see SI-7005 + kv._1 -> kv._2.head + } + } + } + + private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] = + m.groupBy(_._2).map { kv => + kv._1 -> kv._2.keys.toSeq + } + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long, + maxNumOffsets: Int + ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = { + findLeaders(topicAndPartitions).right.flatMap { tpToLeader => + val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader) + val leaders = leaderToTp.keys + var result = Map[TopicAndPartition, Seq[LeaderOffset]]() + val errs = new Err + withBrokers(leaders, errs) { consumer => + val partitionsToGetOffsets: Seq[TopicAndPartition] = + leaderToTp((consumer.host, consumer.port)) + val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition => + tp -> PartitionOffsetRequestInfo(before, maxNumOffsets) + }.toMap + val req = OffsetRequest(reqMap) + val resp = consumer.getOffsetsBefore(req) + val respMap = resp.partitionErrorAndOffsets + partitionsToGetOffsets.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { por: PartitionOffsetsResponse => + if (por.error == ErrorMapping.NoError) { + if (por.offsets.nonEmpty) { + result += tp -> por.offsets.map { off => + LeaderOffset(consumer.host, consumer.port, off) + } + } else { + errs += new SparkException( + s"Empty offsets for ${tp}, is ${before} before log beginning?") + } + } else { + errs += ErrorMapping.exceptionFor(por.error) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs += new SparkException(s"Couldn't find leader offsets for ${missing}") + Left(errs) + } + } + + // Consumer offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + // scalastyle:on + + // this 0 here indicates api version, in this case the original ZK backed api. + private def defaultConsumerApiVersion: Short = 0 + + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, Long]] = + getConsumerOffsets(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Long]] = { + getConsumerOffsetMetadata(groupId, topicAndPartitions, consumerApiVersion).right.map { r => + r.map { kv => + kv._1 -> kv._2.offset + } + } + } + + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = + getConsumerOffsetMetadata(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { + var result = Map[TopicAndPartition, OffsetMetadataAndError]() + val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq, consumerApiVersion) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.fetchOffsets(req) + val respMap = resp.requestInfo + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { ome: OffsetMetadataAndError => + if (ome.error == ErrorMapping.NoError) { + result += tp -> ome + } else { + errs += ErrorMapping.exceptionFor(ome.error) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs += new SparkException(s"Couldn't find consumer offsets for ${missing}") + Left(errs) + } + + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsets(groupId, offsets, defaultConsumerApiVersion) + + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Short]] = { + val meta = offsets.map { kv => + kv._1 -> OffsetAndMetadata(kv._2) + } + setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) + } + + /** + * Requires Kafka 0.8.1.1 or later. + * Defaults to the original ZooKeeper backed API version. + */ + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetAndMetadata] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsetMetadata(groupId, metadata, defaultConsumerApiVersion) + + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetAndMetadata], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Short]] = { + var result = Map[TopicAndPartition, Short]() + val req = OffsetCommitRequest(groupId, metadata, consumerApiVersion) + val errs = new Err + val topicAndPartitions = metadata.keySet + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.commitOffsets(req) + val respMap = resp.commitStatus + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { err: Short => + if (err == ErrorMapping.NoError) { + result += tp -> err + } else { + errs += ErrorMapping.exceptionFor(err) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs += new SparkException(s"Couldn't set offsets for ${missing}") + Left(errs) + } + + // Try a call against potentially multiple brokers, accumulating errors + private def withBrokers(brokers: Iterable[(String, Int)], errs: Err) + (fn: SimpleConsumer => Any): Unit = { + brokers.foreach { hp => + var consumer: SimpleConsumer = null + try { + consumer = connect(hp._1, hp._2) + fn(consumer) + } catch { + case NonFatal(e) => + errs += e + } finally { + if (consumer != null) { + consumer.close() + } + } + } + } +} + +@DeveloperApi +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object KafkaCluster { + type Err = ArrayBuffer[Throwable] + + /** If the result is right, return it, otherwise throw SparkException */ + def checkErrors[T](result: Either[Err, T]): T = { + result.fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + } + + case class LeaderOffset(host: String, port: Int, offset: Long) + + /** + * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case. + * Simple consumers connect directly to brokers, but need many of the same configs. + * This subclass won't warn about missing ZK params, or presence of broker params. + */ + class SimpleConsumerConfig private(brokers: String, originalProps: Properties) + extends ConsumerConfig(originalProps) { + val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => + val hpa = hp.split(":") + if (hpa.size == 1) { + throw new SparkException(s"Broker not in the correct format of : [$brokers]") + } + (hpa(0), hpa(1).toInt) + } + } + + object SimpleConsumerConfig { + /** + * Make a consumer config without requiring group.id or zookeeper.connect, + * since communicating with brokers also needs common settings such as timeout + */ + def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = { + // These keys are from other pre-existing kafka configs for specifying brokers, accept either + val brokers = kafkaParams.get("metadata.broker.list") + .orElse(kafkaParams.get("bootstrap.servers")) + .getOrElse(throw new SparkException( + "Must specify metadata.broker.list or bootstrap.servers")) + + val props = new Properties() + kafkaParams.foreach { case (key, value) => + // prevent warnings on parameters ConsumerConfig doesn't know about + if (key != "metadata.broker.list" && key != "bootstrap.servers") { + props.put(key, value) + } + } + + Seq("zookeeper.connect", "group.id").foreach { s => + if (!props.containsKey(s)) { + props.setProperty(s, "") + } + } + + new SimpleConsumerConfig(brokers, props) + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala new file mode 100644 index 0000000000000..7ff3a98ca52cd --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.util.Properties + +import scala.collection.Map +import scala.reflect.{classTag, ClassTag} + +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties + +import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.ThreadUtils + +/** + * Input stream that pulls messages from a Kafka Broker. + * + * @param kafkaParams Map of kafka configuration parameters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ +private[streaming] +class KafkaInputDStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + _ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + useReliableReceiver: Boolean, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[(K, V)](_ssc) with Logging { + + def getReceiver(): Receiver[(K, V)] = { + if (!useReliableReceiver) { + new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) + } else { + new ReliableKafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) + } + } +} + +private[streaming] +class KafkaReceiver[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ) extends Receiver[(K, V)](storageLevel) with Logging { + + // Connection to Kafka + var consumerConnector: ConsumerConnector = null + + def onStop() { + if (consumerConnector != null) { + consumerConnector.shutdown() + consumerConnector = null + } + } + + def onStart() { + + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) + + // Kafka connection properties + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + + val zkConnect = kafkaParams("zookeeper.connect") + // Create the connection to the cluster + logInfo("Connecting to Zookeeper: " + zkConnect) + val consumerConfig = new ConsumerConfig(props) + consumerConnector = Consumer.create(consumerConfig) + logInfo("Connected to " + zkConnect) + + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[V]] + + // Create threads for each topic/message Stream we are listening + val topicMessageStreams = consumerConnector.createMessageStreams( + topics, keyDecoder, valueDecoder) + + val executorPool = + ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") + try { + // Start the messages handler for each partition + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } + } + } finally { + executorPool.shutdown() // Just causes threads to terminate after work is done + } + } + + // Handles Kafka messages + private class MessageHandler(stream: KafkaStream[K, V]) + extends Runnable { + def run() { + logInfo("Starting MessageHandler.") + try { + val streamIterator = stream.iterator() + while (streamIterator.hasNext()) { + val msgAndMetadata = streamIterator.next() + store((msgAndMetadata.key, msgAndMetadata.message)) + } + } catch { + case e: Throwable => reportError("Error handling message; exiting", e) + } + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala new file mode 100644 index 0000000000000..791cf0efaf888 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.{classTag, ClassTag} + +import kafka.api.{FetchRequestBuilder, FetchResponse} +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.consumer.SimpleConsumer +import kafka.message.{MessageAndMetadata, MessageAndOffset} +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties + +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.NextIterator + +/** + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param messageHandler function for translating each message into the desired type + */ +private[kafka] +class KafkaRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag] private[spark] ( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, (String, Int)], + messageHandler: MessageAndMetadata[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) + }.toArray + } + + override def count(): Long = offsetRanges.map(_.count).sum + + override def countApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[R] = { + val nonEmptyPartitions = this.partitions + .map(_.asInstanceOf[KafkaRDDPartition]) + .filter(_.count > 0) + + if (num < 1 || nonEmptyPartitions.isEmpty) { + return new Array[R](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.count) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[R] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, + parts.keys.toArray) + res.foreach(buf ++= _) + buf.toArray + } + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + // TODO is additional hostname resolution necessary here + Seq(part.host) + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = + s"Ran out of messages before reaching ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates that messages may have been lost" + + private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = + s"Got ${itemOffset} > ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates a message may have been skipped" + + override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) + if (part.fromOffset == part.untilOffset) { + logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + */ + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends NextIterator[R] { + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val kc = new KafkaCluster(kafkaParams) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[V]] + val consumer = connectLeader + var requestOffset = part.fromOffset + var iter: Iterator[MessageAndOffset] = null + + // 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) { + kc.connectLeader(part.topic, part.partition).fold( + errs => throw new SparkException( + s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + + errs.mkString("\n")), + consumer => consumer + ) + } else { + kc.connect(part.host, part.port) + } + } + + private def handleFetchErr(resp: FetchResponse) { + if (resp.hasError) { + val err = resp.errorCode(part.topic, part.partition) + if (err == ErrorMapping.LeaderNotAvailableCode || + err == ErrorMapping.NotLeaderForPartitionCode) { + logError(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + + s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") + Thread.sleep(kc.config.refreshLeaderBackoffMs) + } + // Let normal rdd retry sort out reconnect attempts + throw ErrorMapping.exceptionFor(err) + } + } + + private def fetchBatch: Iterator[MessageAndOffset] = { + val req = new FetchRequestBuilder() + .clientId(consumer.clientId) + .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) + .build() + val resp = consumer.fetch(req) + handleFetchErr(resp) + // kafka may return a batch that starts before the requested offset + resp.messageSet(part.topic, part.partition) + .iterator + .dropWhile(_.offset < requestOffset) + } + + override def close(): Unit = { + if (consumer != null) { + consumer.close() + } + } + + override def getNext(): R = { + if (iter == null || !iter.hasNext) { + iter = fetchBatch + } + if (!iter.hasNext) { + assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) + finished = true + null.asInstanceOf[R] + } else { + val item = iter.next() + if (item.offset >= part.untilOffset) { + assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) + finished = true + null.asInstanceOf[R] + } else { + requestOffset = item.nextOffset + messageHandler(new MessageAndMetadata( + part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) + } + } + } + } +} + +private[kafka] +object KafkaRDD { + import KafkaCluster.LeaderOffset + + /** + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the batch + * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) + * ending point of the batch + * @param messageHandler function for translating each message into the desired type + */ + def apply[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, LeaderOffset], + messageHandler: MessageAndMetadata[K, V] => R + ): KafkaRDD[K, V, U, T, R] = { + val leaders = untilOffsets.map { case (tp, lo) => + tp -> ((lo.host, lo.port)) + } + + val offsetRanges = fromOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + }.toArray + + new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala new file mode 100644 index 0000000000000..02917becf0ff9 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import org.apache.spark.Partition + +/** + * @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + * @param host preferred kafka host, i.e. the leader at the time the rdd was created + * @param port preferred kafka host's port + */ +private[kafka] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long, + val host: String, + val port: Int +) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 0000000000000..ef1968585be60 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.{File, IOException} +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.{ZKStringSerializer, ZkUtils} +import org.I0Itec.zkclient.ZkClient +import org.apache.commons.lang3.RandomUtils +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private[kafka] class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + // 0.8.2 server doesn't have a boundPort method, so can't use 0 for a random port + private var brokerPort = RandomUtils.nextInt(1024, 65536) + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, brokerPort) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server.awaitShutdown() + server = null + } + + // On Windows, `logDirs` is left open even after Kafka server above is completely shut down + // in some cases. It leads to test failures on Windows if the directory deletion failure + // throws an exception. + brokerConf.logDirs.foreach { f => + try { + Utils.deleteRecursively(new File(f)) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkClient, topic, partitions, 1) + // wait until metadata is propagated + (0 until partitions).foreach { p => waitUntilMetadataIsPropagated(topic, p) } + } + + /** Single-argument version for backwards compatibility */ + def createTopic(topic: String): Unit = createTopic(topic, 1) + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1") + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + case Some(partitionState) => + val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + + ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && + Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && + leaderAndInSyncReplicas.isr.size >= 1 + + case _ => + false + } + eventually(Time(10000), Time(100)) { + assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + // The directories are not closed even if the ZooKeeper server is shut down. + // Please see ZOOKEEPER-1844, which is fixed in 3.4.6+. It leads to test failures + // on Windows if the directory deletion failure throws an exception. + try { + Utils.deleteRecursively(snapshotDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + try { + Utils.deleteRecursively(logDir) + } catch { + case e: IOException if Utils.isWindows => + logWarning(e.getMessage) + } + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 0000000000000..36082e93707b8 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,806 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.OutputStream +import java.lang.{Integer => JInt, Long => JLong, Number => JNumber} +import java.nio.charset.StandardCharsets +import java.util.{List => JList, Locale, Map => JMap, Set => JSet} + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} +import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} + +import org.apache.spark.{SparkContext, SparkException} +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 + +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object KafkaUtils { + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + ssc: StreamingContext, + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam U type of Kafka message key decoder + * @tparam T type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): ReceiverInputDStream[(K, V)] = { + val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). + * @param groupId The group id for this consumer. + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of DStream + * @param valueTypeClass value type of Dstream + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam U type of Kafka message key decoder + * @tparam T type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) + + implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) + implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) + + createStream[K, V, U, T]( + jssc.ssc, + kafkaParams.asScala.toMap, + Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** get leaders for the given offset ranges, or throw an exception */ + private def leadersForRanges( + kc: KafkaCluster, + offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { + val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet + val leaders = kc.findLeaders(topics) + KafkaCluster.checkErrors(leaders) + } + + /** Make sure offsets are available in kafka, or throw an exception */ + private def checkOffsets( + kc: KafkaCluster, + offsetRanges: Array[OffsetRange]): Unit = { + val topics = offsetRanges.map(_.topicAndPartition).toSet + val result = for { + low <- kc.getEarliestLeaderOffsets(topics).right + high <- kc.getLatestLeaderOffsets(topics).right + } yield { + offsetRanges.filterNot { o => + low(o.topicAndPartition).offset <= o.fromOffset && + o.untilOffset <= high(o.topicAndPartition).offset + } + } + val badRanges = KafkaCluster.checkErrors(result) + if (!badRanges.isEmpty) { + throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) + } + } + + private[kafka] def getFromOffsets( + kc: KafkaCluster, + kafkaParams: Map[String, String], + topics: Set[String] + ): Map[TopicAndPartition, Long] = { + val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase(Locale.ROOT)) + val result = for { + topicPartitions <- kc.getPartitions(topics).right + leaderOffsets <- (if (reset == Some("smallest")) { + kc.getEarliestLeaderOffsets(topicPartitions) + } else { + kc.getLatestLeaderOffsets(topicPartitions) + }).right + } yield { + leaderOffsets.map { case (tp, lo) => + (tp, lo.offset) + } + } + KafkaCluster.checkErrors(result) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange] + ): RDD[(K, V)] = sc.withScope { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val leaders = leadersForRanges(kc, offsetRanges) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[K, V] => R + ): RDD[R] = sc.withScope { + val kc = new KafkaCluster(kafkaParams) + val leaderMap = if (leaders.isEmpty) { + leadersForRanges(kc, offsetRanges) + } else { + // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker + leaders.map { + case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port)) + } + } + val cleanedHandler = sc.clean(messageHandler) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param keyClass type of Kafka message key + * @param valueClass type of Kafka message value + * @param keyDecoderClass type of Kafka message key decoder + * @param valueDecoderClass type of Kafka message value decoder + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange] + ): JavaPairRDD[K, V] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + new JavaPairRDD(createRDD[K, V, KD, VD]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) + } + + /** + * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaRDD[R] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val leaderMap = Map(leaders.asScala.toSeq: _*) + createRDD[K, V, KD, VD, R]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, messageHandler.call(_)) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R + ): InputDStream[R] = { + val cleanedHandler = ssc.sc.clean(messageHandler) + new DirectKafkaInputDStream[K, V, KD, VD, R]( + ssc, kafkaParams, fromOffsets, cleanedHandler) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Set[String] + ): InputDStream[(K, V)] = { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val fromOffsets = getFromOffsets(kc, kafkaParams, topics) + new DirectKafkaInputDStream[K, V, KD, VD, (K, V)]( + ssc, kafkaParams, fromOffsets, messageHandler) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class of the value decoder + * @param recordClass Class of the records in DStream + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaInputDStream[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) + createDirectStream[K, V, KD, VD, R]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(fromOffsets.asScala.mapValues(_.longValue()).toSeq: _*), + cleanedHandler + ) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the `StreamingContext`. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class type of the value decoder + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + createDirectStream[K, V, KD, VD]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Set(topics.asScala.toSeq: _*) + ) + } +} + +/** + * This is a helper class that wraps the KafkaUtils.createStream() into more + * Python-friendly class and function so that it can be easily + * instantiated and called from Python's KafkaUtils. + * + * The zero-arg constructor helps instantiate this class from the Class object + * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() + * takes care of known parameters instead of passing them from Python + */ +private[kafka] class KafkaUtilsPythonHelper { + import KafkaUtilsPythonHelper._ + + def createStream( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = { + KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( + jssc, + classOf[Array[Byte]], + classOf[Array[Byte]], + classOf[DefaultDecoder], + classOf[DefaultDecoder], + kafkaParams, + topics, + storageLevel) + } + + def createRDDWithoutMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler)) + } + + def createRDDWithMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata( + mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + val rdd = createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler). + mapPartitions(picklerIterator) + new JavaRDD(rdd) + } + + private def createRDD[V: ClassTag]( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): RDD[V] = { + KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( + jsc.sc, + kafkaParams.asScala.toMap, + offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), + leaders.asScala.toMap, + messageHandler + ) + } + + def createDirectStreamWithoutMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) + } + + def createDirectStreamWithMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + val stream = createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler). + mapPartitions(picklerIterator) + new JavaDStream(stream) + } + + private def createDirectStream[V: ClassTag]( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JNumber], + messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): DStream[V] = { + + val currentFromOffsets = if (!fromOffsets.isEmpty) { + val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) + if (topicsFromOffsets != topics.asScala.toSet) { + throw new IllegalStateException( + s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + + s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") + } + Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*) + } else { + val kc = new KafkaCluster(Map(kafkaParams.asScala.toSeq: _*)) + KafkaUtils.getFromOffsets( + kc, Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) + } + + KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(currentFromOffsets.toSeq: _*), + messageHandler) + } + + def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + + def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = + TopicAndPartition(topic, partition) + + def createBroker(host: String, port: JInt): Broker = Broker(host, port) + + def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { + val parentRDDs = rdd.getNarrowAncestors + val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) + + require( + kafkaRDDs.length == 1, + "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + + "with this RDD, please call this method only on a Kafka RDD.") + + val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] + kafkaRDD.offsetRanges.toSeq.asJava + } +} + +private object KafkaUtilsPythonHelper { + private var initialized = false + + def initialize(): Unit = { + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new PythonMessageAndMetadataPickler().register() + initialized = true + } + } + } + + initialize() + + def picklerIterator(iter: Iterator[Any]): Iterator[Array[Byte]] = { + new SerDeUtil.AutoBatchedPickler(iter) + } + + case class PythonMessageAndMetadata( + topic: String, + partition: JInt, + offset: JLong, + key: Array[Byte], + message: Array[Byte]) + + class PythonMessageAndMetadataPickler extends IObjectPickler { + private val module = "pyspark.streaming.kafka" + + def register(): Unit = { + Pickler.registerCustomPickler(classOf[PythonMessageAndMetadata], this) + Pickler.registerCustomPickler(this.getClass, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler) { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(StandardCharsets.UTF_8)) + } else { + pickler.save(this) + val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] + out.write(Opcodes.MARK) + pickler.save(msgAndMetaData.topic) + pickler.save(msgAndMetaData.partition) + pickler.save(msgAndMetaData.offset) + pickler.save(msgAndMetaData.key) + pickler.save(msgAndMetaData.message) + out.write(Opcodes.TUPLE) + out.write(Opcodes.REDUCE) + } + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala new file mode 100644 index 0000000000000..6dab5f950d4cd --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import kafka.common.TopicAndPartition + +/** + * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * `KafkaUtils.createDirectStream()`). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** + * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class + * can be created with `OffsetRange.create()`. + * @param topic Kafka topic name + * @param partition Kafka partition id + * @param fromOffset Inclusive starting offset + * @param untilOffset Exclusive ending offset + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +final class OffsetRange private( + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** Kafka TopicAndPartition object, for convenience */ + def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) + + /** Number of messages this OffsetRange refers to */ + def count(): Long = untilOffset - fromOffset + + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" + } + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +/** + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +@deprecated("Update to Kafka 0.10 integration", "2.3.0") +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[kafka] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[kafka] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala new file mode 100644 index 0000000000000..39abe3c3e29d0 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.util.Properties +import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} + +import scala.collection.{mutable, Map} +import scala.reflect.{classTag, ClassTag} + +import kafka.common.TopicAndPartition +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} +import kafka.message.MessageAndMetadata +import kafka.serializer.Decoder +import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils} +import org.I0Itec.zkclient.ZkClient + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} +import org.apache.spark.util.ThreadUtils + +/** + * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. + * It is turned off by default and will be enabled when + * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver + * is that this receiver manages topic-partition/offset itself and updates the offset information + * after data is reliably stored as write-ahead log. Offsets will only be updated when data is + * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. + * + * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset + * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams + * will not take effect. + */ +private[streaming] +class ReliableKafkaReceiver[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel) + extends Receiver[(K, V)](storageLevel) with Logging { + + private val groupId = kafkaParams("group.id") + private val AUTO_OFFSET_COMMIT = "auto.commit.enable" + private def conf = SparkEnv.get.conf + + /** High level consumer to connect to Kafka. */ + private var consumerConnector: ConsumerConnector = null + + /** zkClient to connect to Zookeeper to commit the offsets. */ + private var zkClient: ZkClient = null + + /** + * A HashMap to manage the offset for each topic/partition, this HashMap is called in + * synchronized block, so mutable HashMap will not meet concurrency issue. + */ + private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null + + /** A concurrent HashMap to store the stream block id and related offset snapshot. */ + private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null + + /** + * Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit. + */ + private var blockGenerator: BlockGenerator = null + + /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ + private var messageHandlerThreadPool: ThreadPoolExecutor = null + + override def onStart(): Unit = { + logInfo(s"Starting Kafka Consumer Stream with group: $groupId") + + // Initialize the topic-partition / offset hash map. + topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + + // Initialize the stream block id / offset snapshot hash map. + blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() + + // Initialize the block generator for storing Kafka message. + blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) + + if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { + logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + + "otherwise we will manually set it to false to turn off auto offset commit in Kafka") + } + + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true, + // we have to make sure this property is set to false to turn off auto commit mechanism in + // Kafka. + props.setProperty(AUTO_OFFSET_COMMIT, "false") + + val consumerConfig = new ConsumerConfig(props) + + assert(!consumerConfig.autoCommitEnable) + + logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}") + consumerConnector = Consumer.create(consumerConfig) + logInfo(s"Connected to Zookeeper: ${consumerConfig.zkConnect}") + + zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, + consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) + + messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( + topics.values.sum, "KafkaMessageHandler") + + blockGenerator.start() + + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[K]] + + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[V]] + + val topicMessageStreams = consumerConnector.createMessageStreams( + topics, keyDecoder, valueDecoder) + + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => + messageHandlerThreadPool.submit(new MessageHandler(stream)) + } + } + } + + override def onStop(): Unit = { + if (messageHandlerThreadPool != null) { + messageHandlerThreadPool.shutdown() + messageHandlerThreadPool = null + } + + if (consumerConnector != null) { + consumerConnector.shutdown() + consumerConnector = null + } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (blockGenerator != null) { + blockGenerator.stop() + blockGenerator = null + } + + if (topicPartitionOffsetMap != null) { + topicPartitionOffsetMap.clear() + topicPartitionOffsetMap = null + } + + if (blockOffsetMap != null) { + blockOffsetMap.clear() + blockOffsetMap = null + } + } + + /** Store a Kafka message and the associated metadata as a tuple. */ + private def storeMessageAndMetadata( + msgAndMetadata: MessageAndMetadata[K, V]): Unit = { + val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) + val data = (msgAndMetadata.key, msgAndMetadata.message) + val metadata = (topicAndPartition, msgAndMetadata.offset) + blockGenerator.addDataWithCallback(data, metadata) + } + + /** Update stored offset */ + private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = { + topicPartitionOffsetMap.put(topicAndPartition, offset) + } + + /** + * Remember the current offsets for each topic and partition. This is called when a block is + * generated. + */ + private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { + // Get a snapshot of current offset map and store with related block id. + val offsetSnapshot = topicPartitionOffsetMap.toMap + blockOffsetMap.put(blockId, offsetSnapshot) + topicPartitionOffsetMap.clear() + } + + /** + * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method + * will try a fixed number of times to push the block. If the push fails, the receiver is stopped. + */ + private def storeBlockAndCommitOffset( + blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + var count = 0 + var pushed = false + var exception: Exception = null + while (!pushed && count <= 3) { + try { + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) + pushed = true + } catch { + case ex: Exception => + count += 1 + exception = ex + } + } + if (pushed) { + Option(blockOffsetMap.get(blockId)).foreach(commitOffset) + blockOffsetMap.remove(blockId) + } else { + stop("Error while storing block into Spark", exception) + } + } + + /** + * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's + * metadata schema in Zookeeper. + */ + private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { + if (zkClient == null) { + val thrown = new IllegalStateException("Zookeeper client is unexpectedly null") + stop("Zookeeper client is not initialized before commit offsets to ZK", thrown) + return + } + + for ((topicAndPart, offset) <- offsetMap) { + try { + val topicDirs = new ZKGroupTopicDirs(groupId, topicAndPart.topic) + val zkPath = s"${topicDirs.consumerOffsetDir}/${topicAndPart.partition}" + + ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString) + } catch { + case e: Exception => + logWarning(s"Exception during commit offset $offset for topic" + + s"${topicAndPart.topic}, partition ${topicAndPart.partition}", e) + } + + logInfo(s"Committed offset $offset for topic ${topicAndPart.topic}, " + + s"partition ${topicAndPart.partition}") + } + } + + /** Class to handle received Kafka message. */ + private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { + override def run(): Unit = { + while (!isStopped) { + try { + val streamIterator = stream.iterator() + while (streamIterator.hasNext) { + storeMessageAndMetadata(streamIterator.next) + } + } catch { + case e: Exception => + reportError("Error handling message", e) + } + } + } + } + + /** Class to handle blocks generated by the block generator. */ + private final class GeneratedBlockHandler extends BlockGeneratorListener { + + def onAddData(data: Any, metadata: Any): Unit = { + // Update the offset of the data that was added to the generator + if (metadata != null) { + val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)] + updateOffset(topicAndPartition, offset) + } + } + + def onGenerateBlock(blockId: StreamBlockId): Unit = { + // Remember the offsets of topics/partitions when a block has been generated + rememberBlockOffsets(blockId) + } + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + // Store block and commit the blocks offset + storeBlockAndCommitOffset(blockId, arrayBuffer) + } + + def onError(message: String, throwable: Throwable): Unit = { + reportError(message, throwable) + } + } +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java new file mode 100644 index 0000000000000..2e5ab0fb3bef9 --- /dev/null +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ + +/** + * Kafka receiver for spark streaming. + */ +package org.apache.spark.streaming.kafka; diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala similarity index 54% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala rename to external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala index 3e98d5811d83f..47c5187f8751f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala +++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -15,23 +15,9 @@ * limitations under the License. */ -package org.apache.spark.deploy.k8s.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.HadoopDelegationTokenManager +package org.apache.spark.streaming /** - * Adds Kubernetes-specific functionality to HadoopDelegationTokenManager. + * Kafka receiver for spark streaming, */ -private[spark] class KubernetesHadoopDelegationTokenManager( - _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { - - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled - -} +package object kafka diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java new file mode 100644 index 0000000000000..71404a7331ec4 --- /dev/null +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +import scala.Tuple2; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + final String topic1 = "topic1"; + final String topic2 = "topic2"; + // hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + Set sent = new HashSet<>(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaDStream stream1 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topicToSet(topic1) + ).transformToPair( + // Make sure you can get offset ranges from the rdd + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD rdd) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic1, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + return kv._2(); + } + } + ); + + JavaDStream stream2 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + topicOffsetToMap(topic2, 0L), + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD(new VoidFunction>() { + @Override + public void call(JavaRDD rdd) { + result.addAll(rdd.collect()); + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private static Set topicToSet(String topic) { + Set topicSet = new HashSet<>(); + topicSet.add(topic); + return topicSet; + } + + private static Map topicOffsetToMap(String topic, Long offsetToStart) { + Map topicMap = new HashMap<>(); + topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); + return topicMap; + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic, 1); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java new file mode 100644 index 0000000000000..c41b6297b0481 --- /dev/null +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import scala.Tuple2; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + Map emptyLeaders = new HashMap<>(); + Map leaders = new HashMap<>(); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); + Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); + leaders.put(new TopicAndPartition(topic1, 0), broker); + leaders.put(new TopicAndPartition(topic2, 0), broker); + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + offsetRanges + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + return kv._2(); + } + } + ); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + emptyLeaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + + JavaRDD rdd3 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + leaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + + // just making sure the java user apis work; the scala tests handle logic corner cases + long count1 = rdd1.count(); + long count2 = rdd2.count(); + long count3 = rdd3.count(); + Assert.assertTrue(count1 > 0); + Assert.assertEquals(count1, count2); + Assert.assertEquals(count1, count3); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic, 1); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java new file mode 100644 index 0000000000000..98fe38e826afb --- /dev/null +++ b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; + +import scala.Tuple2; + +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient Random random = new Random(); + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, new Duration(500)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + String topic = "topic1"; + Map topics = new HashMap<>(); + topics.put(topic, 1); + + Map sent = new HashMap<>(); + sent.put("a", 5); + sent.put("b", 3); + sent.put("c", 10); + + kafkaTestUtils.createTopic(topic, 1); + kafkaTestUtils.sendMessages(topic, sent); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress()); + kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaPairDStream stream = KafkaUtils.createStream(ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY_SER()); + + final Map result = Collections.synchronizedMap(new HashMap()); + + JavaDStream words = stream.map( + new Function, String>() { + @Override + public String call(Tuple2 tuple2) { + return tuple2._2(); + } + } + ); + + words.countByValue().foreachRDD(new VoidFunction>() { + @Override + public void call(JavaPairRDD rdd) { + List> ret = rdd.collect(); + for (Tuple2 r : ret) { + if (result.containsKey(r._1())) { + result.put(r._1(), result.get(r._1()) + r._2()); + } else { + result.put(r._1(), r._2()); + } + } + } + } + ); + + ssc.start(); + + long startTime = System.currentTimeMillis(); + AssertionError lastError = null; + while (System.currentTimeMillis() - startTime < 20000) { + try { + Assert.assertEquals(sent.size(), result.size()); + for (Map.Entry e : sent.entrySet()) { + Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); + } + return; + } catch (AssertionError e) { + lastError = e; + } + Thread.sleep(200); + } + if (lastError != null) { + throw lastError; + } else { + Assert.fail("timeout"); + } + } +} diff --git a/external/kafka-0-8/src/test/resources/log4j.properties b/external/kafka-0-8/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..fd51f8faf56b9 --- /dev/null +++ b/external/kafka-0-8/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark_project.jetty=WARN + diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala new file mode 100644 index 0000000000000..3fd37f4c8ac90 --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -0,0 +1,636 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File +import java.util.{ Arrays, UUID } +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.util.Utils + +class DirectKafkaStreamSuite + extends SparkFunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll { + super.beforeAll() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() + } + } + + after { + if (ssc != null) { + ssc.stop(stopSparkContext = true) + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + + test("basic stream receiving with multiple topics and smallest starting offset") { + val topics = Set("basic1", "basic2", "basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val totalSent = data.values.sum * topics.size + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)]() + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + + stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.foreachRDD { rdd => + for (o <- offsetRanges) { + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + val collected = rdd.mapPartitionsWithIndex { (i, iter) => + // For each partition, get size of the range in the partition, + // and the number of items in the partition + val off = offsetRanges(i) + val all = iter.toSeq + val partSize = all.size + val rangeSize = off.untilOffset - off.fromOffset + Iterator((partSize, rangeSize)) + }.collect + + // Verify whether number of elements in each partition + // matches with the corresponding offset range + collected.foreach { case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") + } + } + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + } + ssc.stop() + } + + test("receiving from largest starting offset") { + val topic = "largest" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map { _._2 }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + + test("creating stream by offset") { + val topic = "offset" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String]( + ssc, kafkaParams, Map(topicPartition -> 11L), + (m: MessageAndMetadata[String, String]) => m.message()) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + ssc.stop() + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + // Send data to Kafka and wait for it to be received + def sendData(data: Seq[Int]) { + val strings = data.map { _.toString} + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + val kafkaStream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is ensure all the data is eventually receiving only once + stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } + } + ssc.start() + + // Send some data + for (i <- (1 to 10).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) + } + + ssc.stop() + + // Verify that offset ranges were generated + // Since "offsetRangesAfterStop" will be used to compare with "recoveredOffsetRanges", we should + // collect offset ranges after stopping. Otherwise, because new RDDs keep being generated before + // stopping, we may not be able to get the latest RDDs, then "recoveredOffsetRanges" will + // contain something not in "offsetRangesAfterStop". + val offsetRangesAfterStop = getOffsetRanges(kafkaStream) + assert(offsetRangesAfterStop.size >= 1, "No offset ranges generated") + assert( + offsetRangesAfterStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) + + logInfo("====== RESTARTING ========") + + // Recover context from checkpoints + ssc = new StreamingContext(testDir.getAbsolutePath) + val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] + + // Verify offset ranges have been recovered + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } + assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") + val earlierOffsetRanges = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } + assert( + recoveredOffsetRanges.forall { or => + earlierOffsetRanges.contains((or._1, or._2)) + }, + "Recovered ranges are not the same as the ones generated\n" + + s"recoveredOffsetRanges: $recoveredOffsetRanges\n" + + s"earlierOffsetRanges: $earlierOffsetRanges" + ) + // Restart context, give more data and verify the total at the end + // If the total is write that means each records has been received only once + ssc.start() + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + + eventually(timeout(20 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) + } + ssc.stop() + } + + test("Direct Kafka stream report input information") { + val topic = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)] + + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + + allReceived.asScala.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + + test("maxMessagesPerPartition with backpressure disabled") { + val topic = "maxMessagesPerPartition" + val kafkaStream = getDirectKafkaStream(topic, None) + + val input = Map(TopicAndPartition(topic, 0) -> 50L, TopicAndPartition(topic, 1) -> 50L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + + test("maxMessagesPerPartition with no lag") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) + } + + test("maxMessagesPerPartition respects max rate") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 1000L, TopicAndPartition(topic, 1) -> 1000L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + + test("using rate controller") { + val topic = "backpressure" + val topicPartitions = Set(TopicAndPartition(topic, 0), TopicAndPartition(topic, 1)) + kafkaTestUtils.createTopic(topic, 2) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + val batchIntervalMilliseconds = 100 + val estimator = new ConstantEstimator(100) + val messages = Map("foo" -> 200) + kafkaTestUtils.sendMessages(topic, messages) + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaStream = withClue("Error creating direct stream") { + val kc = new KafkaCluster(kafkaParams) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + val m = kc.getEarliestLeaderOffsets(topicPartitions) + .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) + + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, kafkaParams, m, messageHandler) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + } + } + + val collectedData = new ConcurrentLinkedQueue[Array[String]]() + + // Used for assertion failure messages. + def dataToString: String = + collectedData.asScala.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + collectedData.add(data) + } + + ssc.start() + + // Try different rate limits. + // Wait for arrays of data to appear matching the rate. + Seq(100, 50, 20).foreach { rate => + collectedData.clear() // Empty this buffer on each pass. + estimator.updateRate(rate) // Set a new rate. + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + // Assert that rate estimator values are used to determine maxMessagesPerPartition. + // Funky "-" in message makes the complete assertion message read better. + assert(collectedData.asScala.exists(_.size == expectedSize), + s" - No arrays of size $expectedSize for rate $rate found in $dataToString") + } + } + + ssc.stop() + } + + test("use backpressure.initialRate with backpressure") { + backpressureTest(maxRatePerPartition = 1000, initialRate = 500, maxMessagesPerPartition = 250) + } + + test("backpressure.initialRate should honor maxRatePerPartition") { + backpressureTest(maxRatePerPartition = 300, initialRate = 1000, maxMessagesPerPartition = 150) + } + + private def backpressureTest( + maxRatePerPartition: Int, + initialRate: Int, + maxMessagesPerPartition: Int) = { + + val topic = UUID.randomUUID().toString + val topicPartitions = Set(TopicAndPartition(topic, 0)) + kafkaTestUtils.createTopic(topic, 1) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.backpressure.enabled", "true") + .set("spark.streaming.backpressure.initialRate", initialRate.toString) + .set("spark.streaming.kafka.maxRatePerPartition", maxRatePerPartition.toString) + + val messages = Map("foo" -> 5000) + kafkaTestUtils.sendMessages(topic, messages) + + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + + val kafkaStream = withClue("Error creating direct stream") { + val kc = new KafkaCluster(kafkaParams) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + val m = kc.getEarliestLeaderOffsets(topicPartitions) + .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) + + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, kafkaParams, m, messageHandler) + } + kafkaStream.start() + + val input = Map(new TopicAndPartition(topic, 0) -> 1000L) + + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(new TopicAndPartition(topic, 0) -> maxMessagesPerPartition)) + + kafkaStream.stop() + } + + test("maxMessagesPerPartition with zero offset and rate equal to one") { + val topic = "backpressure" + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + val batchIntervalMilliseconds = 60000 + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + val estimatedRate = 1L + val kafkaStream = withClue("Error creating direct stream") { + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + val fromOffsets = Map( + TopicAndPartition(topic, 0) -> 0L, + TopicAndPartition(topic, 1) -> 0L, + TopicAndPartition(topic, 2) -> 0L, + TopicAndPartition(topic, 3) -> 0L + ) + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, kafkaParams, fromOffsets, messageHandler) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, null) { + override def getLatestRate() = estimatedRate + }) + } + } + + val offsets = Map( + TopicAndPartition(topic, 0) -> 0L, + TopicAndPartition(topic, 1) -> 100L, + TopicAndPartition(topic, 2) -> 200L, + TopicAndPartition(topic, 3) -> 300L + ) + val result = kafkaStream.maxMessagesPerPartition(offsets) + val expected = Map( + TopicAndPartition(topic, 0) -> 1L, + TopicAndPartition(topic, 1) -> 10L, + TopicAndPartition(topic, 2) -> 20L, + TopicAndPartition(topic, 3) -> 30L + ) + assert(result.contains(expected), s"Number of messages per partition must be at least 1") + } + + /** Get the generated offset ranges from the DirectKafkaStream */ + private def getOffsetRanges[K, V]( + kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream.generatedRDDs.mapValues { rdd => + rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges + }.toSeq.sortBy { _._1 } + } + + private def getDirectKafkaStream(topic: String, mockRateController: Option[RateController]) = { + val batchIntervalMilliseconds = 100 + + val sparkConf = new SparkConf() + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val earliestOffsets = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, Map[String, String](), earliestOffsets, messageHandler) { + override protected[streaming] val rateController = mockRateController + } + } +} + +object DirectKafkaStreamSuite { + val total = new AtomicLong(-1L) + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } +} + +private[streaming] class ConstantEstimator(@volatile private var rate: Long) + extends RateEstimator { + + def updateRate(newRate: Long): Unit = { + rate = newRate + } + + def compute( + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) +} + +private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + override def getLatestRate(): Long = rate +} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala new file mode 100644 index 0000000000000..73d528518d486 --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.util.Random + +import kafka.common.TopicAndPartition +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite + +class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { + private val topic = "kcsuitetopic" + Random.nextInt(10000) + private val topicAndPartition = TopicAndPartition(topic, 0) + private var kc: KafkaCluster = null + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll() { + super.beforeAll() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) + } + + override def afterAll() { + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() + } + } + + test("metadata apis") { + val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) + val leaderAddress = s"${leader._1}:${leader._2}" + assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") + + val parts = kc.getPartitions(Set(topic)).right.get + assert(parts(topicAndPartition), "didn't get partitions") + + val err = kc.getPartitions(Set(topic + "BAD")) + assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") + } + + test("leader offset apis") { + val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get + assert(earliest(topicAndPartition).offset === 0, "didn't get earliest") + + val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get + assert(latest(topicAndPartition).offset === 1, "didn't get latest") + } + + test("consumer offset apis") { + val group = "kcsuitegroup" + Random.nextInt(10000) + + val offset = Random.nextInt(10000) + + val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset)) + assert(set.isRight, "didn't set consumer offsets") + + val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get + assert(get(topicAndPartition) === offset, "didn't get consumer offsets") + } +} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala new file mode 100644 index 0000000000000..72f954149fefe --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.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.streaming.kafka + +import scala.util.Random + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.StringDecoder +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ + +class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + + override def beforeAll { + super.beforeAll() + sc = new SparkContext(sparkConf) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + try { + try { + if (sc != null) { + sc.stop + sc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() + } + } + + test("basic usage") { + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, offsetRanges) + + val received = rdd.map(_._2).collect.toSet + assert(received === messages.toSet) + + // size-related method optimizations return sane results + assert(rdd.count === messages.size) + assert(rdd.countApprox(0).getFinalValue.mean === messages.size) + assert(!rdd.isEmpty) + assert(rdd.take(1).size === 1) + assert(rdd.take(1).head._2 === messages.head) + assert(rdd.take(messages.size + 10).size === messages.size) + + val emptyRdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) + + assert(emptyRdd.isEmpty) + + // invalid offset ranges throw exceptions + val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) + intercept[SparkException] { + KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + sc, kafkaParams, badRanges) + } + } + + test("iterator boundary conditions") { + // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd + val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") + + val kc = new KafkaCluster(kafkaParams) + + // this is the "lots of messages" case + kafkaTestUtils.sendMessages(topic, sent) + val sentCount = sent.values.sum + + // rdd defined from leaders after sending messages, should get the number sent + val rdd = getRdd(kc, Set(topic)) + + assert(rdd.isDefined) + + val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + assert(rangeCount === sentCount, "offset range didn't include all sent messages") + assert(rdd.get.count === sentCount, "didn't get all sent messages") + + val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap + + // make sure consumer offsets are committed before the next getRdd call + kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( + err => throw new Exception(err.mkString("\n")), + _ => () + ) + + // this is the "0 messages" case + val rdd2 = getRdd(kc, Set(topic)) + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + + assert(rdd2.isDefined) + assert(rdd2.get.count === 0, "got messages when there shouldn't be any") + + // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above + val rdd3 = getRdd(kc, Set(topic)) + // send lots of messages after rdd was defined, they shouldn't show up + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) + + assert(rdd3.isDefined) + assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") + + } + + // get an rdd from the committed consumer offsets until the latest leader offsets, + private def getRdd(kc: KafkaCluster, topics: Set[String]) = { + val groupId = kc.kafkaParams("group.id") + def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = { + kc.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( + kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs => + offs.map(kv => kv._1 -> kv._2.offset) + } + ) + } + kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => + consumerOffsets(topicPartitions).flatMap { from => + kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until => + val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset) + }.toArray + + val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) => + tp -> Broker(lo.host, lo.port) + }.toMap + + KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String]( + sc, kc.kafkaParams, offsetRanges, leaders, + (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}") + } + } + } + } +} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala new file mode 100644 index 0000000000000..ed130f5990955 --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.serializer.StringDecoder +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Milliseconds, StreamingContext} + +class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfterAll { + private var ssc: StreamingContext = _ + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll(): Unit = { + super.beforeAll() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll(): Unit = { + try { + try { + if (ssc != null) { + ssc.stop() + ssc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() + } + } + + test("Kafka input stream") { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + val topic = "topic1" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, sent) + + val kafkaParams = Map("zookeeper.connect" -> kafkaTestUtils.zkAddress, + "group.id" -> s"test-consumer-${Random.nextInt(10000)}", + "auto.offset.reset" -> "smallest") + + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map(_._2).countByValue().foreachRDD { r => + r.collect().foreach { kv => + result.synchronized { + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } + } + } + + ssc.start() + + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(result.synchronized { sent === result }) + } + ssc.stop() + } +} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala new file mode 100644 index 0000000000000..5da5ea49d77ed --- /dev/null +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +import kafka.serializer.StringDecoder +import kafka.utils.{ZKGroupTopicDirs, ZkUtils} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.util.Utils + +class ReliableKafkaStreamSuite extends SparkFunSuite + with BeforeAndAfterAll with BeforeAndAfter with Eventually { + + private val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") + private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + + private var kafkaTestUtils: KafkaTestUtils = _ + + private var groupId: String = _ + private var kafkaParams: Map[String, String] = _ + private var ssc: StreamingContext = _ + private var tempDirectory: File = null + + override def beforeAll(): Unit = { + super.beforeAll() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + groupId = s"test-consumer-${Random.nextInt(10000)}" + kafkaParams = Map( + "zookeeper.connect" -> kafkaTestUtils.zkAddress, + "group.id" -> groupId, + "auto.offset.reset" -> "smallest" + ) + + tempDirectory = Utils.createTempDir() + } + + override def afterAll(): Unit = { + try { + Utils.deleteRecursively(tempDirectory) + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() + } + } + + before { + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + ssc.checkpoint(tempDirectory.getAbsolutePath) + } + + after { + if (ssc != null) { + ssc.stop(stopSparkContext = true) + ssc = null + } + } + + test("Reliable Kafka input stream with single topic") { + val topic = "test-topic" + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + // Verify whether the offset of this group/topic/partition is 0 before starting. + assert(getCommitOffset(groupId, topic, 0) === None) + + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map { case (k, v) => v }.foreachRDD { r => + val ret = r.collect() + ret.foreach { v => + val count = result.getOrElseUpdate(v, 0) + 1 + result.put(v, count) + } + } + ssc.start() + + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { + // A basic process verification for ReliableKafkaReceiver. + // Verify whether received message number is equal to the sent message number. + assert(data.size === result.size) + // Verify whether each message is the same as the data to be verified. + data.keys.foreach { k => assert(data(k) === result(k).toInt) } + // Verify the offset number whether it is equal to the total message number. + assert(getCommitOffset(groupId, topic, 0) === Some(29L)) + } + } + + test("Reliable Kafka input stream with multiple topics") { + val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) + topics.foreach { case (t, _) => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + + // Before started, verify all the group/topic/partition offsets are 0. + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === None) } + + // Consuming all the data sent to the broker which will potential commit the offsets internally. + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) + stream.foreachRDD(_ => Unit) + ssc.start() + + eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { + // Verify the offset for each group/topic to see whether they are equal to the expected one. + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } + } + } + + + /** Getting partition offset from Zookeeper. */ + private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { + val topicDirs = new ZKGroupTopicDirs(groupId, topic) + val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" + ZkUtils.readDataMaybeNull(kafkaTestUtils.zookeeperClient, zkPath)._1.map(_.toLong) + } +} diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 0aef25329db99..032aca9077e20 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -69,13 +69,6 @@ ${aws.kinesis.producer.version} test - - - com.fasterxml.jackson.dataformat - jackson-dataformat-cbor - ${fasterxml.jackson.version} - org.mockito mockito-core diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 86c42df9e8435..626bde48e1a86 100644 --- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -145,7 +145,7 @@ public static void main(String[] args) throws Exception { // Union all the streams if there is more than 1 stream JavaDStream unionStreams; if (streamsList.size() > 1) { - unionStreams = jssc.union(streamsList.toArray(new JavaDStream[0])); + unionStreams = jssc.union(streamsList.get(0), streamsList.subList(1, streamsList.size())); } else { // Otherwise, just use the 1 stream unionStreams = streamsList.get(0); diff --git a/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index 4fb9f2f849085..0cd795f638870 100644 --- a/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/external/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -import java.util.{Locale, Properties} +import java.util.Properties import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry @@ -64,12 +64,11 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL) val dmax = propertyToOption(GANGLIA_KEY_DMAX).map(_.toInt).getOrElse(GANGLIA_DEFAULT_DMAX) val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE) - .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase(Locale.ROOT))) - .getOrElse(GANGLIA_DEFAULT_MODE) + .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE) val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt) .getOrElse(GANGLIA_DEFAULT_PERIOD) val pollUnit: TimeUnit = propertyToOption(GANGLIA_KEY_UNIT) - .map(u => TimeUnit.valueOf(u.toUpperCase(Locale.ROOT))) + .map(u => TimeUnit.valueOf(u.toUpperCase)) .getOrElse(GANGLIA_DEFAULT_UNIT) MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 5d08d8d92e577..b1e6fbef5332d 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -93,6 +93,7 @@ + org.apache.hadoop hadoop-openstack @@ -135,47 +136,23 @@ httpcore ${hadoop.deps.scope} - - org.apache.hadoop - hadoop-azure - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-common - - - org.codehaus.jackson - jackson-mapper-asl - - - com.fasterxml.jackson.core - jackson-core - - - com.google.guava - guava - - - - hadoop-palantir - - - org.apache.hadoop - hadoop-azure-datalake - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-azure - ${hadoop.deps.scope} - - + hadoop-palantir + + + org.apache.hadoop + hadoop-azure-datalake + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-azure + ${hadoop.deps.scope} + + 1.2.1.spark2 @@ -131,13 +132,12 @@ 1.2.1 10.12.1.1 1.10.1-palantir.3 - 3.11.0 - 2.7.5 - 1.15.0 - 1.5.3 - nohive - 1.6.0 + 3.11.0 + 2.7.5 + 1.15.0 9.4.12.v20180830 + 1.5.2 + nohive 3.1.0 0.9.3 2.4.0 @@ -180,9 +180,9 @@ 3.2.10 1.1.1 2.6 - 3.0.10 2.25.1 2.10 + 3.0.10 3.5.2 3.0.2 0.9.3 @@ -224,6 +224,7 @@ during compilation if the dependency is transivite (e.g. "graphx/" depending on "core/" and needing Hadoop classes in the classpath to compile). --> + compile compile compile compile @@ -616,7 +617,7 @@ org.lz4 lz4-java - 1.5.0 + 1.4.1 com.github.luben @@ -659,7 +660,7 @@ io.netty netty-all - 4.1.30.Final + 4.1.29.Final io.netty @@ -2083,10 +2084,44 @@ ${parquet.test.deps.scope} - com.twitter - parquet-hadoop-bundle - ${hive.parquet.version} - compile + org.apache.flume + flume-ng-core + ${flume.version} + ${flume.deps.scope} + + + io.netty + netty + + + org.apache.flume + flume-ng-auth + + + org.apache.thrift + libthrift + + + org.mortbay.jetty + servlet-api + + + + + org.apache.flume + flume-ng-sdk + ${flume.version} + ${flume.deps.scope} + + + io.netty + netty + + + org.apache.thrift + libthrift + + org.apache.calcite @@ -2883,6 +2918,15 @@ + + flume + + external/flume + external/flume-sink + external/flume-assembly + + + spark-ganglia-lgpl @@ -2914,10 +2958,46 @@ --> - hadoop-2.7 + hadoop-2.6 + + hadoop-2.7 + + 2.7.3 + 2.7.1 + + + + + + org.apache.hadoop + hadoop-azure + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-common + + + org.codehaus.jackson + jackson-mapper-asl + + + com.fasterxml.jackson.core + jackson-core + + + + + + + hadoop-palantir @@ -2988,13 +3068,6 @@ kubernetes resource-managers/kubernetes/core - - - - - - kubernetes-integration-tests - resource-managers/kubernetes/integration-tests @@ -3013,6 +3086,14 @@ + + kafka-0-8 + + external/kafka-0-8 + external/kafka-0-8-assembly + + + test-java-home @@ -3115,6 +3196,9 @@ maven does not complain when they're provided on the command line for a sub-module that does not have them. --> + + flume-provided + hadoop-provided diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bdec475da064e..da66c15e640ef 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,37 +36,6 @@ object MimaExcludes { // Exclude rules for 3.0.x lazy val v30excludes = v24excludes ++ Seq( - // [SPARK-25737] Remove JavaSparkContextVarargsWorkaround - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.api.java.JavaSparkContext"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.union"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.api.java.JavaStreamingContext.union"), - // [SPARK-16775] Remove deprecated accumulator v1 APIs - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulable"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.Accumulator$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulableParam"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$FloatAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$DoubleAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$LongAccumulatorParam$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.AccumulatorParam$IntAccumulatorParam$"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulable"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulableCollection"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.SparkContext.accumulator"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.util.LegacyAccumulatorWrapper"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.intAccumulator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulable"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.doubleAccumulator"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.api.java.JavaSparkContext.accumulator"), - // [SPARK-24109] Remove class SnappyOutputStreamWrapper - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.SnappyCompressionCodec.version"), - // [SPARK-19287] JavaPairRDD flatMapValues requires function returning Iterable, not Iterator - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.api.java.JavaPairRDD.flatMapValues"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.streaming.api.java.JavaPairDStream.flatMapValues"), - // [SPARK-25680] SQL execution listener shouldn't happen on execution thread - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.util.ExecutionListenerManager.clone"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.util.ExecutionListenerManager.this") ) // Exclude rules for 2.4.x @@ -142,6 +111,13 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.defaultParamMap"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$defaultParamMap_="), + // [SPARK-14681][ML] Provide label/impurity stats for spark.ml decision tree nodes + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.LeafNode"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.InternalNode"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this"), + // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f78eddc7f1732..cf66ff4c00ca3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -55,15 +55,17 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) ++ sqlProjects ++ streamingProjects val optionallyEnabledProjects@Seq(kubernetes, mesos, yarn, - sparkGangliaLgpl, streamingKinesisAsl, + streamingFlumeSink, streamingFlume, + streamingKafka, sparkGangliaLgpl, streamingKinesisAsl, dockerIntegrationTests, hadoopCloud, kubernetesIntegrationTests, sparkDist) = Seq("kubernetes", "mesos", "yarn", - "ganglia-lgpl", "streaming-kinesis-asl", + "streaming-flume-sink", "streaming-flume", + "streaming-kafka-0-8", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests", "hadoop-cloud", "kubernetes-integration-tests", "spark-dist-hadoop-palantir").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(networkYarn, streamingKafka010Assembly, streamingKinesisAslAssembly) = - Seq("network-yarn", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = + Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") @@ -378,6 +380,8 @@ object SparkBuild extends PomBuild { /* Hive console settings */ enable(Hive.settings)(hive) + enable(Flume.settings)(streamingFlumeSink) + // SPARK-14738 - Remove docker tests from main Spark build // enable(DockerIntegrationTests.settings)(dockerIntegrationTests) @@ -455,6 +459,9 @@ object Unsafe { ) } +object Flume { + lazy val settings = sbtavro.SbtAvro.avroSettings +} object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: @@ -587,8 +594,11 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly := { - if (moduleName.value.contains("streaming-kafka-0-10-assembly") + if (moduleName.value.contains("streaming-flume-assembly") + || moduleName.value.contains("streaming-kafka-0-8-assembly") + || moduleName.value.contains("streaming-kafka-0-10-assembly") || moduleName.value.contains("streaming-kinesis-asl-assembly")) { + // This must match the same name used in maven (see external/kafka-0-8-assembly/pom.xml) s"${moduleName.value}-${version.value}.jar" } else { s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar" @@ -691,10 +701,10 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, avro), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, kubernetes, + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, kubernetes, yarn, tags, streamingKafka010, sqlKafka010, avro), unidocAllClasspaths in (ScalaUnidoc, unidoc) := { diff --git a/python/README.md b/python/README.md index ffb6147dbee8a..c020d84b01ffd 100644 --- a/python/README.md +++ b/python/README.md @@ -29,4 +29,4 @@ The Python packaging for Spark is not intended to replace all of the other use c ## Python Requirements -At its core PySpark depends on Py4J (currently version 0.10.8.1), but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow). +At its core PySpark depends on Py4J (currently version 0.10.7), but some additional sub-packages have their own extra requirements for some features (including numpy, pandas, and pyarrow). diff --git a/python/docs/Makefile b/python/docs/Makefile index 4767fd9f1c038..1ed1f33af2326 100644 --- a/python/docs/Makefile +++ b/python/docs/Makefile @@ -37,7 +37,7 @@ BUILDDIR ?= _build # 2. If both are set, SPHINXBUILD has a higher priority over SPHINXPYTHON # 3. By default, SPHINXBUILD is used as 'sphinx-build'. -export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.8.1-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.7-src.zip) # Internal variables. PAPEROPT_a4 = -D latex_paper_size=a4 diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst index f7df6438b9169..25ceabac0a541 100644 --- a/python/docs/pyspark.streaming.rst +++ b/python/docs/pyspark.streaming.rst @@ -9,9 +9,23 @@ Module contents :undoc-members: :show-inheritance: +pyspark.streaming.kafka module +------------------------------ +.. automodule:: pyspark.streaming.kafka + :members: + :undoc-members: + :show-inheritance: + pyspark.streaming.kinesis module -------------------------------- .. automodule:: pyspark.streaming.kinesis :members: :undoc-members: :show-inheritance: + +pyspark.streaming.flume.module +------------------------------ +.. automodule:: pyspark.streaming.flume + :members: + :undoc-members: + :show-inheritance: diff --git a/python/lib/py4j-0.10.7-src.zip b/python/lib/py4j-0.10.7-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..128e321078793f41154613544ab7016878d5617e GIT binary patch literal 42437 zcmagEV~j39x2FBHZQHhcw{6?DZQHhX+qP}nHg?>CvkN2FC`PIKdTjbLAX`34>8PU>@i^KjzbxRdnn zB+RF;R|mYL%{iA-2?6_Z zU7!jh>L$P>>K^$-VC%T%aRpM%5D^BJeMvMC>o3P@>OwN1h1~0HkiLZyal6>nXCpXU zA|vKIC@@Rx`$TDd0~n@W!fuGfSr1hzmQ`yWq~XanN(?kcRyPuEDIz*KgU(MqualOl z+=_@y&<$zZq}dBEC{hsI0*=$S>pvAplua&-aa%mWL7=*UfT}-@@j_Ht;JH{)Oxoc! zj|`hX=uZWvajH_HY&+Ntg%azO8S(7ITriEi#GVB6GiJsJ2C+7>xf*q=k?k2_+wg@< zrLWtr*9~{zL4pZOPDKo&Fx&H&n9Vt@r>SiIdBBeXHR#Rq#3|}BsbNL9i##!W3g;ca zEm7DH7{`Vsa3niI_E2NUfk-vT4h8sbC%9C-4{_w!sF2How@~5F^|w)J+V;{nTjry zxbfjP18pM4NSH|&=62wWSMu~AoJD2JQ(!w~R8;EtEe4LZgdq;0JCPRKNkb z|4+q^#QbsUKV+RG8#wC_iq?c+y3ARM7W?$ReEoBD9JI;1O{Q@ZjEIv~e+#W0gzQOC zE2M}>mKN%qu`E6|=QwD$`i9FsSxe6u5+qSJW|>C&z4+i$vZ*ifeOMIEz{9N(<@5%u zHU(khTf6NFod9N?L(#J){c%UN;%WFb+}H1JZP5`ozj1t^Gl%%AwRx2^!BwchFMwJO z>j)lHhpy{d+9m>}bSXMT)464g=JY@mXYtLEI$bDe&c&;-Dr3RTowZjsf;4~J+&bvd zB1hVO=E=Zxsz%`V5?RzT*3ytwP6Z$0Uv5UMyA<_SNTHD_@?3pLboUCx?J;{b$ncaB zAJ3p-s%jf&eXSSD>GUQOPG=S0gmP@1yswUphw6AS)>U~w;xifo9~;?!_}3-;uC;$T zAozDC)|axyy*?gEc^&DOlNL>`K@bIml~+c3U}w`s}ngJPJQQra!J< z7EYZxt$HoA3lI&vO}PD0z~bZL1|~dhlY%zGIDzb4}z5eUA zEN3yJ9;b@VLUS`u02WueBo_1QA@c(2ZeZJf*tKG=@V|2)TO?2B$NBAs84~~?vH$>3 z|2qeaZ7fagT%1jv+)SPRI}6Y_&$Rw){U2Fy&1>zv#hJACO;bM^Uf_ndGCO0pl2+-X z%BMZyvDV<+%F@xXL6?<8m}wlfoTS3jS@C(hh5#6jZu}={b~{u~9Vr3~m%q1TL0C6@ zwbzIKXqA{=75V1ydHSq>H?d&A=%G&68j5=DyBg)wLv^f`W=q1{FrF^0IfUS$J_Xwq z<=T=EnlbUBIQc+{;y=S@df3%%*QQCdYx?-voihLYUE5P9m6%>r&CwFlHzqEAMh2L~ z31d2})$Ev$J=3H*mVf-L){#`4I{mA?vCoqa_O+>EN`eyr$yjXB*urzD89S0)85V~n z=S%uP2CFZ9No!DFI&-N;Utg4U9q!zM4;(x10W2PX2h~IgeXd$V);1bWVrA+VHP$VF z+n?{$tJ>cX8ptEojgLonWkt8>(qY@yBBMNPr`l$ueF{lV=hQE)#!8YnlT`_jplajf zQlg8tY5}gk`YX%kpFkIUEn&1dxGa8{mUo43n>%wto6Ua?5zp*hexfvE+w{20qx7-R3IO(1)UnqZec6Dk7!00=$_SwuKdc?*pmp5OB%}Kr zv6TmG>6YHs)%{#~`|UjQ@#gmS$=c8D$=2}+OhqRzU9_oVm@irP zsN_g;aTJxwYTL3Slh>C6#M>`fw_0C^;9Ib>vpcP-JXIsA4{p(4&Q`pJg2L&3ooU(@ zy{x^sTs-?em-NeAb%9pzugD~fRr$*)X0S2&bS2ET(U9x4>H=4m;JWum_LUPRZFNik zHX6}j%|cD%t0m5{=nw?r;N#&Dp-gJwTHE0Y-@@Bf!5Ne=(Rl zYI2$Gm&|5BV%Fi}&w&iwaONeq|7koGj)q44#WL6*@T86w3%f=}-tS1_R&bZt@nH(l zJSZ3Ru6p-KU1_o*3lxn4UgxSY;~(UgoDWy8B#a-%nb&f%hzM`om9$45ZxX^fAC7kg zXI6%}5hsTrt(|F#3 zwK}d2^Q6$B8)#-72?Xa?<8fPw1X!qFFsF|kW~lKy?JsC6rHqY}RQ{1cHCw5K7XmK5 zYA$c2j3}k57pC#Iif5=o-b&x7PPovjt@26_f)w^|o_Rs6@`Od7l-oSD*0g1-7X>$v z+W+PA`0=_~YV}GWb8Yj;gU$Y7BJ#52ksF7sBoLrQ&3fnMHE}Z@s)%(yjajE!ZQ#_8 z(rx*MhT3MeI1HHO;J_%0$>jzt?Q-Ydp1wN$BrB;$9B)`o*~O429jfm?(L=N&ktr2aQx!?dT` zN`c*(d{Ab#iLRPYefN)&wxXgHSRMaEnJmz-g&`C)Y&1(<8PUKU(-PH2fJD~cEw1!t zW>K~;Dfl1v+nj^IfGkscH(8psqxaTPk8^a^EA9-BT||(r`T>Jf)heQ{ydW&okScM8 za-?&J0fuIA+zS&=;kPjb3A@d6Y&vDZ=wT$yso*ph4@HCQI57Q5YC%Ni|eq^0{9ar>J1HzAV7w^Ezy`tt?kJq zEXP8M{&Y=AzAJ#dkyNAHx*P2zzTrQ{k;g8Ld3#t;85bbqBN-4;9P=+pU^+`iM)4}<}I#IYg z3GWQ*EF|oN&Ot5rG93anmX(i9m<%SkJ86qCiLIS&D71WVc_4LE#+y=3{D=Gi|nPm~W$T_IPrnphZ4+w{Wev!O(ZEB;u zkM35Kn{VW={4TT*xPQk}Itl~Bub@0BdQs=DTUKq85<+;A8nKB+2QR}l<8dV2rk~(s zT0ZJ~d8@Ad;a#^jT|tegDXlY_&3N5Z#D^Hzp_BmE=MGA+kUx$KcHsyHP1IbqHK7JgoShdU6m-E}MsvS$ zNTIt&g(*D-Z25^yUohuR52^o5JBY09%uf19qN63s&y?+q{{TqtkJ7e`qR`fE$fL)d z1n-&!y7JYwPN;@sZM4;h`kkaC4sfddJLtah&D3pyNUFJKTAOx@N@9;p*~@f!%nHKu$FbTo8!X&$u+$eJ#JN$=>q)*w3i0-${nfgS zXR|?wCkhQqVR5hf(;QezAMNfsVjCbY5FdYi4-?WY>cW@dvj?iC@D` zZ+MiT##BD*afvwl)4ax5N@6EK3IX=}k-0g5i6$X_rFPL)VjED;M*6Qu-lmW$%8ZZ# zsh73)OtXEl9^!=xtPd=?p|NWb1S0(6?F^mUQBr12{X)%0+9x|biPaK(;vgnoBiVM; ze%R>-UsB8C(3t6a`4Tyo#8E!BLl#3D>+&PM-$05ze8vRl+sG}ReUNiR9_Oz5e9fp> zX!L$mid_{R5n+~FNs1!gt|QvwRkwCp&HnBOK>xQbDfcWCGqZKR6&D41j}Hygjm?wfZS|IN9=vxOhYz17fo z29W+`mmvA^;A{VoFEVd6(2W(geLm)&Lnmf85wVr<*aLYZ!+xPm`dvljTa3x(=Nm?5 zumCPC6l+wz9KO;Ex{3y~o=dW3>22$%j?6P}3q#0;`4mR@Axv1=vIO&_2WUqvPKmzd zI1$u9wTTVm8tK@VL?*~<8-U?=0t;GMjVSWV;jrY^>L1X5Nr!M4=w<;kAaZ)3RDusa z?c_D_(6cTUHI9dxX50dqO%|(-Imb}yzfwBE%3qSUV{NXYaYaSp*Vo??zmKan2rCb4 z9((;*ITH1+FN{pW+BAHC*{32uu!5mXPN%bjU+!(!`2C;M_5&Uvs)d1yiuqtpXXO*U zXP||N)dl)W`U=cZIm$Ry#9F-<$1-yD$%?k-lz93c_=8xFJzrX6jHUUS7vBWM_$nr7 z?~mD7CBwvb3#t&p@FD^?CSL+lJ9i_Oy>J>iJVPS$9%ZrU-uN7mQyQALO25$Ow9-ht^r&{C@*{B}1j7+k?ngnC2)+>esdc zeB4nK9N5jwLq3u^H87?Kkd3clJP;y_Jj=s$h=)7$cx#Ht$>=524MGI>ZTVlJg6~UI zY_8n~_PE^@mBf(tV{fdp1Tmw=*=vP3fJs&j4_nz5%U?ksk1$$n z;ZZ0QB_mf0^;Un<&r8b%qP_tTbSZP@>1y^X3K~V1_1FLUb{k_FrqY!g(vD)y(Go4< z&j;=|>U`2LJiDBMu8?z%&YPdC?U^;SmX-qJ(ZZyCCpfpyJ%g!=J%+%_FEul)%;2zI zVqmRo4W4CJtPY<~GS;ML?`V|n7SzMn<>Sp%kt{{#cG4^pIckO<;0U5uaY73nj692G zSS5y;Fzklk1fd=p=q?qFD5-eez&3DL(fvA#imuNiGylraa+w@5T(&X$A0=~Kyq`R& zd6OPxS1t9GaI~#0CA$wm^-y|qmcZUG$KqqY_EzXVkQTqKyBRL_7O-yAyc~^jZBed~ z`gzP<@-Lce&Nx;G;K6<();!`!B0phyP8wo@6q1dBcg*4+Bg?D9Uv3aCP8jQM??kX( z8nRj8vCG%k`e4l|Rm5n|4Xi`bD5>NcNMimi_x7Tqg>a*(q`faQfM)lC@O@Xv1z8J6 zl85~y9jVN8@J(Zb=YM;T|NVS@IIBCT8_1Xi5q(R6=Z=@>+xE-O)9dkeck)cr%j?dL zI~<(n2P*4{&Z@S-FOe)f`bp!%%k3B0X-%3_E-OE>Fbh!fN@GkBh%z0=1Yhv=K}}yB z$c6A&L#Z^J3uxhYJw)VY61|(ILJ)YBaSN3JZ5mHcVt<3fRTYy7 zzTncV1O)Zk($7e37h&~ym-*-n5B(!WJ-p!yFv%B2=Cu(=sy-TOqQLW7gBjv^gMzJg z{T4P#?Azlc^bxTCYmyz?*+$W}6i>0ot;83WP^1-rwB9~xUm^vNU7K|*i(7D-eMxtw9(-WwMhj{TCSp=r9=8y07I|I>czux3|U?y9R zl4f0+|F0#=7TNr2XR(LNSgtQ8HfpP?iNukS!2m zL7HeS+YKYI480WkrN+zj3X|cGWBYleiD(&%DZxI8NDrPFGo*b>SSB{i&N$>Ux_%P> zj2n{4ARao!V*am0#{AmS7Q$NkMc2j5%*573y*Y8tqzNX!5AVD9Jhh)FlwUI&zuHbw zY~@D0WiMxpa)xSLp@W6Gc5K;B+fs5gjH}o*JJ0=wTb1k@;-=a|5EneN~c zUZmcYg};&25S~#V+&=|e?VYXUt6#w2z#=_;X<6HxmIZEK&kRPRgv`Ss#HgT%$!ks5 zo)q*Eoew83n3;_3C=aTruTBI(iS&9pfe$5t)YjIvJ?z&U&Ap3H(Fx{6Iaya; zp4?ntwpOpi*`W~_*yyd#fMOf1xn$Y=HmBdqiMkr0A(j4CviYS4AI}A#>(9)<>z^lC z>qxM2oMTCS-5Fxw&7SpZg*(0AYJri4BXSsyp(2=YKf|Dv$cVxVeAGROIOcN10fZnY ziffYYvtAD5ngom?$7^K7mI@0_dY4$!4IXh5_(eW9dcy?* zHkZtl0H-;pPp)YHcD7eT%0h0Icx$)$VV9r`mvCQyXiwgGecGn(d>qny8Fj>$XQGLo zojU!3$`!8J{&BNszw2L8yu%lW}g zBtBsrZ?q66>7Ie#`1@xMKY3^G8$Y@T?HC@u3=vw}EkdKPuq)Kw zQ9HFQSfI`dQ!#d zu06-gXT`gQZxipkNgO%aMNRB&_QmbB#V*A@;%LN=!nzk+-6A7tb6MUJr!!$N%4x@L zNXo>woxm7nR$3a>-k9&vzdc1Ka)l^GPi^84-P9m@u^AWdg(=u-bnNq}ET&nO@7*^y zGWGJ_DUz(9NxrXYlLzboko~)ED?#-8FaiNIah^w_LVerd3>n_;-9Q53(Sj&do%K~9 zJsY#_mMt!m2!9=&e?{;VRx$S7AvR}RE7VdeHfxVj|D2>{-{j$Ro`(m#ukBw+v*ZfL z|B!uuaPr9?Tf2hM&_~+~93)A3jmwIkP8W6PlSJY|VT#$Gy^1M2cUJdbjGMXGZQa|E zL&n>?Kkiy+b(cS=?j4?7<&SUpQY0q~zCF{ELEN_mtqH^Ec`I@*xUW_ek?rXAtdd80 zYeMNuInCdfkCXSxBSN@nI9}?O;UC`#eT{020by-}yjA7V=A&S=`fd3i4x$RX0XM!f z2c6}VbE-^w?PdJLp-e*%buLZ#9F+fEUc!K%Ux7!5T7E7b8~6?C|D6b2o85YTy~=;l zg}DG)-1_}!V3ygXgYK3 zz9DgB)pR0tTL||owE}gN>CToawE?P}IEIRYTJ0RHUSz6}8Pi(~kyVD?C9=Y@>?7Y- zrb%)9U)E;A9xftid^sx}=_zq~eJlI|VqyOu*g%4+6T0d(M0$)nuU_Z?KYC$L4_5** zTHia@H`={KULR2q2OyyvjeyY6z`49>=*P14ysEuzXmMKIx3P&thL|m?`{{s*h}BCj z=CuXxkwF602)r~-QSMVM&WgvP!J)+aKR=ukA`5U)+hT1khnNK|2DvcxdB zwUXrD6|CP^QZt^}6SaGv);O+lOD2*IMnh)TbZU~f3>zA5K{+pcCLj0NJ8Urjih3c4 zZ5wGX`D$L|j?AH*?jc@(2in=n&i&1h+~!}Yt)AK%C#pW*n3JnKk`*Na`G^)}@w!>~ zEJ`JUio~CIGS2*s78fC&5`cMQHoq~~i4XfPMK0H&KJ{}4f;+6N$dGb-0dw$hoX>4AV}^Zaa5U z@`!U|fMO5#x40jk?Q&jOjFo_p+7xS+MvEGf%Z+IwH>8XGjT5(Z`OF*5h(qWyc z6SREUb6M3-$9euwGQQ8HBk&c0Yv_mI9nSy1zLYTfi|BE!z7xGAhUmYh&!j_Og`r(cHc!lXt~(`2Xi3z=mwKozO=9g_rQ6UIbrkY- zoBN$Ww(+pju@){=Na4-;x9@!h_5(a^kB&$kEIk|cr@Ke<=a_V2WJL5q46R0>9<52i zV1f+Wz+s3C<%`5>9E=5=HA_r#XsuX_TK1P?TVwecF@shDCbO7das!Kef&n{o1UMyx zLuM!*=IvnQl_81LxTYp$c0bEfyP)3-qOTE7YBW~9V!1MGrP9W>%CQAlk3&k^f~*YL zfs7N(Y>+mfCKkD7M0H*wsHBFno~DJx=TqzC_1?6}`<;G9bw{D&o~@%wHi@)gQQQ6- zg2pj^K&EET(C$nV_(>s5-GQUP4Pslc4asUoWVnV6G(gq6+aUViZr6fFp;#tLUM>g- zOP-OGt7l8#WZG6#@gEeUnTJo$EDwS6onzm;^7|?rj&{9!0u(3Y>(-C7h5iQ6j=+Uq8stvPN zNT+=EJP)vUIjX}eBYe387{vz;6+m%h%iuWBQ&GAeo=e1iUd15H89!Z>rA}Qe-=RTq&NvcM zR^MOp*K^Wz=`Tz(pr=W$!XM4>2VwjOstMxAbNGT^dB>UEl50@LATckD*Wtu6>;Ft& zQUj26+5b3U_WxP`+XB zG;~ySQnB&s*q6`~`V@18n3_xB3q3T1RM8VHgp zF#pG+;I<*%OBD_P02TuPF#o%0SsA(+>KohJ*q9o-SlZh;|38j*tz&Dy#fkiLqd(x> zC!xJ2(fN6RXoEg(w$t^L#2T>!hQ1?IV#}pQBh9BMcS!Tnw>Nz!TC9glA-6@V03vGZ zWX73;#g{`=!eh+f=P^LO!75h=zrM9lh4DL_CfBxgonyMBN~}SD%q(rn6L#A~3vu|Q zbqvN>zxI_ibW4hqOgyW^F4>+U)JSZMab*-o*Vw{(Gf7KAYm{h;@l(&MvX%<%g1%9RV!8>ILE@7>2rWD_-^ZV4mKY=c-_eN3> zc4rU7ReFg2cBb=CI6eU0fLjMM@R(D#!q}jbwBkSPrg_ZEeu~@|5Ldu5EAF7}0!H3* zV9kQv`7rZFGTkb^H#clVD=?T!Sr`haP+swi(W5oRdNIvI!^((utmx$)+TgP{!npdY zhw#9&WZb`v!ILW{eOGKX0mexeTkF{f+;w_Lon{bCDVI(|T1E&8Cz)|QqZYb`C-kPU zpZGv=enUJJ?9_Sy1QlQaWG*!-yH8FS9~1wCIjm+7An>S|26AU@b=n~D&DXmj>SVlqLsK~OcptN|m*I+5CJQ34SB zlK#oSjQFMYh(duVlT~A|xrJgTPQ_2&EYb#vnWuZ@&Eh_`HdC=dzYrws5+9;SEOP1(RwWH)DF;%( zbP6z0-hjW%wjAgI(G+>I>MZa=-{os{_oGdF)-(jb#UM7Y01l`x1A{9L7c0_GYgFqW2IUpr9@PU;;Pmi?dNs>5t$P>zJW-{YkA(9$eCDW;!a^vO}B5$c?|7bv`tN~2{0Hg$F7`u7)B@vmysjgZSTznLR8qV_oj zu*G~BSX77R)L{s{x$0n@cePZ+Od<wr5Ixb8)(cT=A|cl5 z4y||FYu@*?p9wLL0`X*hl_|iT6Ya_boE{_GKwxJh0Ahs{vO)+-*)kp-N*ymghV6Q(Inw&h7&@X%Ib z-`yG)e-%hRx^4YnU<+wA1iwGis~OxL)Lf))-ZLb^OPNmMT!-PYtp}@R_S%$zqKS4xvoJ4wRPAfv&ph@p^y$$C$tr zQ)%Z6d5B3qlf2Z3Y=|-sE6o%KSBZ3(+Fw-wUFySk(bQBCf8h~_xw-gRpXiymImpYG zyu5Y#jeY>+P+Ol}aT}IhisH6F2q+O|)2I1nO~w6Jy;~6S`NguUpIr)m|GZdebkS5| z$#F$DQU_ZILAo!M0VEym{O?+Sg_jQsWv1;&&fi~g=12-NZqTbCgbPk9jw1wl`8bGp zV?R)6G`Ebu)V$vsJI92FU)-JM!nq*P+-dcKsJH=Is_Ms6li(G$Bap!#AL<5))HWhu zEtFGt`E2-K28Se1Or@1#{|+2=YatMJJ1vFtk!%pK1D3%}%vt~rL%E1ysOkw$uNG}V zpj&C=EwThN0IvNPJkdM2Im$xHYBM6J3XFq+$~LiaR9@E~(5_Y%#RvKmT?3mhhI$b{ zOk>DoO#e#iM->QMK`L4s4O$H^n#U1YwwCK1g>-*jNPItQ$XX=pXto4CQ_l6vc2j8; z0VoYsR&qMhG9-3bZC-K`=>Hgh0-@rNIUOv8hZTuB)=5$6PyZxF`6CWL@>|={biU$p zpc$-C-!2}2VLMdJVUk{!8!~ggPKNEu-2G6#^}lo;+JazZ19dH z9<3~wVFeOL3EUTG3J?HB(I__?I$c?8wCof6=^=-si|!l(TzEPBT!g&5`wrC0%6H0- zP&(4NmwHq0ZH}*;5hOR4lDOVr-1+S%>%})DjI9mes_6{F&vw82W3NPAk%lQyy5cO| z?|QYehSG&BuJW%RuPmyZ@Gbe6x3OwTg9zl9!7T`r|O+`m-X3*$&p)c#mxt@^5{Bi0HBCh#o)37K|ghtlDhH zLB1{&tiwFMV~DG19s#y2EI7#=h>&E>ps?_x-K}rYU|5r`F7k459d<$G9Y>8cK^@=8 z6qy;%Yc3>pMasI3Hd*h1+@RQfZNJ8j+*ZQ9<*GGjDV)yBR6*))=a)9_=^8QpUuyA5 zLHMbb#@}AojI?J9^+R2dp5nk4?Guh`oa|5zRV(fEuD6ZC;}sK}NxkdlnGy!SRvQ2N z$2;VVc7g)fQTe7oOY*&`gHflqAMuwmilGJc5~jorkk8w*=e@Zrg0t=)Yucz8gb-h- zA&zC_w+;HPF?qVWFtFkfyL1Eo@kF=gvh(51V2bh!-ML)E*mNzqGBp9UX(C=D-J90QkcX-hp@y~zH&NQvNxEM*FF8HD zJ1YwO-lGTxx{d7oj%87sVP%BRM1$Xx zQy=7x?k8KuMcrB{?Mwlas&iMcJIU*MtZAGDm?7nxxBNmOPo-T zsk@zZ8*l{KiC>%oZs{(bJDcalb519;SD>#JkWk9cA{gYlx9B0xpRIFd6zpdmv(q{Y zti?!gN!y#3tbp6;Vuz_gRY`*jIq7DWo0&Mr)OC@?*2=UFZ9V$!X8vQ8t@VL?GTX1Q z^3&+0?v=a>w0ba^QAQa*?%E`D*pDGEJ}h^uS+mon4`a>WdzlgNGw{sUMt2-#c}kSj z_yTEm6!cLa7XRAQpUv;s_F0jaNzerDmOjLVPPhEYP_R6L2s|h&7Ft@Jwuz|zbFb&m z;mv0}=FFrRU(C_ROMA9Ktytw01CxR^bu>WH3wATE6i)>X_DiaS$-Bm?W?(CJD(ieT zQrjVU8-@;>lRiI#+b}22D|+6SbE9M(q_y5&SI5ulb-n}sB&z+T5!6LYsVGGI4H6XL za5PIPM2#b6?(}Z?eLkF?f=2ai)y$XR#%1eoM|dZI+~1Ts|F}giL_mQUul<`9SjnQ2QFYpVoD|era2t3sJ_=ar1}O zj*T0^hZ$Q3KXzAV*W2aujGn#X&i{Y@mj25f|A#O_)usvns5t`tk30Uq#x!$77gKjb z&;K6L{x9zMKOI}=EwQAZS$!sH{gohiu&KBv_bF=M1hP%V!nRaO?mu-fxX8lD=#Yc} z%FOJCpO>q2PY?-esT}E>p+8iSpk`iPUY*`~H+c&v4X-ZpGLM`Sry!0jUOlh+Z^cX3 zZgUSFsnm*R#1mGkla_Qh$zDVc6XI+&G#=E&Z*sem_v}~k(!wdq5pJh zuSD}&_QG;!wKd6|k}9Q^VKk#nne~^AvL>wtNS0y9dcD2LE49+R6fz@K(W@G>PD?I9 zS^stD0IwZV&fTb}Q823HMAO!ZR+PRwRWxE|?nxF;dde6W2jQW1%Oq>3)_V6JDOQ-& z7y+rouc-L!U@*MA{_bAD9z&a{!Vl9Bb}3Y7)s8geRLPK|N!17^#}{r?>ZMXgi8RtW z85pFJ5t^cvKut2&dQRB3C=>&EgA=HwEYRadYiucOQpt6dQPLVFWHNf5|zxrYf z$5lw~S zAM?!pr4GoHqX6Mr2;zCbAyT1}`-~01n{dzaH(1vlgaI!Lj!2dy~$5kjUJ~XAs%Jm0FZ(AUIX{hL+4Wn#q zhz3`@&8Y|%FE6i`i|2!{6JM_fj4ykqV2UGmia_5FL)YKm2@6j*Cl5Yei~{3q_-PeG z*N(rlbKqUA5`Ncxt@83k)IFBp>O-+s3O2pVO9i`IIN0}h1dSg9vPK_2AXJR>9skJ8{L>O(bPLM zJ1QgJx^NQgONvdU2p@`B0Ld11L~H=S0i~#@A|l7Kw1W{-c%gpLEnp8E{J9D}HeR@CgX}CcZeEGiM0wGrZ^SbHhfz1`B!xo-r z+ITf#!goNVcNseO;`zBnXU|-p9p_J=Zu&fV0*ZuvRn|8U%T(3JYc95u zV9PfO#mc~Ujss$#lMpd>nAjt;KH9=HFRUJzS9G}CP&M#d4G1hQkWoAN9=mx-D?9_7KGexvX0(EDWywpcSw_W(SDD%w-i`^?SCW+iR!g_wTP z8KpmU`9&aC2Ma5A4y-Y=;kSqcxIm2&yE?ktI(|16{~|1umKGmsAx&$k(SY`W6x9M1 zX`qR6%7IMOgS4P`l#apdd03#w_LqE<0)JGrN-wOBI&}}*M>uS$?TORWkw>X zTjrutUBX&IumET%S%^6lfv4k(cuCa@Jm6>qm5i4IVrZWx1yUpq2!S)tcMc}XU*ANn z#sO|be&OBR*3L`llq&V^i>)OYlP zkJi#ZolW~h%N*JAcsw72YzjP8FY)**motgf?n-3Ak!rl zL^Blo&iLW96=Ow$7RkbN^7eY|alSx-!nQ(C5taf~3u6Lz?(?CNB?h|}fq~a!L+Tv% z?4=q)0rCwq1J-u{sVV)ccJcX($;K=a)F01ZmJggUUt#3%iQ6cf2kll#j2k zFXz|4u?NBcCTUIkOlkXQ>9-(9;RWmtZxFNZ$LRljQq%jen!{1CWaAQs-q8?7z9z2Z zB^II8XG8HKk9mO$Jmu1|XcPu~+$$+7j4T-4-P>X9kX`oH4T%R(m$0Ouc#Y!70d+zr zRsbBv1x;1UEt#bSh?83|yKanA5T6K%1@sAh1D9$7!WQvu74k=6SbhZd(an2_!1{~p zjSc8ykF+L+_?CR?RpGbmWdb!}R5%enSr;5sDJ(sp1N{&*U(FWP)ECavjt`ddaC#3w zTm-OXmq3XI0#CSOvg*oPxja;7eC0ChPdRzGq0dS{$x@*mM9V2ulywsLyR}03N_Thj&~?wYHC*_M>oAfzhNsG|VQ& zCgDmet7-&=+oe&hG%3mA^$%RJg4>f~ljZ{gL5TkmL6sah$9E%AI-!UFwe=fwh0ZOd^_O>An<4upV_eQ@RR#2g1x)h3}9iY89 zz<1L~R)bL}x!H2fk-`iLxiavHfRlmzI}#bii1b2Lfu(8Gi)~U|+P*dtV2NO%y|KDx zUp|?H6Qu#%I+f&KG2YFOnOb#?Dglw9{uQx(8_~e#ycJOnTjhzQHj2HE*;=fOjM1Na zkCV2ZaTO+zXB|#1#J14dn2a16ODv-i4AT?N#-$w6yn?UcN}U{fxFf{_^;9(7t6IJzXZg-J2Er=4D8gF1;E zgQJBKaj}6`$dMW4$_ETYB|?MlatNBdx`1Xu%hIrdL4O7VkZTQQ9n`}zqGaScC{B3d zt(O@p1#=g|q5zkxG*X5wTqFvId8wetv_?J#g{Fd4BiZjunXo~7Rw{Wi=n{QuJ?UU7 zga*?xJXvKKeGnNY2j0^Jhk*NUu`1^Ij}MNOqZzcSTAH;B)u#4l!L6yO4quoK@fWgF=TpKU-=s%d026H2~G$?`kSV7-Rf(4Gi|1?-ucRbrh$G$FC8 zwOG>rzrt%qx0`%!s3PuS;xZh7LH{(_p{;w zZ=d_(1)Sf=WOn#?RfDx+4tEVDwoTKG>V9CgeY+>3h{;l28+r30cA8+V8kQLmlgPZD zlAshzG2USjG>S4REflEdKFR7hc(cWvcx8g{?Yx5C%BFZ>%00q!6v(ngANUhfT);7OY5! z*I*6mlUx?5>XRY(8_!W)rh7-vbYIrU?bNZH?5As-; zAO}#AG`UlKofccd1JEw3I0WJ=JdssR0b7k zw7~T_K-UZ*VNwDZo2xqLTGQ$}ACk;mmu9qOi8Cb^EZX)E{ekIX;eqVW5pCwY+r+qa zN5{0UWs-VWO~kokpG9$C=ufv5A!-@*+Bs=^#4G3?5r83U!(1GFpy#LuI%nYTT zVd~TYt;&==K?A(19Yb%6(5#3mRj`x{Q7`7ynHHC(eQ(7AGTgK6{5bUk#nFtW;A1nx zFT{I3y`=S-D?Hr~Mrc0Xa-N=tw~3~`J!rNEFnCg;+U^xep}IEn?;G#~B)HigHtRNK zY1&kENdbAUQErIUWHk?TB#OY&06?-aBPf5VOfJ|imBuAzlT5V7rCm^Op&)76p=>py zxx?ZHfIE~|Ni^NiXak3EV3qDYHE)f9zOBRR4}&C?sgwsBJ357&SS4sK=R;ixAN97d zNJl_7jn2MrQL+u5h&dnGgf+s$3Sr{GQnGRDLdeRD)E7r$|C>7ZY3T$T7mU{WwN0 zSrT>xq|q1eTa*0oD0zB@EM>pfc#gg%2(x7=|Zn?5zyD+H(z<3 zsbij*ScK!An2WV^Vu$9zsRFLx5)QY|$uvBAOL?mZVVfWW{}E3S4WL6PH><5Wsk6la zSxVnm(QSo#-8$mq|MryAEKzC92#ABvzMw}Drp;Yy^-9%c0?}fnV?Ko_1X{#m?3$0A z)r~!hWB+uM$;0WS*ld+DP8|dHIdgL`!@W)CZq5Fgv!f@3&tzNGa5o;KWP2t|r!7sE zhlcQbI{vvMoyF0(nkP#61)hHc0=LW4uTO@j$J>k3{OZk+YSTMQ({2u7#O1t3k=k+k z8*9>y0UlkTUPyaBzi|j(7poi3bhX-Pt{L0N==>r&8=k!yUKutqTVhA>&Xw3PcAoTa zdZcca>*b=eD49u;;{g|gwKjG&C(-^6>GxWbc3`QE%^MB2+TKQKzZs9|I{yQUw2R5+ zcJ0}(cjD_oL+=Y>sCCZkK~o0{X&58P;9Qh|xLJQY9NiAf|dOy#SoGWNzj&~9bc1PqUcA$U1xOnET>h&eA3EIARg zM}SGqC9=XJhg3lYx+!N=@q(uDBAPb_-CEw1FN_D61*Y_{j;dm2tJ!NRxg7MH{2i0= zyL!Ui^muyIT@Xks3c|88g?YokGXYDsp$fmO7Y`_vQIw>L%mkx+7U)LX#-YTMcP=xF zsj^(5VCf*fq;|&MC}+^PP(X=MPTsQE9NFd)XeI`>`e$m#hT4Pcg&i(554+mapab0f zIEBr3QbmAjwuS&grd-n`JL&5i(?{5;nU>M?b){;{$i2oQH@7VO!j*3{Ws@h+ShSWC%Y!X1mzv^=L z$Gc@nX-{xt`a*U%S{LdD58x)%iJs=gEND)sq3^yAbqogEjNpofuXKv~PKK$><{`06 z9@EXhRo&1--Qkg}+Md$@H-zdJvLq6A16$mZR@=j-T{U&n6gajt(z;32scBx8sw(m< zrlN{$v%k$b`3gjzvAI@E7MUM<<4+8>owWXeZ%yo9(P?UKVtfYXcIisiqRj76B6tME zyTj+n!*m^p3)dGLpC`d4>0@ z&1n+LVC7N}EHGxcF4tauT#p{IjI_lN1$SG~HE$-vZ3Dh6NyI=(-9AzPH$Kbh$XKtj z80J1%z;{`!x8GbAY+2EbF*HJI{cBdFMSmc!9SOW}ruo>tGA#ms&H}ZV2wUO{ufM$+ zMOVk~uD)#P=zbo`vmx$XD~VwzdI&%Ta{EoL>x=00ieMX+QujH$LewR$u19p` z04AVeTV76+iL*sh{Piv%o4O_5IjNOV2z~NrhP64I4IZ20b7%`&QN#2!OBTV7)S!p{ zQiHqNXA}&eo<5i09eu+977MxhLKM%xMrp^I=b|ylp*|w z9loi0X_ROAYzCX!bN!uV4G;D?ZhGkE*n_vmcugE2<3HUI;!*^Yzb46F`)2gM9kypy zeY#N7TkqPnUHd*sb@NWy5q8k053B{#u^bt%c@d^6xl?#q#{88JN;D=0((Cg9yl6na z5|j_G_0}cDJO>PmI6Q2$E%iRAYw~EEYrBtAZ*$JXyCK-p0@gEkOr_ym%+&dYjqYXC z3+K=}-sHqH(8p8V6Yf{lxxSoXsWsKYUv+bvZ$I4M-+yjGJKR1iW!0qNN4&R|5QzUS zc+qei`i7dXrfrMXwctrd8+d9oXved65(^#Z#mh_|e+Kta))eFHI`$kVRguayJ4UPX$NUs^ z6NjR|qN6W%>)Toc9kZ>u?j~QXDew$K4Sjj4nVTvH9**+BHV)w;H#!KzMLdrwo{%|F zHdh(u;@WDiWip_u*Pi}n4|&d#v{t2#CUK;yaVm&M=mKng`Z~|MbU?@DAI{eBkp2vf z_xlwt1fKVzBp?Nl@KkS;BR)_|EZ~VXS%XJ{-DECtcO*9O&<&o_Hi0?pE;%=({3O%y zBViNO_xW@de>umC1y8#zIAImSJ03(yPqNtrXWR&|?MYKTDxlzsb`C8AeqfLEvEJZu zIH0bN@Yg!ipc!w*fo7atbQ;|2$E}l1PD3#@VS4gqcKi3b5}g^~hA`12m8HqXjFH6H zW*}`Mw>=9zNaugr`|{>Cjx5jr^(m^Mi7?=SV5y^LVuIUr4^7cBZIRTHl-&+l0Th9v zSQZGtC_og~>+in#uFRKL0i@)scM#ncN#uRJeCO{mErJDrt!u1!6>(nz($vMv*%^ZL zLw9CaPdZ8h47HzM9Xx-L4HCq8xMC+?e0{LrstbjW8NtBszncmkgioaFSkRrX zsBDV`2ubUmBNI`|4j0&n;2&#Q_n+^boo%(Mrqh49m1MnKwe0lpz6W2!!1O|!1{76` zt@ZGU+{L7Zb9_fv13{O{d$I6VCstOqL^3jIPB-V<5a1;SQfLc@hKBHSxr*X(FlhOaeMATQYt zTFi;IjNlN=xhwPpa4tXf3?SMJ{j5M^1pHOMP`(06D}-z|y*>os2FF7?fMU@{!b6ig zjFn^X0quLDHnQ=s4JszNrHwHIYLR0eRm!VZ38YhHo*c)SLOZqlOPEK9@=stI-` ziF+q4Sxy_z3B8&eT>x(@0CT`yG!)Y8`tq&1`I%Pql4)0}U2jPK zQZ?%_I!ejPp1eGN@$!85?a85~%L9PqGGp(oSC^0eOOY?EZ%#VWWFum9xv@5vUV#T= z>VJBAaymRcIDdJ1JUlu$JKKA9(AwR@;{4?4NdX&CoPqv=+7nBlb+X1k|06$A{JwXW zL$gm{3V}O0=sHNW!O4-}mT4m(nn@iIuwhlCD6)4iAg_DTwOpbx3$#e)4lz288nR)l z{T4KiS|??>$H~Y+VK2j)U@U#P-jTYd~`9LB;!O2Dx<$DN8z6>LZ5mp-)n27;TZ# zBxQBM`&J!j^+!l&=4J(_M!16406MYSL8oN`r&cQliLnDB4Kd&gRqhLvZ$Pz6!K!a( z^Bd9yVWg=*4<8jM-GScuy&I5-xgY`zP_#|b&2{2c?RL^j}0$GkB|R2!T6>( zpgg#!fMF`SN~=f!Vv~A|Lq`~v>iuF`kLwl7ht7qO7vbX8nsOwLL;)|d#(%Vp-aF<> zHz@Oia_syV77{Ghz#&y5@hVW0D^dfJwFrwg9FA;ffIr}2GriQS;m~5Yl3OWwLld|3 zDL6Zue?DcmXQgppFU%PHAwhuC^s+aw&4$%oZBuSwgBG1ryI+7t0S`6);NJ1HJHGhg z2hXVPY6y-Gh=*$aIE2WE;ZHVPyshQXU4NUF2}@eFY(*T`ZU9wI0}j_^<^W?KUlhLGxilmy>N;QIY$uf#rIJbyY1y1sEcb_X^n)__3>whvl;R zpa*qF4AglTZ)(_IzmZLA-Sr+H9HP5~q0{!}TPWd$;`kn+0AK`R81A|8vXZyBSSE+l zx>;?$ufgc=cwsrD0{-!206hkOaUT}nNl?ldW(n78B`!G)1q?S9P{Q=A2q7x#g4TWn zcZ2%PjEw--gLo^w(SQS*{=J3$O;&#Oy7h3{RT%&RZZ|{EZJ{vUH(Lsw-}OOu=I>cz zp#DUX)_i+@`Gs?+f!(=w7BMI8I}{rQt4$sf+AxF?a!?`kCAs5(Q3Dm;^vcd>m-QQP z`YMZ&at0c;oa<;i`hqOE8$fmUF;Zq%Lv;5G<+_%DPr+|-QOHuke<0v?0fVVP`7i{3 z+Ef!wGldEO@WJK}j!U>U-0$*sXrxV3ANZeyF+^K&xV94n@ek@;37Y{QF{RN;kh7Ek zZW``}!S1dk*wzK$R^|kk-WU`xGXsx&vLtt7U{{OL+1c}bn%@ZJHLvoB44YT;W@YU? zBZB;(P%|DAdIX^^#H-#VMN^M7>fA`z@ckSxVz|9keMyqgiVQk%UV<}$u}p@+gXnB= zKU0eo8HCdzk4VYCf{3Q}j*2cEf*L zg>DsA0({mBxN6wnZE;nj?U}4UjTlyx`zPA_-?}o<$eLn`*dCW=qfsbJuPR>C2vEdr zl>-#MxK(@WcNenx;z)Xn&ut)^SP3W^3K0l@Ftkhl;&H(cBmh2=#09a78aR4}w`_R1 z77n%LgwZeT;Ui!A;KcAjV}Qve=%Ng@M(r4gA<(KX4>c>;ZQex>bOSz4I+AASI@5d_ z=P#sI#*xWrljJ$4zjbMynZ?5hE6 zFDNQ1->7;8FA7-}L50$!e4^AG6|vuucy%J%hDv&8Q7+SUIjI6*AsiB{rlYL!VrJeD zjdD`K)v=&D4d5scx`Ee|au)1*4&hc4f!;Z~nI0Svz6-%q!S_O{u#^v#;>GekH_W&{ zFCHVTNW)Ms7&8p4%NSi3%gdG0nZn1>Y4KQE&CS*D6AM_M2ODQ+Cz8Cr{JP?Z#Dy7E z>LKz3`n8}&mJutwvEhp*_!WX`#%LEIT7}fnFE}!6@;vU!Lm4oU6zyxHf?YlP%s^H9 zOQH}_yj?{?Q-zpJ?Rwy`4C!B-P{tXynzw2LzCk`{z1olITB^K7#1s5yYmf(wH&NgC;eZJd+4By?H z9%m!lEZF}Qc{4jd(G1)Fd-2%6D|c$Qb4Rm(gb7rU7Z4RGBvj7wbCftjcelC{e^4|C z5?r_$#9-~Xx zASo~G#-j(8;m$;R@gS+rN2p?o8OH`gK;3zKRX5ps}^yp8MZA;-91sGoZS{9FW=vY zT_Zy2qI5CSyg&rzYNM@ZhSWlEx8EJgR~J=C24S^B9-0Jf5&zNMFBn;*$TTZcUm}Tz z#mC|YuII3WKB+y+Mlx&qM1V-?TfoDbTZ~gD*%W6L#Vp{Fn_Ev34*RSxy~Z@0tTpFJ z?osr0ye!9+!pKLQ?Rk8oTv^BngLtFmw7~p*a#$FIbWDvqb2++dei%cKY#=dXe>8^P zJ{IO%`*WsF+*J7J9nA zzP`m=vA*;Z*`1=qo>HnDV+c1Jr;vd%_hwgRU=Xm52m1ny$Fsq-O1;#t!e5D`}$#fjEG~w;eND^I9h)Gz+rAnpkyD*H7 zVIY|3quIeS859nq1ZjS5n*6e{)y-ACg^u1EC;JHBmaPR#!ip=l&YWA9FV#vM;QMx$BoA& z2W<>`qIk@+*Nr?mn2Of|!@BOZ0zNf@j7~nh2D&!?sf~3{^pw@aQ__=N^0euy_deqo z%ddan2@D_bZ%SCCBszG>zL3Rqj`7$LI*~GHDw9>p&ZK(O=iGs9*|39yt3e)Hj?B zH9Dv9Ij`Tr*$y<}36Kz(r?`jPL}$ScAzBEkWF3%(s8?B55hNvr@oODm^oWJ8B5l_W znlCAMIqA~ir5?HVy|FGTr$WRrOzX>vxG}aQNjeHNl?YJOs&+O}*)~V^xpbQHCvHV> z{kOW8gJ`{b@uAT`Ki(GsE8uYG)kz9TN%BAmmBc8i6ugY=X?${`K_zD7PM3akiZG|v{sy>N~)e0t4W zma??K=?guhJkH+vN1*|2f(Q#J#=Bn0R~o|D5;LRS@;bSxID-tDr^p(EIt3Z2^`CKl zSVtN%BAAiK`HH5phbf#g`*afHtI4a4UO_!}j8|2We zrnK|RKtUtHLVmrXJ74nP5V(95I97&w20(#jS$>p!RczirTw;}M#vU3DdbcsD&IRe) zqoNDb<`Q&H)L^0$)1zg2v!-l8355s6I5l&N0pXpB5p6z}qRsym=%qr!(WEmQ;;+WX zXES;!ZH8uQxu4@~hPX1xry4#CemJ}=SLKv|XS>p7FYYzc6%vIDnhf=f@Zlv(F(EB% zL`eL>Znp@vq8K?XO{7)|R4(D`qAIHxCDZl*t(pA*mMF0nx@}L0%`O&}j&_(Z{NGyi&<(v;Q0f~8oXx-gf+^LWwXt+KYlzPIjFIaWnVge-RPffHeZIB3GkKA z<_WZpSjAd9`V-ufHfEl$N()cSjb&^dQAO36x4jY|nA+q}vnXkOXQU2?1B%MjT|c(` zT}o3(o5*bWC*7%|GK%RR~v>nZ+Wb4tSv2`|L_H1&$Ispkjf#MVaK{N?522=^Hv z^^uDO1iI2u%h;&N!aS|mzCr)zl z^Z&q#dF$(%Qp$s=>D{~r1{P&M*A*^Wn>zjvsO&8-hrs&5!KSCw@JrAhzE!&GzH7Gk z>hWvyzd%h(hs?G^|8%*1g*6F|2Ir>-dqqR*)l#1|5-z+C#JTSjc^6DjsTT%3AGday$OY{3rA21#smc=-m8(tm>9wB7$vWM zT2aSmnA`4VFI-9iVvzz=Ly%?)lHGPY%Zm^jCuh}(OAR$hOImo^7F-5TK!_XD*{5(O z4+};UPx0ssrr?lj)DUuLHSoD^-axWWK&C;tB#TNDhHSy3*NM#_9d?=P{`~ltou8ll z(z6&%xs7XI2s6GM?2o6HH|2~_pgGmt4#n1h%$uVerUI?uHJpr&nc)vJGsPcfcgLYh zlfBk4cvbv0vDkYFS?4+s#X0NC(8?J#=V`HNUUcc2b4AC@*TZ6u^FuG@H3TU@NI$1^ zT~wTOoRvPvELY_$(MUi=nmIItg^T%mY3B2;Y6j`mQnF}H3T%Z%N{en^B0^Mz!<4yu z2{ENl!gNY!st|bs%Bd7uOs5>tIlN@?obh@nV|3#|fz$XbN_CRH*4$1=WGx=KB9m?2 zRu2KmYo$ERFRDv`USK45p%)D1&O{jVNz=r1TOkxqY6KA=0yWuIAZQ`aAh^fw*EP4K zBG>O}ZgDmi-PnE1Ey~8Cn`$+;+|_s*SWRUFZYMr;a+1lI?*(=+)pV>+lZxO4K;j~BAazuh4&^Vb}xTWJww0O3FEBxDz6l z;66R0oEAY_+#B;UDxOUNLpW^Q)clt9qT2&~wobgTVdG^<@isfJIq*4#F^19|>Cmai zk8)~hD}SYPYa36V+|z^ogTrqRoRhxFmlu2J3%n27#4$5Nw;ogj$%5GPy~Uje;fT^I z%;q=tK|^9Bcs1xAPj2DBWPM3cIPh0V?Wf)sOnYd)k_?4KA$m{&RCaUNjMYX-0HuMY za;pqsvcpgk1lHNb2K+B`hvDT-5VxB`Afn@MqkAa8i*++FyibFpCpSsa z5ry1~bl=)cU6~$4%lku4j>L1c!BO}yn(@rlnTLP8?}BS$zk@%C^abk4&fv#+9XKew ziV}Xf(}hW~^}Uioyp9)K_oZg4Z3n#AZ9RPZR+!o7xYpVzQgxoSdX_JtE^rk$e zY)&$?ip!!vk}D2{^Ch6<8uq=FrVUgF=~H#c;1lM2Ap>4()|b@#85Hq0^*#w%`U%r~ zTZR^|htN?*$@}nm2sUQcZ}GvfiQW+wAxKnVsInw-v~A?GY3n&;U!#HInJj~$h6u;m zQda<#lr7y!-l>5RkXGgRsv95qbVx3oFU!z{=lmm{CI~?P9nMm4n6^1Uk^i6G-Bf0n zKkA0-L3%O@u@LPt%GWA;Hp+GsOA0_55XFmR$wFo>*|A4PKzN+3E+pHC3JI4HTaDA9 zPr$xmy0c!m0V3eHM+|22%*gvnNi5OA-yY?CMI;fQ2hvCqh4V)dznUt5mbI;?CPTMp z3qfXhCrM`Lq3UfCLUO2kL)q5d;^$w2udq1gZ6YF|LteN9WUL!5=8N^T%sMh;g%2qtb2tFCe~joH;eIjH#Mlf zxe>d;Ok$QalP?9i#9S_w@fh0HS%=`0N}6K!MP*DeyU8#MnxAEw6-c7FzSA=erWuCM zxpaL&FP@ke?Dzi&&AFBS5UCmV12RN-ilZQ&5fws)=@*V9&_F(f3!0;W8=}AvEy6D2? zLEUZhj~+sZ0ood7DX(Xk-`-p>-0gZc2^=xGwW1KVpWuaHGC)y+Vt?%wd&Rkt9!P`9 zcNarBxkAsSRHKwG*A)nXIOWwc)EgItpxhO3{t{Z#J+pW9#d<|OK8W^>Y+U~95$Q9! z;S-fmCv+kH0lw>d%!@C7Vu3DkO<@DjkF=3dS40mP5UfmdxC9Ai%3nZ33Cx{kAj5v> zlkG4(ZsyAdHp#4LGd7RKvYv-1*nsIY23fMEB!!dt8X?h*yiUo@H99$Gt{)__MaV9^ zf_c3*%coS+v4#TGxO6iEs@P9b1W$qjI*6yBZfru`G;#38q{+jiPJjYY9bqJB!EUUs zXeJ+5`pd(E=TA2#dG)t4)~L7jl9%M<<4bfQceXjC%N?REMLWOa^wmh*pbj7KM954` zi`I*YX|JmXF)+_=z3Re}h3waTHqwD5eh0VNy${{FJMY@Z*)b7&a+~87 z23CT|;*2WB*Kqsi)8B@>&~`oc`_g#Vb1d9B?_$e%)BBuc^uqIsWMqrH(p2u*!Nz3W z+nrNo!}~Bb(%p`(HMP5tvEvhS3y;1yu-6tIhtQw()#KM;5GxrC7{=V?9)?e>i~qH+ zOUygT0Lj_YW92+RzMrf_fSj{05|ATAQn2Cn10OU9pzYdZbXYx+JHD4}3MW_}(0M`^I|%Tck6G>DXLTPYA502myz3q-?+fDhcr$YWvPHESK) z(02hN$N|ik;e0^=L}dda<}Q~O_s@J@lIvXGYmOZtR83{IsnLjPHeo$pft&e-!!cb+ zmK_ln$w|bIX&n9d?t0{*&zB-|_ls)qW?lzXtSb8lE1i=1@%h z19p1w<@1C6^TU(lE%;$5jO_5&d*AL2pYI(%8}2{fJ3~=!vys(1Zrd5*?{={CT?4NZ z=3|S`iznoPNbazYUpq$l?B)J`1Xs#i{!tHOa4b5ld|nU)67>rabWW38y@%|owY5C@ zfrBuBUhf`SansX~kl3bP`@&Y5X#=zI>`aNv@Rk;kK&X8khg)8n(xmc zDMk{C3I7EnytyY3$Oj0}$kkA`Fl-8bXH^tv;i~r~G{@Pl;GOuku5Ki%^PTybTL1j> z&x^TPKKRRkUG;h(^`h3v)O6wseQRqi&2G%ErK(jctylGQaEjk08dBd%IuMuLO$J#v z^1O*793zd<=-t4TPK#Rm;#L~#tAT%t*5Y?%js6PKHV(TuCluXsGJ*f2cSkuTMPk)_ z_JS7M(1pp|b&Cqd#qHmI#2L$Jf}|#Zkc5AupKT5Gx2_tr;jhiH9`Ig~Joto#1gK`S zf9&e1aydC;JVwfzf=6L&PZH@c382l+|B|CcRn}@6T18}*TB(|WK1h9_x3}{(*1X4b z02_jq^$Eqe!V^<2ZRUD6WM3Eg&B6CT7XL2Ce!f)a{qv5eH0G7asg3z-PjL*?l+>ma z`Om#56*Bx??lLb-m{}&L0k$3fp>@FbQvRCj_3nCDPeR~RfjN|~?;%?oDK6h&QW=ph9{WUn9I1T^&$tMw;@lw~ zAuxwrA&s;I90?r>PiPy6(Dsql1}~4lIR<=+O?m?vR$4XS9h~1xd!V)=wBE9o!1S^I z&EB(vx?%X92y`*N=Y>UH@I5#N57-X*YEb}u=` z+T08F#pr3sI=Ol34QVz9Q>Umo=}(xZ)1PQb(nS7lWY1e5{iSSZ#(iwa8y0`HyzN>M} zm4Y=qJos)qH=Zn&kA&~Gsl9sqI*Rr9lL*1>Rx|{nQGI2vo8hcMmfO;C?T_itoIkb+ zwJ}WBYN)^;Wo7{;|@nP!=WYFx%+W*zF|jQW3RbZ z!{f(gxU%*KD=ox2QlBs9WLjxtrv}cTXqf(e=x8d z77s(8N}&J>{RKw&MWBlI_>;m5vDdq@fXqB_n1T`z*yg z5CYIH%cLY2ta`-3!4zM|S~QHqqnqfDiTpLqkC0NCP^jFtcv~B&Hb+)7HkqHBuDZPu zo?Fl1hFY;DRdcGAb=m=ZwPiV}9?dT=?IA}a)U1XFp0v)8*<-%qWXX_jVBNq2-6Fn4 z+)60RvGnH10r}MVmiz_YHSnW+4|CADl)pd!%Dm4nPEY>%eIl_kK2bvNK{K;64hQ6QI@;%Ybhi^@RO3pIDYYX7V>TB@j6lmyqXcjXhrnr> zrL5P^3HY1d)Q#H922!ha2vksv5%D@?H*-f+CQIOvn-x(2k9-*}&Km@N0lQv-?+-St-!&NRo8S zv_-jCy-mx7aq8si>cv$I=p7QQ>+^YsZn(xBI&w?5T{o`$9Xd%qh}r&PDMt?=*%mPg z;lW7YP0NULXD;Ouoj<&4HKMd0D%lOA)@-%DejU5U(L`iEsil`MW9c+fN3-t=Wqb3# z?S=#|`{;n=l>08wbw+#S>bk#Wo*0)ZCpl{QzU$rVvud?o1Xw|S`;e?4n0ZTVJ+&8* zR%=)eW*eT;5Lc%qPe|@wQp4`sZZB$WtMi&@HF>kT8=BOr>GUxQG}X}&sW6obwm+`Lx^XZ`DLj(Gc%^o-Ir- ztBw#GEY`WqS=nYS{_^qpDF{M`$LGV7FTOt5KOY`G&2EXkEOt{8S1={Dg^2$gE8<^n z@&jEyo=MP$g>k9reYVO-mkUR^K4dAopqr>ADFVBbj zC&%9&oSv)HFPmNzzb!B1Z$p$~2m_d*sGIDmq4rL{Q3_`-p-WGm;f;a#bP8fs)W+^4 zFUZ-!safMY_xKKc3dJU$cp!47>)qH7D9#;KPZGrrpWY=>#*5)rN+4ZPp)6vX>QM`b zaXO0|IB1>S8X5O`z;R*c^>uYrPp5Sg=S<5aLZO#Was{OlpVz`av|t03{TV>?Dh)wOcVe^9S01qgv7pJGrhHSyNsO)GeHl8Ya+vtC~Q_q zF<*b>PbZW1#JSzK$vhrr!hQ7ut)le?<}LI@i1}5I(+*GUmg8tkC6eryIMH<4L!Bxj zRFLdriZ{ca+bnPLvcgRk-Ig`0C5+k`LeR?1dpqhVhT7l&ktMU5o9@5%_J~H}%d>5) zspk1IhS)9lb+>KHgC)q)LeMM0;{;5VdP;UX<8`x|U&F)I2@>An@`3HZ)I~U1WXRBm zzc$nm5>AoZbz>MVSD>0TYs1dDvnY7p?2>+6PS8gbZa!A9c2&>~PwVl@rDs;2>+k`C z(~Q7T&|%)4fM*CefKD4#IldD6-SvF3#w=OL3L@!|p%ZtB&Tim&UMjEx)YM*7rg%MH z_F@tb3%668EZ9Gvf**)*3g0>?p31CdHe0y2-|JcZ6F|gJCKP~LoYc*#o|)-G1LTt@ z?pov};`yn;r;8@c8OD1=j6i4GE`OCnNwt?M5cMq?= z@|aYE?fDb|F=LnDYSP`6~%-6}Qrf0W@i}rUHv%{Fb=%rx`d_MB}mbJ+8K@eT2b`3m-g22MaM;Yik zj=Z7tc0@N;rl}HE2yj!SL8a{sule5TX^?j@eG$1FUGqENaCcldDo#qGDM5w_?~2-N z|ACdLz}>iYTPL38zwiEnA0B4#+#k{LR2G^7$yfkikHAK+G8d`!?c9MA;`Kg`2mfm<+Q?ro!x+o^R3bczR3x z1f_&)GLKh=V-7bn%rv{L_zNDZ!JH!x6L8mpCt%PA;`HfoPR6dSnvH zdF*?{TPKcYhX20vQT_kuEas%zbO0I{tTDeVDie6s6Y}q!g~M1z`xigs_UvMA)B7J> zmn!@t9a`Ot=hFFGoMoNgJhaED!T5s7pYp_0F6ckUahF~Bu!C;6>a32mm2U_SLRY*G zs!2}_I;1ou^?}*LmV7WR?@8;9CE*S~)VxZkAUZDn;%!weGTw$hqnV-}Lv>YJXa*=( ztwvYfbR$BjTws!3ep$^I73MGX)UY;FSJB0gYrN;42tux`BL`!U&~l`vy8}%LK}nqh zip233&ulIp9Pr;;Q9f^T9;fT3eix<=+Gh0z;XpUm@J5ZLVV#rOw*+Bp%VQK;<(Pnk zI_Bh3di`(hmV;dkQVr&PSbV>p%w0=)8*-&xs!;>rrLSjn63$Pp5rbJ2ce~saoW7@EQNEgpH#;jL zDHGSZ>!A-elsDr6SJ0G>l$FhjFL^zCH-9SwA?F0NR@c`x_OqmKR`YpbWXoHgse6w# zWJAWq9zI$vLrsugCNXh%cJ%^xFPBx-D}or#pd+T^V})G?)Zp*vmVr!^%S7VDDNhQGUPQ@oj29$ zhflwp&pYO9+$wFu(XIJoAAVB*eo+$QM?}Ie5Ek2%z{p@3ADqJLay7m(<0{{ja5~8F zGh5Y=nY(^7D@}iuZ~r-;QzdfDSV;;Mj3tq)0k+K!{c>!+?Edg+CtfAD|H$Uj8L*8u zH6jja-MoG6BlTg!EDNp<{Xw>g4;A2Yn@%{609wO@0A5P~p$-Z8CSI?(<`POu{7Ov* zs=Ms6LvAkJ<8ltBq>;_sIS+!Oz!QtV&xUU2|4ciP?h(MgvH#%@Z zim{i|dTf2Y_06`InyJEn<7HKuc7RMw%k^x0WmPf(H_`OTo%u05B1_Z9)o!QLr(17- z141-kteV|dul?t9Qdh(I%&_>i4I+_fo?&C&#`P)@0-{S8;fm?iq%^d9R{x)BDBm;P zLMli@fgghhoUNFktzRtXz}OAnEa&TmjcFmTHE_I6@s-bUae^2$vFRbo5H(7XPXdp{ zxK-|mD&A3^*uO`hQv>z&=Od2x02V+8r_`Ffo>f(`b7@AnGxB#a)w*B8bO6K_JPBOe zGYZ569+}8HzUj~b8KNUfYfc7bxkS^X88LMdGZb)9$Qgr*1~;TxRAajPh%E(D_2R0! z2AiamM*=gc_#?Q8hfbbA6s%`x@tqhY@DAC|JA7aG|XOaV8W&mO^( zJt>zHBV)3&J>=QA6vhO{6((c@+u)Av{L|BfUFCR_Y{%YU2%q}EWkM8Y`O@IJow|c)?mNu?v1>qcolY(t^Lj#iQo>?#O z7PR~6l3I^!!I67Q{L2c={zwKj8uVfsZ^=m{O5xz-_{*)tcf-^|9xq_8%+~QHN@-}f zp*$+Z+Zr|_TCsq7?LfoJia=lJ8<#y6OZcz1xdDi0*<}h52n|>RQ}F4K$70?LCe^!{ zc`*kr77L>^7oZ{V+PDPm#-wV-%X)zjWTkmnD}39RNSVxUgW@|_7ci~hNrZqII`OMI zFp<-3Y9u>uEIkIq`lBz1Y4gvuAn`keN7b<;5E{F_Mivy@yexC$Kb+Fa1YZ^`LXK?J zI}BkJPv77ON1!rrrh(lM;i;UFxECx-Q*>#u`@Bu9Pjd8%_loQK%@yU6qyWQ3wVd9P zWIG8sSoS`KvTMTFEASmP@9;G-(DjVKL4X*6b`|y?^G1bR5zeBIe#-P}O-od8;-j9V zd_y9kd<6SUi^Vzc-w+}HtCORH&o1iOXFp>2q3^EeiXeAMq0F3-2v4MTHYDaGrsWy_ zB;aG={?yK!PBls0gGuX01-YsL_i}+a z4<0;tSnTgTfBwbZ{x`+hi-Y~cFAw(xy(I7`4FOC*-x631k(+21<`u3PLD&2~t|+P; z-Lp}X?bNRi?65Faos*JT%EdUbTIY>4b6Fuf9_kG3f-xA@lJehbw;^S!SdcPZ63P(E za7RoDa-j`%26sVb7Ca{9%Nx`=D^yUC$~o6K!B|T%7>VjiW1_>>xDxA zgeUl0&nFdndy7Y@btq_vvs$967j+Isrk{GK+yq*9YZc7nat(3}oUln%PRW;k;LmXi zGuSOoVJ4x8y>5nL^)p5MG^4rz@OeY8lqKa9Rg!Qaa8XpjrO<{krG#M%909A?mY4(p zkpf^w01;i!y1jv>8Rg{=-GX=7?_>bcaOOCngM~hZeRrB6*{u0raNOufQ7f)F`lrj? z^ySZIC;Q(F&(2Q|_Kx(jf^VleEko|J8@KFAr(IRN!UH4DO}g1>gqW6pd$d|Fm`|t| z@BXu2{MG-9k4gedBsa_N)~q4?1bX>o+z$9vR{-NNbaxzD(~Fua&7=g2K5<1u?;9{a z1YF~F1DSI=d+ftReoIRWO~0~m;5Nq2y*Zs!3igh;v;6${mz|%V{L=ID86{TRYlPi+ z?EUSU0p{@kbbu9<>lc*=fN)gaUQ~iU3Na)N1(BHZUzM}u3h4eGnYn^#};JPiYUouB+ASZ2=I3Hw?9EGBrMS?=gIYnwZaL% z6WgL$Cr9v}T17VMu1C<+h{&qwMy_*B_JgDGAF~z-$B9$oiuPD41 zA#lb(hX`mS;78@4jRwV@ZYmU?G(;i$MZn-hH3jkfU_(L-;7IR&KShS_RJX?cZ|Raw zj+F9($6kaRA8^`s=W1wK%!1bjcg-wq(izBm3}|D!|B)nsX+Or!8NH(JOqks_gtjw? ziVD%a23NjG1sqc1)%0JaUN0Op6{T&(IE*yzMxTo!rR;g9umA#vhaWO!#Ut<^HhOhH zatL=KmEo_PfQ_96P#jyBs0Vk4;O_434#C~s-GX}{!6jI54IbPzxI4jZaCaT_k*&LL z@2!2i_tjKSbx%!w{nu$ZJ^wlLf3H7`0C*fPp>f)rq@NK2Xl|62$$!Tp28~|64V&Vp z*|#T%vtR__J&XPhZ{BO1fEG&nz|@coRivDNTWmM$gS?x46J?SPmv;&MEX^~Y0@|v* zw7>QSeVzt*pe@KceXZ~eTVE;8rTAVEW}`JWN4Ux`3RSiCH7^XG2)B zjY3kpA>Zuuv7Q6HEPg!=_*}F=zK%z+&xkLc5nSkj(L|3<)kySns|N&^$^5}p%9S}A zCJlH-j2#!=JCf9Xhi+Cf%3$9gut%ha70r=!M5_5jY21EQpQ0NMhMuXps+bXLIMz#- z{A$jd6^u50ZFTgxuo+D3eeP3-Z&yL(E^<-L07fkKFRz*T!C(mfbN$O6* zn2&ugwZDQ}cS#-xi_xkA4$a+InG(d{z?2y2JSYcV#+A8DP-BqS^A3hd{s`3cA`<6xs7OA?@TER#2})$U>+fW=4P zM(db#zMpsI?RO&9joyWQfUGgT8m47OlPIci)@uQh+N<4wGzj39)rCq^q5-bJIzby5 zwo}hSlna$}WlV%^1;95{(ZmKsp6pS)P5I?7E83uwP$8Ct{Nc@YHZ8(_S^Gf&mPo)~n zX#unMoKIlY?gvj1TI#h>HW~C{$x@-xAjt^mVuSN70t23b7RmE4B@O`|R-I4Q3^lcg z%y35ps4ATcz75w0!^b6XLVD|nGdl(ZZjCwmdoI#WbMhjsrF!})xAl1>-&cCvp67F% z$En>q`wx;6zkl@}e~kQb!-;f=4d>^tg=-wn0p8-bhb^pXIp}-dx5sj!#`hhg?L0np zgzvk^-bhZ)56rR)lftqr!I&m&rP&kk$uDtKUU(3ls4wmSFcAdy2*`nWKN;Lq>&0xX z;5o2Cn!Amit@b_JZ0y}kf+wxz0GXX$Y?+(G6tI%)0l1g zj3WHD{K*kh7F3$6YfO!39cE{oyMnzVUu5czvt{_0u9Rn7y>azGmtm|6?9}m;qREt^ zWmJw#y|l`msLLoqXn+lBL{{gAdhOY#3)zj^)vxwrWt~CC%bz#J0_Gk`O)DK9)wsL4 z`buNK^aTUHJPBkQ-mBR%%*j9HUp}?X)gGF7A5(HRvZVJp_4D$%*PY8?0Z?=VNrW<3 zC9j1D@B9b!3BV`@aIKlaXGcCj>%)%IiCPoqw|%)v<|s#J#O+c^#LFE8Q_bZStzC;s z!}=vTB*=I1wlgQgE}()BgfS%jiOtG;VNrtp)ck#zSj;ajo}}bj5t5Vw@-Cp=^&YxD z$g|z{BoYSeEkBpTbE_>CeCBjjtkb+Cs~+|io!dtluhqty>U9shdGNx*s@kUf34wi# zVrBwbQVNPP$;KFNeIfZHxh(`ZTRg3=_HN1pR;K_5qCI#pQzff?l!`Yi-)SvDIS;ta zb|g~+P|A%TD8ioiTu5FkofOR|W+Y2bzOWGPzu>I=xg;tT2Mie++7jGYf*BU9+QEc} zBu?8q*kpde8mnW8>lgtjhHGP{8mCoB>DwhGlY`_ybQ^^hu6X5(@@lMh1$=B4u5tAF z?ddjHBKd$0AIuZ2Wn1skhqRJL6ijO?j!Hhi^$LWK}qJ8$i~+T zhOH5QDJO7)F}sgXYf+sivq&vHKqg`=cW#;$vW`oAyeQ0DZ>_AuUXS8*Jt8Z;tjc|M zuWYr>raY(m)Q1c~|0nQgQ-h^m?Hw7X%+vtb)zk}imwL56(@nk}_jiHL?|SucecfD( zAs4@xM~aBd%)?2)(o6{F8X>xw?U=X;QhrG?PewHaj{e7(Xvik$o|(sgs=mE5H3_Q4 zlN^OuDy%d~W=U0$ohb+m^y*}UGTF)NNBUUyK0$)rBeVDkOJRu4ydpYajV1IA<)1}hOCSZRG|&J5Ri3{oM4VlnJ~^2= z+5bxc*arUh0OelJ3-2fcpn-RsbzGTde*cpa(7i8^F+cie8YbHca_S@+9 z-(SeqaT1$aNZh+LQidf|3?Ch=66O4wm+^yf9-s_f`ovPo+@XI9#MtEda&{hO+LjeAkkhy{qk5zk2uW53j z&8mxe^YC+0^b$YlF*kakiU3~tPC8!kE-`(6w2T0;fH`m%VKdNJ_2QH__S)yc5@#JT z5M?kC%3Cx$OZaX7+9===1QOmI^CGj#J}5QXDZAXU6~JX)@A<($0W=?&o5oU01nxSs z24HBjD5Z<^!XYai^g&5|23dRIhZbn{;>NyS!XUn8LF#-71HT7CCib5=c=1C8^c?md z8`r}Qeuvd-nXu+d>+162gLAT)T`_L-vg?(ogkXUBg|zYY+W3wtb!ql4^LPxq>kZZ! zN^5%q4LAw5zki#7zt7f;w{E~J_Fz!SB-A{AVD5zhq7(Uw&9fBnvjm(GKUx1YY^H>b@>Nc>63EZi6=VHOhvum{HjW zLA$v#unM%{Y_jo@8J?>bt52Ugb-ak+iF%<0k4Z$`XGdxAdd(VhZ2!Ssf|v2k5(8*0 z6jlKQZcItMDeeJ-$_|v1ZAEFC5Ln>*aakpE!CFums&yGu#kAhI;kqv@pGnA$H?4i# z*h6!gOi?}EpsmQ+A&ZO|e*1h#Biglrk1hWsC9a$-R%*{7X0Npke5~bc=%(e-F_!Tp z#|*$AkqQEpxApJdG|84sC}WgK&^Sttr$JhE(NyhDJ4|5npx~!XENiP}`eNyCG+KJ_ zkpFDum2`nuJjigl(6hUOP=x*!_jyIp@{Y%!*HO-7ys$bIsyAbPm0({4M*P$44}qU# zg^1liIUc*~wy615hmJ51T>K26g{U{}4E&>Q?y>O4z69 zxP1uP$bm5s`2s!TdWKoYaXDv#)iXV+Zh+6s;CkH$NNz5}*F+S$^Z;i2p21 z>ou9|VW=!tv^<6q;Ri^S=^}`|Yy^=JR00^UT&T$)q3aWaavKpJI0Y@ck3zS@E+SOk z0|MVRGg?fLJd_e4wsW7kg@f^`54?AvsJ!}vTU;9^INDN7y=S&_vLU(M#wxLD&uXyK z;2amohE7hQ+#^?^n2suyzY8HDAGbH#T~ZC(grt9?@T}l5Z!hiYi53)rN~`g?#fC|^ zFWOu0*w;WHs&5ha?rgcwhBRJRZN9WzVP)A(6T~O0_<_OnQm|JBx)8!9VIs;4U;N8C z63?;n^~9o39$H9xx*%#Osj?Tga=zK1iF+s>p}!2A#eJ;(6~$%;KqydiADkb z`Yg(rV-l8EB9`~43m-u0vC(T<6Gut@eDf@PV;d~mjR{`lJpLsOXG}k?%2ak(_cxYk zX>z0Nkb!%N_X+=I)f~)+aEu;`Tkm^c9~;ugSXN?^T}S&T|h$MG6~7YykGQ8 zjDD}9)!8m>|1m$F%3{fr_7s)^GsLy#5VA=3=q!Fgh$unP?u$J%F1;6=Z%{jn*}Zca zf>Z(F0sIZ|bi(GIqc?HH^KS~WQgYpUWrytNtHp2AUD>V{?O#5?!;HNV!%NepM4}mE z5#&)KR^Rq~zE84`rd-sEtc%gBGub+Jl$j%|L-U84Tu8Cv@J))@>|^cwvF|`T*X__B zmi2+%;rc%d=+JfhVhi?q!F z8`Wrio^|GMr%5E7PG7(75!F~c?R$@2!hr%`a;7&zEcOVN1G?ju&g+Dn&5 z$ZEAdc2!?yhiF+&OK)5?Ajlj}@%8z4$x<=C>0u4(f@n}e0ZUes&RTPIlo zyXnjlRi#KS3AXcxB&UK|WLoJ)d_P<~5v^dS7M|b?&b3qYu+k$IXsg8Np&niNMSPBd zbZOTf9=-w&?4MoX&rxeOnpz|p43mD_5)Dl*6j5_b6>5!jMNfgtowD8K_;3hcz#fD% zq8$u>S6R*~XM`6I$<&6b*>WB0XiNe%Z#*Vsn=o$X`!Whui>D^tIt^@fVRg4eMP`+OeZw6%&fc%@AXeR}z|8#iZ2*!bKPDBc z#XQb(7lO`7j{Z(0&Lj!=g@?zly;<5;VCM$(<#zZW5iR)Z(3Zc#Lk07rV^W$}WlsWD zd27Cc=jsSI9b^RF6uby{u_AZlPcO)r+|^*HmaHq;2wClx_VlLwj71l6=R9LcmF3m8 zmftB2B6$)N1AWIxM0B&52ym%b-ibZs6$gdd2Z=V&77ozxG;)3tq;908IN7jR_Py{t zKzYHRsICWg`m)g|w5qM|^7;l^SjA~{ibY& zI-@BjQ4whJhb-cAp6P#XyS|^N5RPM6Ir-Art8q>_<2ea^o)I=zITZ{OLp;j5QnC2# zY$R$%H}=k8Q@_14KBxnK#jMf$_51>14x;&>n|NtIr}G*1Jn}aEo@>Ks7qkL8HNIf& z_DF7}W<*=H%{ z?G?KBJ`*}{j!@kqI7ws1^U;S&sO3?Rp3xhOFT*@eFb|{H1XX(!6MZ`gnj9bi?=qLR=vB^U@gxYKf3~K^d{}RhHAOeM&hr-efEkJ zU6dz!f0tvA*4?G;HDstF@1?0oDg%OHiz|_>(praSTHxdf5!#`wmdn%yvX zM0#~NgXu&F>vsRFeXJt5h~(99NN1!XB41O*(3SzNVJ!}u;8M9L6YQ_x+S2{-1qs(| zPMgD|LgtcUB1XIE1bR$FjUy7cd4BxjLVGYIp$hg2kOUhr49dtFnGequNSzR9F-+vg z11}He6F53dpgMVdNHMvc#1z+-9nN?P4X(3R+i%E2$~Q61cEJWJ{yG%3Vre#|O|I_g zHg_Kd==*1D#dk8qsfd;?iW{+BhC=hz2`QorHX}IDbnzGwF8T1PG5KT`k?IW514D;I zQgCQfp0%yzM_cK%Z8^Ogx#Q=jKV|mxb99uFzE6SonNitK?KP^F3iGbc|SZRHR=#<+vtZ*0nhXu+I;}F!O;IF>50iCbB)f`PJzo%p}D>0V5 zvU8t$6C(FI!S+L(`-1n-O5s%!;!z9vz(uB3i`pqqp ztJT#Zul69(K$S=C1--E1^-C-#$qY+DHXl}0Jscj4^im@*Ug=SMovG4ugt{%Vt7?Fc ztqI+-IoqOI{ck?Kt1}mX$052CWx$hZ2;S`i*HAA^DiZfG9vYu%yfM{8c@@Xw_r~S! zeWR6!Pz@CC%S(Pwd0DFYv_eC*nvJ?8p+R4t4XTWdUnX{uPC%$)E7|-re7`mvw*1*Z zN(j)PKrPr;1M6*EX{3twlna@MCV{!iA>vw1MBq8Mrm-AaW8Y%>Haha!madnAf-!9= zllMJAyDiNgs4+z|*?8f@5p#7=#{oGs8}5^_dnV8bRL^^o>;ms5z;WJ9tL3IYnt>)b zImDuz*0vp&d&r7i zR4&MsL|yWZyJ@SRwMwoE51W}3Mib1Yq}EC0Ci4yEztS^*{OaN@#10Km008WJiU#FB zRjO__R__Ane>137Xeq?4aG`b_YoQKH>hi1;`OqLfOpz(nh@=A)T;ai)8CAII&`T54 zxnA%6#lK-%@DG2eU`mmlSjgb+op!t1+Jh>0C!aIv6>1N>-NM$T%@*{%D|Q@EWbxdI zMUo2q7E>L~PX7cps!4=-&Tzc5?(C+E-T$=R&gZ-zsv@s@Cz*-?Hyn$<$8iG(t3#c{ z(&Xdxr8|NKz*((dFLf_Z#~;4Tx;P6h+=zWvl3_DT7z5n-v>H%x9A>(VuP7X7xX>Cl z(QMbu7oJapH(i@89G+!A;-T)|K#N1PM0E>d6O?L33r-R>?pV>8ZfG-TFug{9F)pQJ zi?Gh4S#S5hemXm&T%nqEsz+a>If~UAUH6{)*zp$a9AcAuf!|b%9A10)V^VS)%boyM zEabbc48U3kSF{0ZyUZk}9*`^K8WHh zTZJvEH=DPCz;#?I0HTmSYvSbDqF1NOju7A%I6#tZ!f*FZ4X5*$Vx30QNrsC1UmT(A zEV@%qLdtsh`4|OAvAK(kD>@Tq;h*f?-I1)htg2l(;RiCjp73}4I_1@o;RnmTAsh4I zk-VWs31@E0QlB~XprjNsgMKma%q2vZJE1SHzs>BJ?XZ5Zfzk);M}YDVX|K{3q%S=B z@5pmiXc!Iobr1s8{ce+BpwpN*?|cCTD-2*=_RL}qx9!LCa!q?7dUuzz1acM0pTUSn zGLmYO8mliMRQ$qd2?8TbUdpuZl z;Zm?whIT&eHc}i&}Ec-vdsFpsc4KYsqCQla%AEfvG}Yn9rD1Xb2=CWdbt++d-2 zx7kmLCEP6>1+Z6bl1aQ}YH?tG2;V!AD1G@+S6l^3wzhrwJqYzh4j*)v2}28t1hwR@ z*ghz7QV;G6H+&uMAumfT>6be%0b@Y$pcRd?cOz|FlY(K6^YKVujMygUqi33A8n2v= zNBgJ)z2@dtQr6<1tMW=xfw}CyHj3)@F zL~*5}*3jCjK4)S7a|Yx1`?;3j&}$>9Om01=8Q=*1o?=!4qEK4^VtaWMbl`h zqNiCTkDlo{0oV5M`rlY+!26diPe+Bz<_lJMCc>@4{3~iDY38vWZ>wlL2d-wOSaCLhZX#Y!Y z-pJ^$?-?192$E2(g*xq5J z)_`}j+~29N(0@_Q>}@O@Ke<`BzK8jr8MkdGBy-eEuuce`$n2*ni5||K14DRDWUr zp=$s4ru{?xr(6B+)U5xc{y$CY|4#q!O2EIi!Ph%lixZ_iONfiWKVoR0jay-(NQG LCzn+IpRNA`+u!Y8 literal 0 HcmV?d00001 diff --git a/python/lib/py4j-0.10.8.1-src.zip b/python/lib/py4j-0.10.8.1-src.zip deleted file mode 100644 index 1b5dede8f2d627760e571cee6ac08b472eb31d34..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 41255 zcmZs?Q*bXpv+n(mZQHhO+jg>I+qUhj*tTukHdbsW-@8xMKKoRC`(iHUX6m_^>7MT2 z(~2@6pr`-<01`kM+bl_H2(m8spPl`G79s!_;NZz>#b97yX=mwTU_kHSsiq1A0I9AO z(AYNE&~f#E0RVwM0|5a4=L8jX$a2X(_Sxp3~6c~Ww|CJ>zK_&MY z5gu!T2>|d}0syH0Ez88l($vny+0@C+)ak#a;fkc{6#xIFUGrHxZ;2=EJ=2(7AON{r zs&{YHY#OK5N>ePvxtUIoP28U}QbCf2BNHR;1LvFS|NHrTHZfylP)xBgzZKC$3kNmx zGW*!!<14~AJRvZM@@yHO+KTk>|Nc1HyPFs=Wc1LXV-H2W)?1D8X`w%sPO~LpZWvFO z(HcVV(3pblic)Jy2+f#y5t@7;MDd^FH#-dPw&Tzu+BJLZ>Q0$|-f8Wrl1@yosa|Ud z@gIYLC?f+*;)F3B)@pUk$DV0X9hbf9sqacFO`RIxZ0z&qgMDdhn3CWGKr$9vG^Xc}yD^#0x#GJ9>&t?a07^M5z}f-xlQnbCevuw~mn7VJMW$kuJW=q|?=ME|w8 zVZSc)qLf(952OH_mRKdZ4$SJ7zZn zcm-6_V)@&%YxQ78CSP0@y_&CGEKE$CJpHTmzPmYm$m+_X5YYGccg*eW>VB&ucz0d< zeDnCY;qKx0<>~Q2pqq<}E88>3`76uttn5gBu^p4kZrh?SlhI!YjC)YIWV5sz3rg6O zYud)FRxH2f`fSN_rC8elJGTXm9a(*Fi>@}!6Ha{F>i_=PTual>;l~FNS@X~PxmYVu zu^47z+BdqmN}UEV+y#!o1M3Zy)LkY0Vyt>yGRztDG3@>95mrNdKMVo_Vmzu5Jv@tI zZ2mJQmkcC}lp$M*r5|5oudk24Z}ACjI1^oMIeghY9rfk^=Nkwe8ib_|fdZmACsa3x^3-&<)N*m$n9JBG z=vCPF?ji#pST~LJ(W(hNVi%I+9}_N4{g7R`m&zg~`dfHccfy40bxdSl>ux8)xzXzO z5Njt_?Ai*p_}x}u?@?)ZAw~(2%hxo3?{xr2is5ySc z^y<-z?Gx-LUG$R)d>NAD9=o~NWeq+}xK_teoBo)>cnA_&s3$gCc+z!oN=RggzdVe4 zBnh-q90?BfGn^a5u@>x=qzh_r*Np4ja$eln_M<%AaJ^r4v4$c3^^)eMK^eyNZQtV?evH;>uXHX!3=_QHpyFZuEFsn%)}`7IpXB^ZRJ zO=`}vCD2hri^si-qFz*qE}T^jj?^-2lPePh>5+`uPO2{z@0Amp3z0U)Mo}jogp^2u zDQw-+1C@_mI;TMXn0X8JxbXYEZqN5sk5hfD8fDpuW0;s@Jdlx)1|f0gEUDyQZ7qA7 zdkB>|?sYA%>ItAHhfcn;yz+BK8}Uw@l<~91@gL?Ncv+gt#`UPoXyPY?#H{kH^4|nW zUy)^O%LH)DY9Pr}lR7#$8VvaM8EmlB(XFVi9A6u#8%jV$+UOdZr0j!+#)j`2d4?7V z^dF=SP~qgoSd-L{oQBF%IZ)POt6|(?1ir0})xsJc+`)aQq$8*t!5rxFX{$dBi!Ymz zsnNX^^VkR&S|=DFnORLm)tGQ~I{q>b0Yfa5y{c-fb1VXjk!hv436@PU!f>Ero(}XlTq&5YXhq=Xe=_l79Qlqvl;7wbG%b2iG0zC8cMMCc;;Xy;5 zm+6fqHxA|FrW0VqzEh{AU(`T6&^4me{Jhtaa)@^(R(9BQdtLGaTNCmg(pqAj$UP?t zbsDowfW}+GA?SQy#vh)n;wn(*dstCeY(|+8IeaGxNT&;}!cB!d0%`vZw_LTUsF;4} zq6pDv{Hs$deE2JAYjF)0SbyY<-BIAM7|R)CvG{Dd@orML>vk-WCg#rPwxx0Vy!eUt zlt@BP0^_YmD85j!D2E7MX|FH1Gq;av_S(mmh{Y0!miTh`~F zig>e!{e}5Y{^~raKe(rmr~|41rP$Yc5X@LfAu)L(0Ez@x^&{s@9E;ftMkra{b;oYH z3`5KigZiyZJpkul+f=g_?!W4>ypWaIlb}txR(JGi*}!dhU69SP*?{>d01JI$DxXms z3p~8IZjdC0JCnrV%bY?F;iL{S_zkE%Fa=@L;JRB`V}}}QXNd-_2~Ha` z$08Id&@;L$h5P43eS7}}PPowK9S-8c4jhWOsd_6`4U+h8 zL8xHRIcqtM-Sz>c&LI`L>;zEjcSOdLDR*Xg-B;#8L~YmK)b}J>YT~>sg|5VJAo0U= zxV8~gn%W&X^u&{(eak>Mp_6{Smonj>TRKNTB4_rVUcb}CXcTgUqz!a% z!UK6N*-ZYr8W}6v!h|9A%@pfcBh6#uMB1XfrWqKl!q@b>?Hx8`6yClI&sjzjs~B8X zy|)-&NL!284cp9)XEAbI#fuS_utNadbFQ^?SH8C>&~85p4;N69d8l8pL-bWzJ<{<+ z=iS`vG%`Jy5po#yhQS#|Rv^xElxUgRxj82sK^qERNPD`Ity7P6^7tR$NVA!y`7KW( zt#q#l$kC?=wgWvE7RIrc#NtE@X67-0H136GlCI^5xsZmMg1FBwh+H4FdEDGb;AVa)3&Xhsr%7+)Ovc-0=qFfX$)Ki6RTegi(gu z5lk3){(9v~O)?Kl9q#|=cA+WpYPUEdwA4Cnip>ztHvxsD3e2kJw4$4;f-NX*t#XCj zGAoOPyc<3Ss9*^NC43Rm7hDzvvEXT2=Xj(cAt;{GX1UUr`3$b~A*SyW`>SR@+Uh0Z zTv(WTHwi5MHnU}mwfI2wayo#QpQ7k9+fc`vrR4Cu#R5s^zRw zly-5qqW&bD)z#?Gh!%8dWNVGCbf%7NSc*15OR7ru-L9BObt0yxI|-Jl^YtLvb;pPv z9$|H9^o{?seB==`nA=-b#H#g&UmKd}_&_rPPkxUiGCL1*$NeA*MM~3M+dlQW@Ro#v zu7y6Bb3e4b-!KfD1cY+#5Gd$E?5}?_M@5%v&IMfgV4?1f@A|C?_tIm|>N;((7!Qb} zY1faLBjD@H+#m_IM^|};3OuB;mHC)m=3>YA{*hinaE5Ny)&8(Y&Z4VIqZ_^p#)}R` zH2F`Rlakb;ox3FK}bTWF$ zzeZt=`&Y#8Ornn?D0WwPV{_H++Dc+bx3M?YS%R2Rk?gg?EucwOO%Fa;#_#O08j!nnS|jzt$*8t*G6B4FVa1#c1;J}vw}m|gvzj= z!P`Umb`csrqq}Xit2>JIt(Y0iwSu*5TrN7dj}>oUuuQGBywu1~{?vc5SoaAzEs>QL zwd+}Z=N^7VeLT`=vxP^YPz;P*F4S9brJt9U3r2kdAm~zd%+uBFw-q&uDjTnR06l?N zCh2u$#&lyi@dm^T`18TRWHmy^W}b;{IvDlz<6JTFGJP-P!J+JYcyGmj+LzB_hGKKs zGlZi!bOS1LLo3Wy6O87@^H<%}_f%`h^$p@YdyZlIh`|VaTjIZ;EY#5w#hDQ zk>$SNE;k7kCy#ZuuOV434_Pg9+vI9ze=z5kC}XwdI(H+f7ngGn05kC$yuK-G!e6PY znjb0*q}V^fe|!~iL)0RWB<(rb@d{ZV5T(=Z})P+;;^BZcseygwsQCFa=*_A2UIkyg8>WRrIDM@ZYdB z=APIxe}%+tpxN}EmQ}fwl)vpjtUTQ>q;Vdcjg0#8DuYsMHt*jg;Bk!jn@8+%It`KU zP2SZ=9;9_FBenGn9!E`GI^eQGy8{krz+G4{|gj+wX=-4VSLXX7m+`|O`&l{d?&l;@*=#50FR15WL$}^AJS&? zWU3F&cy(yawQJ=F#!>lV`kbIsJeE6CW#$Zr*S-bJrX!o#hi}M($UJ-Nk0G~Hd=_+q z5GPc;>Zka)stq4xYU+Ex3z4M0`Xr}IVW8_wU|TS<=tAdapBagseRWTuIv&QwYh=-h zfx17w*Q*?Cul;(H=Yg4AISRUcY5rA9lr6G3#O`7bw~2gTPHfa=bTW+N@&g`AbGjJV z2x0#+t~JAb5?0iG&^;_}dhys_{Q)gmP)rJw)F;xgZzm)ZPd&n&sS*ArHwj=jN%ATQ;Dk>KBvAyB?tJ%0Y4xxIh&_3(N!;l0TRD`dm& zm#QY_gsOgUl$zNDRCkJED>oP{d)MPtGSuJ;9V|3-V#{{gj*6OCT#cvMdG9yes^l~f zw++E^H+I22&$_~qb8jS{v?V@KRB&z(yKD`^57+Gc}BJ8HWmRtwiv?BbRg}xE? zHO!_JT^xREF>-#yz~glWnScKb!f=y|ga&uU1S*%>z^8~w^>|y z+j5tKc5+VlyNEv0+J}Lc;U3CAYt4{=uJxtc7(D8P)br^#oRUMcpB7p~^D>=XgNiM{ zAxAipT4AHa?}GEUr9L5VJF4PDX^ux6bi1fUuUW9;V0ujFklZolLldYcOSrbO$tLqz z(dUpdgJl!`>T5nhAnQ4ikR4$MNW@AZE&qAvW+9Ao3LFy=Ov?HuYg48T(QFYHT=`$)%W6`0`K@4{J!7)q>}sQ8D@SGT#v<=+uErL z-(kze1Xgt89eD5W3s%ayxzSQL8Xm#jV8x`~VoW4+5^EWr#rw}b&nbJ)oF~>`-slT? z_AuQQHE*BPFE{v&H1FgQcgU~oPKGN)!5Q2(y$)iV1$V%l@JS1PUtfo86BcZX6Z=S8 z!X!70I*U)wUgH=xh^yRGyzL4fhr4zieC^*C^HxiXN^t;L(<;`9&^pgjYx`cGFKzsU z1UeI&KJgUVD$uqgtLsJ+;uhNv^wb^4K$u&JW*P8C*=iIc!&>(0j=u6F-i5rxXzF5w z=Qwd%7ovY-rsfxw%hMN+PqO{8RMiqngOSp&*>Y?*&Gm=|%!gU3ddpURlG1JKoul+u z1P=m9)k#%ogSKF+*4FA@;qOStmq*DN^Ng>=z>)qd@hs&M?}C}0`}M)}n+E?s7Lj~q zdkJKC>9Ox=4D1VN|A>mX39iZdL`EOu7W59plX|`b33@uPD(EkG4sqeg9>g_{$jH}S zX0Ih4tm$?08*#hhbMPIf(E1r$HYT+XriOK%($UD=p$kwmQ3Rv5wD8`n`X$E{{w_B= zwfGh0GtDWE30VA^d-)ya=~PlRKHc8QgKq&50=&0c;5w%n01M?JmQdeLIJ_n|J4BEG zJUT47YV6@!?I$aiqtcZ{DrKL+Jlc$Mk(rOv-y6j44&#+dF}3Wh9LX|Rm!+Lpk#rCw z#@`=*--S|V;lg4^7|4^txc;v+uvh!q7)+TPe81dsD`*bkWyh2cmn< zW$^hfubtt1$OCCU_8k9NhQqy?z2_P1iqc zUprt`m^YxtXJKG-oojkuAvAU-?s%zPu4V|1@ZW+)`e#bx z_E~+qb=cJV#6P=qh%kt3QkZRvi%X_LAkwovBed}n04+qF-swj33W9bqyT1_FR-E(Xf78}0g!(r4d)&f)u<5{d<9L@r1cxq| z0DdlXK{Yk*Ne=u5f>0L#kmr?Lw&AQ=`n%N$Nj;lVS7}>A)06DTFvud3c3%Ju;=_1% zO^VAyk3TIe)lzxv;VkbKt)%Z~BvW-76L+vI)#}bV=8%~aU-@n_I*r7n`IcQucnmK{ zXYa)>;)N?od#N{v>>tcT_6Sz{_S(PD_6%?S#Bv9g^zKpVO|i>b^3hlmp^=L%U8-CI zWq))wu$;K>BM!Qp1J|NIy@TPn_j@ZZTz9-l-FU*gz9YQ-0+vHH$~%uYMT?sEim#>C z)H_o4_|qO-=R&P2;?0M-D@~NoAZt)76qY6T%8+vwuC&C7a#aFYHl`3VXE_l#ev4Hk ze2SC}qV5DjvwM}B;CF$c1;k!1$>o%Lm$&;s&)NKN+4wEqIhP)EKgTj(c5dGTEzQcu z{WRy<`12big?E^EVK|m_=AG?{Gx-NJ3`+Wu*ebIrVX_bq*Bax55f;f1Xl?Xj z4eGlzqJ9`?g_m6tkmluLsk7W;!T_nAkzRjtq&lbCq60SJY9+*X`f$1hS2XosP9|;X(>;*1W#=8Q2f-v^_c^ z4Y2fV*q`nmt)FAkiIEYp2XV9-!FseNMZ*a)Y(s}3GL$b8t8p+EaMmnwsiC!EZECq+ zj&055W5f(vNtn!H0jUiv@(Bj)%n{&}5DwX)c$l|?kynN!Qj?mRl-d0(OPzv#FNnTI zIO)+?g^K0Mu$4+1+bYKvV0{j09SgEDWCt=%Ftb70fSOq3nh~{m$)J)N%6gg>7N1Y; zlh=FGChvFp8MPh7j(fI_D!C-mf<+zsZwMO4_yO6PJtMm_E#N1`FbxNe0yl_lp*AF| z8PVYyHqZby?{35BtKF^z%|h`^l-$1{AS}7Y(ypE@fs<)lQN>&+#xoC}o>?9QZD?Y! zf$i~Xt-!5l(t|lM8O>&>InND(smq*@sh;Ni;9LScs|zPxfK!KHni35@XU;Y2Y_XppjB6UOid-tv9 zCN2Yyt%RUUm1`&ksb!4nfzbcqi^xrGW22hFx;HKbn#qVT_;3d8%|J<4L3tn!b4mFA z@jSt+2-CFgNUAt$$AIJZhsxvk&TQ*~&G|bMygS!B#}qy<8x|XO=_t`r(4GZBG}Rp`*mgLHY8=&DL7gwRXS=1Q&gMGZUm*E z!S3oN!3v~|M8feRt_6MLZtq8%)*H!JElj7(3jObi9dF-zu7}Sr6 zhy>-D-ek@q?3fMv3%fvT6jb7Z%}rY7(P;Cks1rm;RQTh7v*%A@4sXIlzJWpf@$b20 z+~;Ks!ko#|Wm($f6JHrJtXxLF>3m%9hDA$(+JD+vlMFx9ehX!{m%iRty(4&B5AH9% z*~33iXx_&`cm{R;?j5;qE%+Up6z7a1A!YUbC3&8crb~G+&48XJ`3iqDqaTFvBd8{b zBhTRr0hJwRc1!L-S;NHKFg}M9%dFr3Cp!FBGL_On*U*6bk3~To0Kok3=wM~!W@KPu zZ)0O>;$mrU=luVs)&DI>=-S$EaU%cR7z{Y~N$RXgc77fp+Mth{?Q}gQu}18Gq3;Nn z*mA4W$nY!4AJV+^?M>f_73 zVf+rK$+vA?=a??35o^*PGs~FrhTS&NLL5G6AA>R0uYF|=-I5|D6VEEMOSPv6Hxip* zTpGvGwKf&;eA=+alj%?%+|lV6b11BPsMV7msW6!yv1LbL8IXzfEnVQA1!2rXKj=TE z^!RL;FucCJa{mSC4N`gld7+t$?vNZdp7A@pVk!X#WWWUD8wd?hXR|0xb+a0`DOU(}HKu$JKWwy3MTp?E zuOaaDkliUUnUMM*sJc+rfU#7aXl=F_0SJCc|72iB{8D>FpX7rBA*r z(gw+yr+byn;y$)EQ}IE+5G3ppAEI@l%ft9uGLKPlESxdMBhuW+2`}Jif$V{10FW-r z5ib+J3=YW=GsYTYF3Gi3;kwugme&g=thB#*6m_xWMiS<8HF6LYR7rh&#hp0^7dT|x z8&I}D#tWxfqR~@=HWCgNIrRstvL>^%11VrS1(+yzz+ZM-9`t}{iac3u7I>lW-)nXE zqfL9(Gz7rKFgCCN4yZ2!gF6lvGt}6|>L@thSlwAG3yl+FrCK$mL`<6Eveq5#=XL)P znG-m^fk@bEdET1HnOL5!c|r?T#wknOOGg0l0#U+ zVDuLyaEuvk0=y-X3M2YTdzrNtrTn8Gh)V?V1+M>1WJq1(jx$v!V>;nSqWUt8=0=eu z2+lK-6+hdq>P~4882(i$|I_of4&=r2x6Y2>VH}JB_!yo)w7qL6{gUzNoN40=6kb@h z(J}#>y1H`x`wOf1S1syB*yUNk%n=(==bQrAVm=HkszYn)FofP*ZLrR}S~_C#MGQ+U zxG8#pV8MZLP!Gi-*!^~hp6E&Yg(x}D0MRimqaKb@iB)l+x@DFW$d%Sr=KuH`MU+J`ANtLmk zvRgVni5K&k;soc)a)tu`0Z_1WE27Mz42$t-Ob)t91#DOYg%4S(G9zL&q*Xd#Y$EF( z>cne`SAST{2hJ2DMZIHxQa9|0d`ckkz!Q2^lL3tMX;a&P0J{gQUULyjj2o@Z&TpuU zK%Pga6NS*8YT zdDl%}uG*Pu3lc)r%p)X2*?n2riHoBN&IA#dW+LmFEeA;@WYU@vl?_?uVWFO8?;@QA zTf^S~(562Blt@pP@E0AkpPh}H^^cy7p9Q~q&&yq<-{}J&h1vV>NZLPcQx>%cLP7~M zn!V07>!=*P8(rg)%`a74er;3o1r)?WV@RZv$V{ksk~mpI3ebNl_aka+=Kt0Osyri7 zC^BzJbN4|<{6SWjc7t3ECs=e`wi_kL&BaB)ANzttrM_a6QS)RQ|_smI|l56@>);{9M;hthOEjZKafUAY{$^HZUx4ZX&A|yFPT(sf&Q$ z?Ya~uNV1O43aEe^pSA+p591+%rK%;oyjidVgKDKyw94ks0&?lU=84+J$yF6q(wq@S zR%RGHQnQaupz`{M3hicVUU;lG+10=PW?&fk#W;#c&h(?Ic3Kg~6{M`WUawgX&^V3G zrncVdD&YS0Lgf8gMbsr(MYksOoph*QDVRzt4?wQ3w2;-7lqa#lZ1s{AN1Yec4}^|K z07bh@!w zYuY3YFhGsOmfYF}xNvj&y9j!D^&hB~m+hCGI=82Buk_?R-X7mL;!13vGwPGC7W>x1n4+(W4WA5kk9YS7C@(QqKVZw^%LWCq~ zhlYop?(BYwhr*e6b&{4#8L$bf9oTE9^XvOgrpruw-0&c&DpNIVv`YI7{SApd)C+9r z{M$-&v{b$3AdAy}l`cZlZTjnb$w+s(P&?cO?j;3s-8N~z z$-xd|U%A{$?|$DTHc>Ufp3<{vnJr`RWux(Pcy>U>=qMnJ6_aNMvLxG^J`{U$_Z@$w zpb%C>Cv8I91o5&rciES>CNS&yy{3tpO#t(W9A;lmcGslm7N4iB4+kR&wL?GPmq2oD zp|B9y0->O|)RoIagw4>BBVQ9xogw1&!8z{nb^7&Y(G0^{7_3wf=4TK=?l&o!ZqM9A z;2fpA;L);1jx=1A1T(ZrxP#WFN!InYbIs-L-C0#+jgzHZ!PT2v9?hY(?A}ml<8U7* zZ>bkUp)e|j@ESwV-}%SZ?@%7C1x`WepGe4OYWlO>;mu^*n7C&PrK1TTwJLWFtCOOk z$AZ>Tm}yW5C&nj*G*Cn_O4Dp=aWVW%OS^$Zn=uWRhdoOJPo)WC*G!*nK}5llBTuTG z;xZfmlC+bf=|iMygPKnJ&3#LEFIV8hOC9>hmr%pZ^ZL8o`$l(&BfTr@{{C)O9(Ipv zs>GvzTN~um>F#sisa>VyqvW*n;NrpDCSg)7j^Sq3bHEOCCvI^DXjf-Z#?7)Io^vL- ztrBguh=g2m4#^EPc`uQy z_@u(KW7S5RK9VK>=y{soPv0Y755swk?J+@G=M%fpNyJ}mSn79AcebEw%Wp+iK2aUG zN96<$CezxPv0!l&0eDD3DzvmFa|2P!`LO55?!{*`?##FlU&`JuTywTbtx)|06N8c^ zZ7f*92V*<21YZRf`bVmm(W}<7x_>if`tx)(TF*Io7mf~-v#v0k%Q!FID{9_fY^P)u zw58tBz}Vm7X}$~YJi7I*0n}Ygu`opQ0|FH3VDzt2h$?%`?AwEqy6H}KLZsC71i8jm z)6mUOY`irJdA;R@@T*>!K?10pZUgs8`{?1@Xff;{_)u_!`+Q^_HI4GeiiI!Xwaezu zx!`^vX@EIR!C9MJ2){fbe%p^6h?;ph>A}@GG4{73(|JP(!>G#c&vYzvxF=7?_d`IGY4mqfI{!TYBbEcPcBQ3RPL#}jdFaJZg_r(9s>S&cZd&nA^#NeTJPcQ1E)_C_HDOLQbK?YKdUlH}ei^BN)^uKlaZ~|+h z{yaoW_**4Qw|b~Kt4hBFU9ws*F?R2wRws=%Myy8*V`!LKMrev=PBqC~B}_D?mRMtj z<{SH}H9?0`eMTs}Oe111i6HFg z3N4!<|0=zkX0^$*Efx4B&<7Gb01x)v zzj|5vnb&KHHC^k2L5&@&BTeObv)gH=263v8-jFS%`mIS~q1XD(qr>lgC!aQ&xj4SO zXm`z7Z4cvwPN3l$K`2|%lL9$?+nY8Bq;L;RXeGH*#i8y_F#Kt!yY`5R))cM!G^Nj4p8ESL{5WV! z1V%n6@)~G5%!;OORyo=1VE4mXzO*@^WtH~Y(~36&uZG3IhcE6WgyZ&w%e*X&WA9|6 zy%26DHHa-(;?M3SigRQZbc^IAz{L+c35OEp8~&FrP_`@?+>?eE9vr{r$@R2R6QYUfx1+nQ=*PE*e}~}& z-{=f@3q?f(Cye;HIdJrL`9l{63;SEqOGg7}aYZ^EN^$TC3I=%iz61scwY$K1GdJ@m z*|R6{41LjceF7aZ@$~Wv5E4Vl(ay$Rm(lcX1o}JsKK?hgYAbMJ?y&+^A4;{VuPn4K zCDNMi%Lh5WhPo>wPEdAlf>p4zNSs^TeLEx55EwB|JH!u>afYm(0EVh@qXBdQrdaW>Y3?4ZE?azXs1 z#7?D|r-O0exfCfT_p+GM5v*f0ZSYhq$4(+Su zyStyWj?)0Pxr_Ue2sg)(jhJl7$gvkq_JNHCal_c`Bgv=HEaA?mi1|Z>z#v6H(yiDs zNl?4_J)NESj}alfK$g>vT?A2F){(K|FpcDZiX=Kh?hYC1g-%`94v=y|t|J+9KZ?ok zS#Mt&L`wdhRO|Ycwo+Fuuj;dsI4CX2&QhPB?|ZtquyNwTA3hH7gn&x`s1G?bFgVil zc(B=t6PBBsyJ>(ks-#B(Tm@6q440#Vz$yM&g9R0&3Adq+Byr?Yff?Ri@XrYOQ`0TM zd=io}=@|@b1ZQ6Y9pWvy7yPj!9#m5=n}T%>YYxT=Kuh6D#GwH?9lOU%s#5F@OC_vf zk{1*|^)e--F9sko$$s73{aN?^B4+y&a2b>raMItJum1J%amd=+%lYBQ2)jeVv7*R( z>p2*Dk0If-LL>Z4Iv<456s~qdmeGB(H(K0;_a4_nEh_ z$C=i@N17kFFCpYC(?WT(4wqSec6OH6^=ku502HA$?lq81z2!zoHmLC{x>EgcaI7YH89pC z=96CesVET#P?I*NqZCmKs$QI=b@ghs}y8yR;cCSjE81+#NJ#Zd;xLi2!|&2bi|@?NpbKl2 zszls-k9Tq;w{}K}w>3xPzHHyY5#*aJcqElTLc!DSDQ)|R)wrSqXDMxx1kv4GL)p+s zV3?bSpN-wWA8ZKvoam2Im}7>Gu5r|9+vX5=04;%jxcz3jA861VA$rJI;c#K|?gPrR zmFN-b)>He%ofL>Xow<>O5tG%Ro^Es72%L<;K3kmVfK*_0Kt&w&kE8rQT(FOuSLKa< zqXj0J^(Vb<4$_P=iLI#01N9BoD*3>1hF_$fM-IvrN0Jzfn?7ekJyLh~;HrpV48Sl6*Qd(3>TZ(tZ45o~>Y;{mtLQD{)> ze4b$?mFgZk=Fh*wN;9Sv5#hE;7BT0Vxz&@%tR8i$nMtWuFJEBlRct;qE3Dt3u)Mq< zC=IgdF*imS#A<#)7I$W_OkdKxzdFso%*&I0W1+yQdsMiW`x}+ic?F~9St$T-00|*o z=-zsTy#Z96fSZ3>iHRZ1P!2&Z*}lMQ^Imm^mH-U~-P$fK>H@32bRj)MQp3wH5-r$^0v1{-y3$_Kgxc#g!6f5&0! zu(ZWSOg8von9x-N#Tm(PlQ{xyWAzwC#E5L z#&!0h5Mve&YLf2Gv=r5?lxY(>1>BSz`h-U!O4Y7Mf`vOK$shx)3ffWgDO}AFmAGe9 z;MlfcBMn(Q83vYtuQqUEy>JhZ(?ub$X^AqN>)2vQZtm%_5!0ff zqrE|^+Q7|st}&FWcdylyu9km;EcThBVbRz?<~=}F6)97_uNa-TQ1T7qRu!jho^TWf zQxmsFb|8TI!_LrdsolZ2Q}~|6tlObD)uC-zv(h|rCn`b_yt@(h3Ga5HHoA!08{5q3 z97;_y#ny>>O?N&2+Q>{#AVMGV;hA`AButZNI+zbPM5pO-UaMk1jG*oWhXoa9#I4p& z6P>taERlg+5CB?Q$0;Cu#TM40THe2tQS2(uEBBc?d>_!PlVqH?DlswHI6RU8t=Opi z?+eaIv%3Dkz7T~q%$bIbK|Nm#F|m`4M9RT!!3~qk9Z`9lk6AXRFn2EdOyQGqAL*l4 z>72Y+uWjZfkw}O^I!wFQB5x(yM4NafYPANrjS9MxC^c>C24`8HGyQay*${@A?P-H} zJ~BNw8D`5=yQLfc$}qBXD9ePDp3&Mfo`_k8RkQV4A=@38Ni~z+n)X2YKG`*Oky~c+ z^l$_6I0zTglGaQmhZ6c4u4QS5%kmE2fY$JcE|*6o$X=qB(Gp{8X`_5*GGF~tmIX;G-`u`}&@@X4fk|m(!g4a@L$+;1F){!Q=N}p5R#hty z*LKpKBkmu#_A%5p>H!e>M&KR_mIQTf2=F_^-Gy9u4@0CVCae0Uipc76e8ClV-6Pq5 zV0O;9#^S0bJF?2FX#pc)7*QLkmZ{J!45F6C{2v0nJU-r-{@#x49_;eYz}@ZgHw{B! zDgU_%l7br5VI?g^BX8vwGVX$8hhtPA|9WY5w7!nY=(`(1rc~dbVRT&Y2w@!p_B8p} zC5f~L*nn7l5VGSyXf+cN>jtowDLORGs7XccIC6_Rr(fc7FD6HQ6MtJyNeoV`w0N*7 zRQ|?UlvA`T&0gp@@sFDr1>p)xH9pCQa?`a2K;-MGJ3Mf(hR@y4}+Z9x&=~4^s@T<LKN(%hqSb*B7<_nO^{e zRl_;Eu?spRl33U&TxvYUaPFr-d^ZIvvx z58EVFytFsN{G&r9mE%dNF=KVt+y*#=KTt=O;$YkEBf?iN_LQC0FwM8QYJM9fG&83@ z7@A0&1;_8ZW=*{-Vhl?3(9tdy%*hos-Rh*+1XCR1x4Lc$bK(!z$emt-<+itguOux4 zK~19Y9qzZD>maWxc2FI-1?v^EgSy+D*?B+X+B9pE&+~dLcYyLi_KG?p6KGGtQt}8A zEv2)%=P^|krJJ)nK0H;-C9JHPz$T%mkmO&d9$B23!6FWz5VX_Az~9M+Q^E0BhEAbf zn#iWWBf^7nrb?1{g|w>{I-)y+#cO+~&YuPR?P5u*8=`aqFtx2n-z5ZFu#%~TBzU^Fq1N)}{n^9LJxkM17@EmBByy4TB03Fo0CNAs-1VtOIbiOhTV z8xc~-rS6Ulqw_e$)Y}TgCCa3Q-~m~N9Qs-(5#Wo>ZKj?Le&pJ}9fT8LULo^?xPT}8 zN~?`j=1}ULqtOF@aDSZA)UmkII-x{6Y$rQ3q{&~+^Kz>7qv;MFyl`cPN!SzFqY>Gl z{Mi>cq$&k+fjlv5QKN8eFNR%a+BLrx+=%igEl0+0KxW9ZYp&%k6IcG8cr$ zN904kRTBG|6!Hg@f3xm+R-t%*nT_pL42V z53ZWMtCi*E9AQj<#ul|4x3s63YzKRH4*aaUS@a(J_}Zzt<560*BJVu2E-rqCalf&D zwX#21AGv|d6Xaey9M|xe;J-xT`Y6nR@mFj@E(sT*9kaTnp^7UqjHh_dT5dD2gq_`< zb=}Gvr{vp=xAgK0fp(FeJbSCfyl*$M4g!u3by)E-XDzrC9(|F~sW}NrX~Xh1M-rSN z3Otrl(1<)n0<$Vo*K!RyQJ8xQ;}jLLH1LmJO0IwIxUP95^!j#_FuBX_SNtPHf9GEK z>CDVKE(Gg=R{LQrR+xOI*G=%vx`oaI0oV22&9+T@WExs_T@G*6+Cn2`qyAZ2N7d!8 znEOh`mqu6j+HY5#z?koM;n0U(&Pr*~AUX^f`R;@2mkkAZVn1VQ9r@%s&Z}ToswWB~ zFLv1ePCnAfz!%ABw`BdX=rleMIa~TTbUdyS7}~j6j?xHC=4iCVE~bmM=l=&kK)}CK zzA))y7MRk*I;u*Tt!A%@wMp4o#HWQ4CX`mZz8;254-nq;?p~`ZFf~AA_lG+)2qnttGLIEX4IeE)sb7Y%K zpqc2~>Yu3{8)^@(7k0SNJnU*ugZ6Rv;}kaELKOk3*%|@}nQ={%>?G88q>r#uGcBX( z>q^y@k$a6rZYF;mC;nB~#$88b_`qX%YZA$)WBGiLOcYOl${$6Gr}Ho|&^j72*oxR% z-m&Sc9Tl-yRkgnu4XJMk*d&0Af7Rvgk9UiR(w^YP^cC!AxL&FoJb;^4CwiI})1Wz_ zhQ9kg)G-)rGlDA`zS1e`I~ip%pGCwnc}zD0S9L=Vb%#f?YI{xt+z_f?#F9wa4Qz2s zT5S)TcGc8PQ{dRrNb4q5rzS;Ns%lx}2^CdjoBeIh$yXryjLo%TvdH|<8-HT3?WFY& zd~0I=icV8=6XP>5w@X*D=4Ek@62T)N-W@(q9;RzwT)4h?VrfLwMvHYo$MxtD%Sc-cQE;~v zUGru#+&19Ll0*!o)D4jWxbbO0N5*=M#W45D0=~;)z5V90V9SbbjG+-y>tC}XE&6?N z?MUE-GtI~Dm1z<9a~7z@MAQ;rbp7qkFupo|clBjUNB8qko(*yDT1gB$(L(?#klSx^ zU0=kfC+BYlR|MOrl)BI16{0S2bv>jj2QUE*+lq3Mj-4%<;;(lB+0-rZ&PlC|Lg@M5Z1C6|pF^A5iW;V;S+WRrqy|0oM;hGKKBHg&_4K&}@8}x_uvo~|7ow2C|p6l-{Yk08FannOL#~!>j z#%tmL8UL}42$v!l|20kj8k*5VJ8aLa`ed#qx8AjDyY?YTb@NWy5q8k053B{#u^gGK zc@ZWmy;FEu#{69mN;IYh((Cg9yl6na5|j_G_0}cDtN;v)I6P{!E%iRAYw~EEYrBtA zZ*$JXyCK-p0@gEkOr_ym%+&dYjqYXC3+K=}-sHs7*T+-c6Yf{lg}$6&sWsKYUv+bv zZ$H}K-+yjGJKR1iW!0qNN4&R|5QzUSc+qei`i46w`yKU%ZJy*~j@);>(P1_c&Yr&I_Ht^JF(2i&CBo;c*i=>=iAM;bxO&p5;ZjQd#t#4})bj-Krx|@8lrob}{HT31F zW^StNdpODi+c<=a+~^<-7x6r%ctYkx*-Yh_i)*X7mPwzgUVHkRJ>)q}vs#rpn#7T+ z#;G74p$o9}>Fc8C(g7Wpe>hviL;4Fe-tSkq5P05;l7JLI!c)CX4*5VWv4AJmWDOn( zc9WUJ-I3V9LpOLz+XUvYyA<4z@{>%*kAzKBAM)ue{&J2N3!ZjcaKb8tcRYxao@BEr z&bSd^+mohxR6xNM?HpPL{J%4c^eRG)2p_MN&sn_Bdn(&;*)dTOa_V0Z}}kzx!3b zsix2DPTPS?Y2nK%t-Bj=(d?H&f6Kro|K}TOv*_Ia|WUY6OOhhd^TwtSuf2?KQf4+Bi zw$-YdPXFarlI?QUveU!!9()Z0(+h1HP+Tp3t%pzKE+#dc<2yP-qi*w#;yO!p1t?b_ zskw_AAVz${?raIDPv6>^1bT{poBF%aaDJrGBmlvSUm z0q(s#|H@;4xg(kw3@F?rD;Gci(szsV^Dl$^BT%N>ei3*U!iVWY{FpRMUssb!=+)%- z0(e^im;>&j;gDw6mv7a{&$ObKOuJI;dPDM;s#%ZGQA$?!N8PYx|#9snel z8GC2Fx_tCsihW^yL+Qwqjf&Cb#@bwZ1s;sK|LN(;>G1U6{N?HK@aW*|Z134YYj+Qe z^OL711#Cod2KEbTPhJA8lQsVNANi5u_r1Fi%|3xC1n%Tu>mbzzCr64~rj3AUCT&C@ zhEkd0XFx1e#EsgENc~~5w{f&V#@nyBdKuL1k zn;Y%LSyvbCMIa&+WzWi5r1z~+(ps^Q+s(~(O^u2N{lloOjndh=z_r#YWAfwz@s1dF zhKBzIYFS`lrjXgUv-u5Kq%hu9Adim<)G@((hV-a#ec49JR(K0=RbO9Mle$E!h+M{S z0>?%oqb2gMNEqMr222v6qeE!8N5u{O&Mnzl;;6__UzB;f*0tmi;nQ1-B+YG>BZ>a*Lm)$#E-cG*{Gnzyp|1(ew8G ztz%Uv$N}$ZD8}jit@Fx+3LxydK&tn3C9l{|1;HAxcMOmPk8CPqLFm|gvb>eUgLRMR zt7@l!XhM#e;&3|gvOq80hVH;{G7Ja2lHDTj1}uxbKyaFiX?cAyDcy@CT2)|<^*gve zT!gnK5hp}=5fid#PHsCmK0gH%+r#5?(f1FZwlIBjq;UgYNmH)_>BDwx!zdK7d{f;9 z7cYF6bKWrURLna8u?KBx!J!57I14nR;d9A>N zuE_jOue4aSVHg0rArENb3_+VCRIi3Z3ztmp+z^F8TFR#oWMKaJl*0|wOV@f~#^4VL z0t=^?y@72utnO-?aswMI?1D-B0yGK`knx8Hn#Uw!S|t7@5MOmyL!eG*5dJuXh@|09 z7N()C?V+0mZzDoT?4;LvV1X#Y7I3uNv=D+- zgbiIzwrNEnX1E5H>lxNnwGfSCh>-a!R14t8t|AYs-Spx8!I2o4(J{{3sGWWzo7TGP z$#yuzo;`$Nu*Bw5C}F)4a1x1;l)!RYUJU^$ct{_%8g zJqCYq9~R$9FxVJo3CC+Cu5b+n>Ngh1#q_Lhc{Gv-t^EjYKJ}X!KfYZL;;r;X1CF2i z_ZGD?S^3rLR^qg)G63XhrCxi&iY*kz*=I{((1|8w=XNqn3^XNF1zt3kep#k-Xr0}; zcG5Bw_Z^yR1S3%{6523?@Ny76^d-5$ft-YwBFUBW*=79(oGr^@q@3_ZEp#1iM_&+K zcLQ|%K1QhRYE=AwpvL?dg8DPrnenvF(VM|xH9m_|T8ZmS%S_rak`lv5D1@z;HuzrAywF}1(o8(@;#=^xIZr*Bb-me5K9=V52DK$ zU53lcm3pgykK@zgDY%-Oqv0p?u)aptl42*4JidIp?y#i+LkI9FD@1%je=>XtVWb#u zYt3k|;zRZ&$IS<3%cWNin(6`-Q1lI&`sTML0cK>hL$s5g$ko`z1#IDEt;M~1_*JabxgKM!a4Z* z-#{StyZ&`#;xH*x@NO?L{380gpnk>M5^`}Y*+ycqFL!%T{kxkpQf(BQCFe#(Q3TFE zJ0tf0UNZLYDxKQx+|lA6VR2RDeMdzK2~oB393^bg-L0;~4;vri38vmpF+oiOAW$F| z!P85i#$)_3#(W7J2h?ksgC0TvL`y9|9R^ZvFeT8e^n&_Fk5OH0aH1FPX2s;YNH@@m zlU6=yJjIMfod;?mP!(g?R~fq6hQz(>F@&w9g#f%m=t%msFgLN(;6?+T7{QM~!6M~E zuR1$w+00ITqz_uCKDJMetkgwT$v#JrUwc;nL#DJtQX%Suo>#lq5}4Ki_q8-?gHZ>~ z3fl&}v{Na%`Q2nR7zbw9V22zL)gnnXBerF^yQgZDv)f|irPv&?OH!y^ln$$a7l=TV zEy}^nuv!3B@Vi6(>Y@tSAgp#MLz99n>OZ>K1|y3MnPz3`OJwn|_*neF^&EE4C$neS zNDf<{2@orN3pjLhi!JOlo8qjZSRo|2x%J@UaO?WgYfQs9+XrmP@{2ykm*u!pxhW9e zejcAFR~9lc$ulOV%oJi_@ZT{t?%Y(+Mf$@8dK3dm82h6M^!Bna-`bxuMfAm|iw{#5 z-N%&O;nw!}G`NUumfb?*zdR8;s4<iZXK$-Ki`y^X|(~h*aIvj#@vE0=85TPjT(-<%ihSD*=TtwCUrx54f@@!o$ zCmR-^Ux;&>JA>hikV~r+7$*5!7X-B%=?ks17RLbdSG|y>ytrN&uDko_NxFvaSihZi zp;m9p#gqi!zDK+mDiba~dO-%mH}$(}h)L$OBUdoEHs{c2A`%H5ZYp*dsq&{}K|(_) zTtR4cfcr=@S}`IzS=w_5lUuQLcuKI0kDBt5%&)pF&;!I+J)b?ARES;bT2ldyAO$>f zz)H^8Pj+sb5m9L@*baNP`WY`^9Q@#1;fUC*iZ2FEyzuAu9bSHzQ4$}7#^#vpW6v%_aY zJxdU3SZMo;l4fXMLa#EHBh#M=w_vkk7M@uvWRQ;^xfZRPf!uXFlX*@1DXcN#A#bcSh zZsf|r6xg08VQhd;jUZ!_508Pa&3|eW-4i`!E%B6WXqP-~y6V2q_?z?XA9&!y2mG57 zFH#bfzhsBsVme10ga{Q(J!q;YtJFJ_>QU3V1G#H=8wFk;0-G19+RygR55C*`US$3a z4MZg2pRmY9)oq>*OpJ=ahlX=>aCBm3C5i>e82v93GD91uV&1Rx>W@S!sUJxf}!H3uq2g0h74{#oT87Qez>sj-=0~{LD8;Xq}@w%ZDFJr=LW>9*W6{PmlmMD&@;-D?45rUKA=qy*#y;i*DLu-L-=T7 zX0%&gCpQ&mZo=m&dW}Jwf{fJq&$vFUBMm(wn34Q1)6OU6^y8C*V4na4QhB2fAg!5e z2zP7go`y|h7hoFTI6Om}E=!Vrki%y+Wu0FJ1{w*E^XnDe`H}~R2?JJvV`XS(zD zU1_ry_Zry>iNgg=hIU5y@RFs3kQOl_Wd87Ow+OAGh*%&^q*e+{F5&H>DytYJ)Aj&g zGy4G?SYj=7+ny4eT`Vje?J%tjJ(3TR-F5N5eeY~9W~u#v>knSm;66(r8&YOEo8_tf z^5gjkp~gyLjYuwFi!C(t@#g>m5cPjF7!gn2$HEiy4TmU-)lDyq)B z?Uew*)Fy|PMMm3DeN2awW20{$KXpC2Kp0>C;COD3Rh<=^BDFKFCX^@! z^A)g-9swT7jmg^SUF_-wyv0|SmtYby4BxEP8z*)+n9xX2R$cBoRII-`2~O5mAey5>m@mgSH^Ol&+k#=l_8X^VZijWt4}frg!ri1X$GlTvxbgZRYqt zps}~S9D?WvgiTkek(Z!7ysLEAeb;R7)#KOZe}S2nh|IP_|8%*1h4BiG2Ir>-dq>`z zL1LVq`&L2&jNm#qw6Uo_?ID=!ZleWy0MJhiC7M89aV_=PBH_Zz^lArTsI4maZ)=Xf zfrw-VMgY|K#6zI3`y+NjakK1)AQ`)AHT(r{!9XcIHMPBL1e&331Zb2?n4JhOB2d0A z9784eAXJoiuSOPCWdx^Vlsx)rLmksFx82PixRe3JA_b_1AiWzTneKL$7a=B9?o}r) zHPqlpY2|5KP$#5-5U;7TPvJ}+29su<;?WsQ!68JcA>>e^;B(!)fh4(r2!(n{77Z#4 z*@8>26F-A=*k!K!^W$H3etz;x&tjwHHm-dk%=mI}Tb^Cslru_!mQ-^)6u$;!?i}SX z6<7_gfigN~hCj^A6n~iA9se!O_FBi_Rq@xvVqPY6o$Ejq=WI(ut7p`lAIYXy(xq$8 z(H;w54~spGj6?ZdIhFDP3B?(GP*|?YS)!4Ei!^g+2n!eU_0r7eUDXWItEF_&oEF#$ zN0wIIzC>iU2>&c|_7ZAJpOop8%~WAf1)NhEw3tshrgM17>Nz7+QKt6BgW#v}S#{WJ)c|L8;fr2KF=-cjYT)r`rLBY$J5}|R7T)-LhS8sRj(%l z?ZOg?1smzXZn>3z3A{1Gogw|k)06Lxhw|XBIdzCC*#bY7qP%wMlM?N3jT*QAFRqXw zhWgFbif(N1|3{ZST`~9a9^CQC2d1{S_w18p_EG8gcsi1FZPC+_!c_K7&>!c;=vRMg zNFHsUT=xCh?I2c}$T<$Z)^-w}(uc)&n8yX#7}~ZlzYAVMnEa)dBrOEW1kNYsGP{Mq zghnz7yT<6cld6C#ot)FSx?$&Ob&PcDZ7@cSuTJEQe zrESd8iG~`!O0ms6ns7`Ldp-^NG{NFyBtO#M$;G1qF{MjW@X>n{ZvuAc^uwzsJFf$x z_zL|u8iqaapDYEy5znHLPKa58`}B-*$pme2Z!F8Gcs2zL;V`LF^IO)7ZV&9)I^oHN zjh7|G)9k$Fz~>mo7;1N9L#G};3f0n9{z`Of8&6X1>B0WN;kO6QNnhouj6L)P-Ur{r zF(XR19#jL#KHBrW#hnM?h|-zN<~R02LuMp+HRv8sZs8zgeMwL_@K;HxsqPm{H)_6; z%%DXfc2EH_cXQZ`)keuTrGce#s|?}k!%z|g*4f1d{4Z={^gtT`|6>v7<+tlPOdGsx zz>5EeuuDqYaAnS#(?Js%huz5OPoXV#K9$Pbq)m?I4Tu$FC{DHE3iuUm2Ga?G*n!cV z_Uper(;n@fDHBKEJjA=6zm%k(bz|{l4@toO?8WWV>fLA6YK#b>oT^8)f_YqAdMm@S zkP%5hB70RqsmOHUO`;{<#-D3G|3iil`W(X`*&Tu>b1$e!brtw!0)ikPfB=jTv`74! zAok?X5TX|U6+Ky&Z}8NaFQbQa6dc`qxG9rKKB^nERALeJvSgHgMyD5iDl!nLBVto5kdQ7kC{X+RV&vLy?d zxn##483EyOwz`mPqbej^Mr<`QE*Zgc1~5L(3pYRn{Pu{!Or9BiUm1xdTKLGNuzN7DB@RB1<!^M2Do|aihhOE*8pSRU*(%KWOh00zVGx)iD}N}6*ObBP!`ZgE!p7l zb+ai}H}hh#UM}F_I~>!Ko@{i{g~Nlk+vXoVgb)L?wHqXh*?7Lhj3Bt%^=uM2VsdLm zA!0wl3&CW7qXfnN+AH>obE7?w1(VON zy2yZGWtzh!NC;E@0vb+W&MX5N_Cud+hv9KEUpBBwW<{H^xh$6TJVe0;Os6sEk~Jj< zoy^w=iEi|D%q-%elhaCv2vgoJyn=bXHp{0>)3JpD&A4rf#@>G5 zzG|O^P{N81T6oS4MYoFR($k_3TIfX}G9N1$EmqX~! z`s(rPFo=~N4H(AU<@|WT3k4r2&$pFdO(-Y-9K)#=>Mu3vDuM&_WL{hQg_5&X@ z2%zoSWOP^^&a$Tr8Wx&t3MW&h{gs)%v~-m?w|R*B-gpV z*Bm=QsG924rp8B9vkB|*3f#;u9FFNqvh0YuNKPVtOylUsch@5ieZCZ(yI)j;Hv?M zoow*ax;8^*bXrOIHK1G5$n;P(hhpLr*y+KS&ky#`4^NJ_kcXi#io;*;eY-b&zIXg= zxc_|b3{|Tq$q)MQ1MkN4d4B4jT2C-miN}H`TiV|VkDuM@Lza@H|GQb`2YbLr5ef>hE2imY>EOa zT=l+$<~Z9G+!No{)r~}TzB4~l>z{xAc`-N32Y(r`t6mSZUbH%ynoeDzZ*8rm*^T+N zRJCTM^{So@PVu`$L+V>e2ja53$sp@St~YUnW27+}y&Jf)X;Eum+)878HSkaIwfJ3G zqrZZ*jl(X^2}QS@OyK|M-BC`-kXSXJy`aT5d|@(o-J*eUar?I)amGSTkkkYalJIZz zv#p{2)>VTx^0guB0q+&*gHKpVfND1T$F80#mlGP}F;dnPB!#i1B(h-=K%1TaB}a>@ ztkpEMir6f*QZ)ljNPXb9xAQgDyvK9^8-kYg3B|a=6;m#4=6W||-xm4J!S}!x|1M-d zUn+Y4yyF>-c_ngYWB%GR90NBcvnfUXb1%w-41bri%nK7{mdROwZHIqo8}Pl1zvg(o zyB^k)5O?k4Sb`0yQS)u4M=oJ)W}F(tFp=Tm(+8Qd^utE;A@l54Y5qJgMEUw2y0y{b z@(m`95$WQwPt?cJ+PCnGi;ybL9pVuJbI1|WNJ}7*5J`AS+d!1Ik8C!0dHl^W;8Sd} z8_2NIrU8#|elzQV+KTY?maPP)kNt1b#|SjF+XS0;x&tNM0NC)d)am<~M>U>VbL%NS0on zV=vp6yCu-B%Xf(Q_Pg{h!JB3Gl4GpRxnN(6o`$TGlc(;GW^*ugikp-Dgn2ssiIybI zk_KrP(RZO(*mSXqFzE#8mQmPx{R@E(|I5<5x-akMClsVh3l|kUl?JLa> zi!*lh#WZSGT(u9z-T6$CFo@3Dv)R^yrtrbY;tvhN_bMx}V-Dqdc2zBFij4`{ai$e| z(>g3}u*$T4TRCJ=&N2(sX((-4r4oEzA|Cvk_0))E2!JATV1_U9!tbi}3$jq*<=OzU zty!KSsF3uSwp(d9;S@c6q( zo7$Ff2U5*QXi0JIe$t%(u%nK#*Ietv!cno{^zD(BZc7z3`)Lt1E%q^xr!_=kX<-gjA&S!FmMwUc zZ5+W7%p{lKb6i8l8(La4sBG6Sx4lv@3j=KN1vMB9B{v9dRU8tL-E}ZBlDRjSGj40! zae!gT_UrdbB!WI^_=v-r5sQp{)?ywA18CP}QW^|aJ>uYCimziU8ph$#N%Y4={+i}T zNU2OHRBl_mtqn|@BWoF(%+F0%-QEc4)^oU_R%}VtoT_D=b^u>(Sx%})^UF(1sBe+966$iSy*YM3K6U;}{sPY$_))%xIp|!<-=BYF z?&lY$C;$9Dkysh8sG%@GCQj~wQ`pi&oBXXZtYbL@l)}_FVgzR%e$YcyAT=8 zxYA=v?MT6x&BYNT(6ZAg0bBwRI4iT1_1ZZBf76?~QG3}yYPAl53d& z5@h6NMI69mUq*`a27zAyu;G%@N%BjBBqx!RCZyrtk1=JmXVus;cXYn(h|CfqojcW9 zcJL0K{w4O`QRmV97-o;o(;~=l*)PMTXd&r9Z#$mr{3|!rQW0D;7o>EU8PT4!oFqMV zG5BKd?4bD1zXh`34WYxEd8}tuGcFg^5G;a-=$Y9X1uCCPa@n?4xx<;jR>n-=AzR+W zKba>^Ta$%f0xw^V>l>mB&1nZhU@6Zj`xI;&7 z>9*^}mA^wL$qO;tU##Wm0VLZZAt5A;1m3iaI7f3Sm*_n4s?~_ndZ=VKj9RnR`ucV3 z8b=e6`J|R!zKpfg$Q;eSE0pcc|F#=4yzHd|np5t(z}6YI8 zuFtB~dJ$j+`Rzlpf?(z?vGvqmz*_CYaxmMFN<& z0G+>>&!^p9f2$^PhK6|O^I~CoS#^ZiV6o0+&dN4x@t2R!PeBnnJU$Q%ID#pyEkyk1SP}nllQ$aZmzu2#ehgJ{rrHGzW?6&tRkY)9r(kjuWt%bD zKKev4c8dA67iNPr|3gQ^$vaR*z=s80a%NLAiakt9flp89Z*)PK^&CKWNPo~899Qt4 zQFGshgXBesYyh8qJ~}*p@$!7Qe{%fo!Rfh5{j%vs@!Rr3{x(D@hA@B`j=JeRHMHL8 zJ4)f~C2Z+Q8r~R)PgD?_qBeFXc|pz&PR$zMxyN_lQz&opi3=k4biEt<0mZqa>Pe#5 z;nTZB%6Ku{$_S(@DwIWRQ(bBSGfro50|%|MTchJ%4>&IDyuPlE>glv@;+$!jL@0Fg zNv@z&;xjuyeB(^qjyWK+plpx?FWJ;an@&D!Ua0Kd#dM<7F?Kb#28_`3AtsMQ(q2UY z?L#IFbXi>c$%(>&`A5NVs~Cs(6xWi{Q%Nc!K3oX83&Ruh2D`n&dnPJ?p^XCuS3+W6 zcr(4Vb-Rq62{SPi*3t3t0f+_GmM~Bn)i0p zaSXM=0U}FgH8-7q?d=hb#MfurL{lyEWgcR;+}GW}4ZFpLbS1vuX@?3`xJUGn=90eWb9R)l?zyWmHsLJt`yx(2VCu_`-g`yy` z9vM1Gm+0&Up68_kD?m%_MP-WD^JOn)@vv|^#mR#G^C|d&7^ldsgW{>oYG$*Ad;7hf z)jt753}r$AsKrU$tm>JWJ~Ti+dE%}`ULu~K8hpC=ggK*mqklGD4Opvm?bVZBg75D| zK4V7MhHmIpwe*C3l!R=e+3n)G#p7wSe-y|)`= z7I`|gIXA`8I7#_3cu3nIZlb$~S6_Kds=@Yrj)0gk%nq;=?tQ=bbYdjD89e_rIi`Sz z1nVR1Lp;#PH8H}sD%Z*=Iz+B4r2^3De**9u7bC%Ui#Jj1nvY+Afg^pnaRAu^ZX#50 zfAy>D=UA_&(W@u@zrG%^Pop8P$$E^?J0q+J&Qa#;yb)od>u#LPt!(+`6q3PxIe*e@)l=T%h+B3`A!>mE^Us z!_J>6mB#kW)MzMMc{7Mj*b8coI?e z2}$ARZ7C+hY?7%kyQb$`^gEv3k~~2v;hN0lm64dk%?vZmZma%+$7(R=$Oh==|Hd|x zfzdi4T=XT*%cYY`Elr?WWga~;3FSQTJ(8^xq?wVw?@X%yAJJk?t4#->!GksCmqlX& zuX;lNy|Z!{>uCStG;V1ZbDQ3ua9yhKk3_V(8PBEjx1eR6-#mPeQ-kpplRqWJQ!eN~ z2f53xd>Ek{sXA*TZIv5BLgqQd;8o*C9=>MFVza*gNQ zQ$fg;b>v_S5>}4Xba$XBVJIm&phz5l@yzBT;eh|%it>3IdYrDC`dyehXq(j=gah4J z!y7f0hILL_-x7?iEss(7D#r{Yv@s`_((8X~w;b$ZkZLgR!{Yn(WX@X3BWH_qi6JAK zcOY&%eaEVEs1oJ$wY|kZH%H}Lx@yTF!Lu}#T?{KeTX0CFenYObOEp>my!7>qPQv-A zHDWNU;%=9lg46dDtjbpt@n&a3BxT|{cRlps4du;vz!fy7BV%Q=;zM4~-p$|2K*%`( zU#siu8v9w&H>>%)FuLU}&(z(=8nPkdVhJ>o@XV8(_ zvy>Gr1gilGK;%~1|97QPh8N`u%$@9&ZPv-~d9;2J*tWp72GJ>|!>9BOTt)~(yFiGN zi}`wG5o^GcU=~veuZrn@#|v(?Rw}5d(4j6Vf%f=HcjYaM@NR*~og}=DX`VX;ygNSP z->e7qWjzIGGRnJ&p$vJ8Rp(8$`r*?r=kt!Cjay}HIJz}|?88s$-!Do+{D@fi1;S#R z5*QgQ;{z(ZE?46#Gp_PY3DiLkKeJU0nYrsXv(ofe`R_mHbE-s+87oPlg0UoWG~jKs zL%$rGFS|c{+KE@m?LUgSL<6?brbfgeZJW2ZePljtSY*M~p+CqrF;M|7x9NoA2%t4g z2;j8@5bBVSZ{qcuYc64=#IMw3pt_@1a2=Q0hmy$eOik z2@VhE#eF5jY1n2sLImQRKGA^-QjEQvwqxt#txvW+)XWwB8xO0>v;%BnTCQi~D{GPo zq=}|a?#z$j5?Pu)u68?}KAn2|8!)2zV%6-vdhK7Ile!wtXGX-YZ4ik>^9&nvH?CKS z5D;C$C|As{CZ*xsv-8IyF#Ve?H=94-f%#a7wM&>seJ5 zJC|mJJ0pJ=Q?2_YOb5Vh!IQwXJ)=NOkdcYKq^G$W=?Vu1n< z3OQp?(cpwMi)u_~AF-uCt6p4H*YGAO<&l9$!Yk7x52A#3&p8VYfBPD~HQk;7Q*%N+ z`)HW&{EMY*=!QnI1yjJy=Cem|WlzfG#ORppY!7)hE=4fGafJ!l;O+1dZ}(L(YHpi> z5l%1b_ak!Iz+9uViXL`xm|oXjOY7N0M?_eOoev*P+x}|rxXdmjR*%l@j!l37Y`1#9 zT9!7hXa(UMgp-18b;AcF*FB3~;4NtP(=o^bfb zXW3;A5hx8<15@znkjG-)3?|jPnYl3sE*1-;G#6kY@Y=Wp>&B#N#>;wv5M-sfSS!5S zm&lpSZ-e4HSQmI&K}v*x8KU@A9XyfKX=)@pZmc~9%=)7*h-vfBwIK05g{1045(tf5 zU!w>LZeCWn@h?tkWr8;g79mHr>K%r#il=Wt!V#zp&@_k}qCAx|Qul&oX^Ji_cAvMY z^-7Lz@m_IVzqz7Zk`!RLsFu@P(rhPz1k2vXPr6x zV5uH{BQi=hC!yL#Hn=etA)P=;QtiiH!vP8}!~NBA`e^^Nv%_bH$LA0=`pJp}@8x>2 za_tw`V1`4eEsGs z+8!ky20J1?p3pr+v4YM<@;Kd@0aA4J0s%0xz{j0`Ka@ZmW_&!Hq15Db>K~1$}*U=Q#_g1Hte8NE--QQJQT!B_aNk zRvGM1TWSrtFZ}9SsZc+AkuaLFyviIbI&`t1X8mCsl+iGPbIH}|C!-tK?f+S7Hxt23 zaWFHggw$#B!KC%0f?U;rd$~ZI2M-=REcW-FKmTHH|C{3M#limJmxud;UJ`heh5#mD zZwX!uQJQEL<`%9QLD&2~t|+P;-Luh>?aZ$a?65Faos*JT&c!&f+UAWsb6KG{9@-4; zf-xA@lJehbw;}aZu^?l*B$OeM;f|ORBV{P2h#M*3djI*Pzb;rA?}GNt-mgf2N3^Zd4Zld2i^I zvMimVOA@XGE}AMh9QuJw8Dsb*jsR9{OUw#@Sph&JfQhbW-QGa+j`DJd?!mk4e=>lm zI73$GXralm@2DD*&YJ%PWJjb$t+?jspDuUPmp`AK?0+*nJ3l?xJJQPvKAwhJhTLa2 zZrPPiyR3MH2S%-%bhFb4u`U1hXtiFjyihOR{b#@UtN#}todlLhPMF`VSwr{Of+RQQt@Ot*QKfx^|Jkc8I$pwn7!wLTr+oD+~$MBw7gErc*N6^%W*s9p)AGbbf zrKmMG6@9?-vyszrzX=_YeP9yIiyIxGb=m%cL76i!Fqj{@ULFQ~7noknn^nd)FEf9+ zo$V3;_sueUHwTn`(9ntMin|v@J5gd_QFu2(;FN)m5zt5=mkObc2F0FkDpaF1R3ZCE zz~ID01#$iGmV{Wqk^TLC1`XY*ZjJlj(lwnND&+=`y$Ls7;I!?|<HV(8hNEBgp{M{*0Y7xn#U^lzkIFB~)#rESG{ zj5Pj6pNk@=?0HmJ00Tqfhm2YA2>gqUZXM7d!ktKE`YR`3N9h-U3Qa%ZC00#{;etO3 z{gd_#wIc%nbNqj@*hXO+kK_=Z3&0%yAZkQhbX#WwA4(6G+R6e`Dank}wP`(va!&fi zFfD}8cmw$(9Ba#baoKpb`TTsrXaK{A>!dfs*1JRR?Nd-1%+|xSE9UABZkFYO*KWhm z!wryiuP0Ry3*wD;wpcsC*sa)g`L=>bj&5gp(MeZTcR&~jZ@U4LTapp5T%IBwbZMMP&j_64R z>P@~j`Ovmh-9jJOidT6TAd272xNk~)aU^shy)yy*rq?6h8f51!8$+9crFIjiD^e!5 z9;}kW`3jsb7D6Mr8_$DIaOaPUN6cYmN?v;jGSp!fv0^`Sj;$$18cOvLoBtAUUU^U}j{lSjth6wxQ}MBIq6ZZ&7eh$`@7aZM~MfT}8u&I&Un zkBOTqQf$_*i2O z<6i2kaK#q$X0`RX4IgmTB}axgFAGInYQ(R$Rii-3fguL5B<&Blet<BPv;g^xG>!mPU+6`A3Nbr`_)299}rq zxy9~>S2SEWFIrY&ya$GJi7amUd<~&5bwdrkC+l@E9X`$=4 zKB#86ft~g0^3i{F66-`Uu;GNB?VWzH_w2yT<~byq+j?$ELP?e!vZNpYQap8~RrA(v zJ@f2Q@l(K^9-O~CJsuvJpUn)nS)iQdNQ@fShe_miFKs@baDXM5hA>FMgXr=RdqnFmS)m{K9r+YPo;js&<%d3{TR z;GacIoeg256t@vYVExR-C>7=d1tEB&Y1D%=rTmdJ;JQ3cv~_G&jSt^~^i7F;pTwk_ z;|v8p66VcWGcDh#LVXeIL$NjMrD6cy={!;A0%X!72AV%5TG-UjCJxg9#OhbtF#9sN zi|6z-oZ#$X>3r1V0Ih9QLL-nVy-7Z=2CJy!sHWgXg}C`y8^)L6Lt4GLxnC9FMr&AJ z^r5%9er8~o2I{!TrOaAA19r`1c%zZ4k$o$f7W2W(;xFlPzL0iE{3%c|X!Pb)9vb91 z>|f48-ji+u4FM!IS1jl~^|75wL3PX#sUhjl)AG`oj=xMh^xYax8FXr9eMl559_Toe zZ3=uQZl9NJ<}WK-uHp%GTk?9`dU@}?YWQVvj4-vQNDb22q%H)BT_gKAh=DqN1o#}E zW|Ns#oXKoTU<7=_NbrYzCQD^&&HUBmlLMIh6$02`=UrKFH6C%vnD#qf5;Q58PQ~r~ zBi)0;3@NudQgVvGnYGWJBQZwuDH+@>3@k#whIb{3TJvc3~c7?38yCcsI2Ul2g$Zm3D$ zT^NNkkX-~ePSTCnygeOft$fwf(byNp3h$MPqhbEWgB#VnD>P-XIVGE^D zMyryNLphux2f&zUHkIt>SGbQYuA~FLj}wsj2B9xOjHhwHFgMbR42p+~ljh!pIb@`C z#2#zD{Qc030c4%HXyeqnbsH|Hk;MazBlAUa>lj7ww3<&@I+xUTmV7ta#p0D3E{_(l ztf9_Aw^~u!Ce|n$S}FH|#lOfXUvE#M!5-Dao4LUXD>cxr54t|knGm)HMV3D=laO&$fVU=bM;0GQzg0MP$ZmN060C3lj1 zxCnB#I7=cWxU%T+<}izURJzM!-b=5@pF?07vORH$eV=mN(Uo-d(6BJZO`z00GL_V@ zCC)D;rFtD1U-?8{b<%O=BC}55tOuOD~VZBHolYp6m$;n8+V6*hDh3ce+j24^Oe5C@ZD|Sqj-3oSXQ}6 ztKN@B{IbFLR>~i#GiaH#vnEp+wF&L9()`4gu@}JsWm^lf(Uca!KL%)Bj7(HwZRI2O z6sda%s75Gc4N3$Y(j7hD6DC6uQ8{Yx@z4RaV{=CW3_3OY1cPT;bcm6e&5$*1u37^uENgU11{V3G_P zpn|;`FLdBDVmo{*Cyicuvn^;TBAjcMvQ0JrT{*Q(3tfte9#eY4Jg)a49u}$#P$JqY5c1+;L8^Dz?L0tt1@wF>_tirt>%AvJ6T*yR%;hp#`+B?K7 z{eDLSv=!t}>76H?R&Qx`x8~;~{n~D~@5Wf3>7*o0dC$esp2#hu`ZV*-te$b<`17z` zE!0T(Osv4|pb@DL-bgz##8#|*v!;B<0E%Z@p2n-d^^#8P)#o$#mdEg0>+p1_*|pov zyGuIt=65@bU(E8Z#B_pY|WiP_SG;Cm&s(b4*)_(d@mw(VioYH{-`L zUb~h$Ft7ZCNIU(!hWm{P8T2B%wL-yh{g+rO4{sv(I1+?mfqY>xx^lP+|FCP5_@4i<$TtTH~-Z?!r!n#V<9Pl3Va0lu> zN%Q}_J#NZPbIn-nJQWH|!3id&-SuQG!9_`HJln0HPPvzd(xUE+@^e~uTDBzK9W_&n zS`AFyd3I*~s`haKkl)}ReE0yF^~Z^KU3Qsk^hQslbn|vbSTc`5Ome^F|gVPO{ixSp49Ppj7g4e?0_TEh=2P zoRSwJcq4b!HjKC{Bc2~W2s(Dl+`~(F16tz-Mi`fSJWh9A@h;>@qyP)SvM z>XbLN4Kd9Q1U6Z#fKzf*LX1wSz;*M>5~eE!I^AFdX=8uA7$e07U@y*iosd1OaY&;P z%7v?Vj^03GnJI2EMv$cwXQ6moYV(<2pB1tl&$a9_NG0t9k&y}T+{jS?ph;an=y>hl zeppD$^s{k&lvdiUM~C2mAw`=DqG#lQB=Gma4%b|9+{e6(uY_A z$K5mwu1Wks-p%dQjR?-_^Xe11<7NpVWIbux0%5gykMVu1|H-jfVB#^k z9a9d&?KDzPP<>oQ^W~NOBIN-1=#_fJ5*+hExp0agCu&Jcqun{*Pphc3ILe06x$iAS z9iI{fgkir_+g#(p$KN#VthDZ`BiYw}SxTw2v}8vaoy<1cFIJ6u_mqR?5)T^D9?10y zi9?o!SE0$=o}m7OxO(LDD`zLOTK}CG94Dt2DS}w`>CYVT{^;&=04LvlEY#33leVSf zh!^nD2Qi;3rJm3-*Etl5Hz5etEKQ^;Q`3E` zl64Q1QrRaz$wNc}zW_hpKpnweR51fM(W@lv7QNE8$X*jPImuh+R-sGV0FlluDC(0` zEY&miqegGsNTgrckYU_14tde?VhJXj)_M(WxQkZoL-k~=>u~C3d@#gOpTbrm&Z18F zBPD`SIF>qAJm{!~YH{)bfz{+SgRHAZy{U(HM}JUq{x9g0GqCD}zV6wxBEk@+07!Q= z{>*pgLrJYQ9Z`SMT6CfAdhzh$9*q0z?DG>HM2XD0mOi0!Gq(DR2z^kJeocwPr|t5? zrYMi2N+T>TT3N?XAMQc-LypZspPm9rL~E5~AH2zEVNncPymSSl;^1S{AEQTP`y3Vn z7O`xBDV1IWNUE-A&3u*|OP|VxU&xs7nU9li;s!4GFl2})T(7t92U%k52vETEnti%f z0S+CHFTy-3Z0apQjZO@mE`aVBle^|%U+SmUd|G`sdaifgM<_+IW?Z=yYj+Emi3Ks$ zZ20^ksh5pR`SguUccIqkD*4x80V+3lO{1sC>zVQI%GauB-bi7H<3JWk19Z?dq+X30ofw*syT|tWx0Cu!M-4mKxX` zx|5LFd-{XEa=o=BKnoz01(6LzeT>BSq&!cZ?M8h_-dKC@sTe$|rfG25cGn5G{Us5^ zENBc_y(Q+BA*iyqE`yP8FyKxaiE2rzIFTzgd1Sk^bq`j&%HYc4r2ZSZ#Y?H9z zV1ug}1$V70G{#xwIs=pX$n!r7ZxHlJ#6@O98^<~)CEhTuJ)kb)rRLib^s5U|cDZTb zOIGC)a+6N?&O^ZxA#S)=#>e%xVP(4Dt`-Wi)nG6jf@M*|vC+sGnPAgCc3)O)($W}z z0U(yCDA>i9v+6*H<$R_ZSQUuqvr+mwhuQ+MBIS{lgl?LZ;@+So0Ogy+Wz{0~S10+0 zP0{%SYDN6ivp4j%Xe%^yUqU^4F-NFUTZr|Clu?USgI-gQbx%>;v!aqIfDu*|@(=+o zp^Fx3GctoRZ&>TGD#tpW+Y4*$(S0J^GLk(=oF-|H7pw(HzUzc>ph+mCl}|4o5%F~i z8PFJY=!p3QtY2DD?&*5Y_pG*Zv)?(z5r+_DG-G#U9tOiN;$Pi55qQi&Az~4wmg=UZlRSFXNsF07NYqwM& z2wh36MzO07sB2Z5sO#2R$LP-M1d}R5>j0(KruX|owqvp}5_Ky{8?Xdv*6eQoF zJfJ5{A3`I_t6Kx&;!(w82m*N$YJ%1!B7%I@78nv zc6htbaW%7z`@D^FjCdQJGQzSxrb7+hwF%$P?Rtf9>j6XZuyx#x=de}${#g6P$ERii zhwn7|%#_!n87KYqF@qJ11Pislm4=|9E_4`@F@TC6HGvWNhIgeM=+Xa9ADaBDV!OBC z!};Aj9@-*S8ISmEehCxefmH=XM&u$EW~HX7*9i643z2V*Cjf8xh2rV)aX99*T&u$U z!}*hXR(s|uF5GJ@TiqhBY0L?`)2fSujNE$k<82b%qA)=rqki?z8=nv`FU`|UMT&8w z3o;nn=juagp`LSCK-XXToo;+1jmF+ijBnD%CidF%#U9gtQlm1&pzk>d4t#~G z@^~EqAI0Z_iZKoy8+~xaWlz!KS+Zq{8$WjHj=8)XVS^*(_d%5K=ArsC)#q{(bS2_AXAjLb;mxD{w=CT9(j!}vl>Q8?F7sGUs1yy+V->8vo{phVYUtf<9~gsljmEXsBLr5;|Z2$%bzM<1y) z;-H&64rl0{+;URM#7{TZG%53+CqO@f27rzA3h7ym6?d?Rw z?gly|fZKY{x57&{EX8)1lWHkD3nI4zUs3+EL&dPr1j*r-BgWC;oh}~3u<4_CJcF)lyR;zCsy&0w&Lw_*~4>!{` zQz8Dv|BH|od9Rr{L5;W?%=s1!h6nM?p=2lqj8%=uIFnWj+|&qnFnHnAP&YLk>lx>P zmRge!=gx26MVy1-+xA8*f6+r!!k#>fyA?W z$ta`d6Zip7OxF3n!i>~!go4OuE|{y_QEhza{i~Ru_HDWbkLy@26F_@=6^t%o6;+=G zD8*q9ced-j4HS}$OK0=$nmRPw&y>2#VwmQ7rS=Y}>#i950W)ymyz=MvbL)?DOt~<9 z?p7-Ss?3%=DclrIcl*(UEdB#l>c$dV{aIEM!DpQPP)S@1+h>_ZCRm-lb_oXasX1Xw29_*O3K~ z0id*QS4pbA1B0G&LnLQ2jjOv}W7n>_3R!b)9f`4}hv4n4M7I12$1}DOGGyI`Ax|-3 zr!u>HB{WWI*>VEJisyp|Nl9QK@!F zm!;Tpvqf~gkxcSE$bwBV=a3!gz=km8;b$TwJjf&aKo~g zO@ce$pc|VsG7albKwJmVD@N@-uL}h9CtLI;Mpc!UWDYBu&SroQrqP!3m70W4nQ z8OZCjM4aeywiuTR$Q6`d82)QislfDETI|PbyE<5i2p0nYK>KS{$;sLh;>rFm|1AO_ zQ}cKE+n`dJrhLpI7kcZVCiLHv*Yk~! zSRA%F{~&G|bF$3O*>vvi38(9g9oP~ViW%c>!4|)(4LnHmA>eiW-M&wO)om*pMba-W zs`B$2h6muV1_|~Fxi4(?NvtWRIAmkHLM=d~9gZ;=t$VapFi^$*+9G0M$yYx31qEjMC)7y$dXVcbav7gJzru4JPT>QnLb$&dyEE%%$^A} zNz?rby^2uIain`xj5V_jgbzy>I)jrE|w?s8P`Jz8m=b!uVke(hY#klzJ zK!35R&{=>oIx^(|MUHRW`WDpsVT%C!#&nmmj+e%ZbVDX1@d$Q97)n(2@?Ir2#)y2xrGRPsW&5iv9E|+iG`;v;~6;h$PDo&j_&CD3L$_)%6yvlk= zDCGXZk%w^JERo1ttQrIKMoMWzq4MHKU;OyCaAVVp-<4QTc>h*sfjB6yKtNOOoE=wz zlcs-Hs4ir*i=sH8uvhM+5Xgw+N+%Ly>qJ(+ED6sNPXYpy8f*g_5k?38&d;UM;!ii9+C9W$Ulym%bg^^ap6z1Tn>e zrl9JI9tRTsZRN+ z+yvQoCY@ek^yIix*J^@y`Vz*O54VV2?S3_f&lr)V3-G<(*!%JRB9|ScC-Q}k`tt;< z#K9v2=Q~#SHF$;MYmg~2e(I%kunIecejW1?3Wu#2Okid&gFRr?h3$h#LZ42tWK;jz zelM|QyAM<5%qI2H$#aZ@@fsP;2L6{`U*?0IW=6FSmFKM+-cc(e{WVRg_WTUR`+Ut7 zY||w2E1K_FiV;k;^nRWMI78ody^|bLijX6e@=?N5OjdPR%urcGve2Jl=YC=k(cE=b zP?{2{tSEj1CP3()F&5Cm3E zhyM(~*BFnr`H;vpmo7*jrP z-U+kZ3Hykv;s|uNScYfd@75m?k`Vv^mfxt$m&|&A2q62v%YU{0uI3+|Ao}yaQVQ{Y zBf3k5^M6k%7#jZbbB2bb0;Dvo%xtX8Jj`q~h)QxmC=%%3SqB6FZB#A9=K?(^4|(?i zQqUo?_1`Z@UIOqf;P;+`2Zs%dp`o?CwX>n&KYkJx09{%x^!J<1pHx{XI{AmN&?r*~ zOu!ld@dExyP5FyzYHMw7@9bp$5mM2=$RM$#zaPk-V00&6f^)Zi4Fi@{2S8^^iQURHRP6S?fI{&{vSd3 zPmBGZK}aI~C-xtz#J@Z3AtC-hD*d18#XsqS%zx7V6fr``zeS9H>nZ#h;{Qx#{YjP% z{wMk05?c`Jf0NYullq@snLnvnkaOZ6f83uP8wm8j>D2sx=>K$u{0Ys-{3rC!>jeV+ beZBn0@x#LXKG|PEHY~_#@C^AX0Pz0+FY9IN diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 00ec094e7e3b4..30ad04297c682 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -109,14 +109,10 @@ def _deserialize_accumulator(aid, zero_value, accum_param): from pyspark.accumulators import _accumulatorRegistry - # If this certain accumulator was deserialized, don't overwrite it. - if aid in _accumulatorRegistry: - return _accumulatorRegistry[aid] - else: - accum = Accumulator(aid, zero_value, accum_param) - accum._deserialized = True - _accumulatorRegistry[aid] = accum - return accum + accum = Accumulator(aid, zero_value, accum_param) + accum._deserialized = True + _accumulatorRegistry[aid] = accum + return accum class Accumulator(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index e963affd4af36..c111f42e9f0c8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -834,11 +834,9 @@ def union(self, rdds): first_jrdd_deserializer = rdds[0]._jrdd_deserializer if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds): rdds = [x._reserialize() for x in rdds] - cls = SparkContext._jvm.org.apache.spark.api.java.JavaRDD - jrdds = SparkContext._gateway.new_array(cls, len(rdds)) - for i in range(0, len(rdds)): - jrdds[i] = rdds[i]._jrdd - return RDD(self._jsc.union(jrdds), self, rdds[0]._jrdd_deserializer) + first = rdds[0]._jrdd + rest = [x._jrdd for x in rdds[1:]] + return RDD(self._jsc.union(first, rest), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index beb1a065d2803..3128d5792eead 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -25,12 +25,9 @@ if sys.version < "3": from itertools import imap as map -if sys.version >= '3': - basestring = str - from pyspark import since, SparkContext from pyspark.rdd import ignore_unicode_prefix, PythonEvalType -from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal +from pyspark.sql.column import Column, _to_java_column, _to_seq from pyspark.sql.dataframe import DataFrame from pyspark.sql.types import StringType, DataType # Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409 @@ -858,6 +855,36 @@ def ntile(n): return Column(sc._jvm.functions.ntile(int(n))) +@since(2.4) +def unboundedPreceding(): + """ + Window function: returns the special frame boundary that represents the first row + in the window partition. + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.unboundedPreceding()) + + +@since(2.4) +def unboundedFollowing(): + """ + Window function: returns the special frame boundary that represents the last row + in the window partition. + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.unboundedFollowing()) + + +@since(2.4) +def currentRow(): + """ + Window function: returns the special frame boundary that represents the current row + in the window partition. + """ + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.currentRow()) + + # ---------------------- Date/Timestamp functions ------------------------------ @since(1.5) @@ -1664,32 +1691,18 @@ def repeat(col, n): @since(1.5) @ignore_unicode_prefix -def split(str, pattern, limit=-1): +def split(str, pattern): """ - Splits str around matches of the given pattern. + Splits str around pattern (pattern is a regular expression). - :param str: a string expression to split - :param pattern: a string representing a regular expression. The regex string should be - a Java regular expression. - :param limit: an integer which controls the number of times `pattern` is applied. - - * ``limit > 0``: The resulting array's length will not be more than `limit`, and the - resulting array's last entry will contain all input beyond the last - matched pattern. - * ``limit <= 0``: `pattern` will be applied as many times as possible, and the resulting - array can be of any size. - - .. versionchanged:: 3.0 - `split` now takes an optional `limit` field. If not provided, default limit value is -1. + .. note:: pattern is a string represent the regular expression. - >>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',]) - >>> df.select(split(df.s, '[ABC]', 2).alias('s')).collect() - [Row(s=[u'one', u'twoBthreeC'])] - >>> df.select(split(df.s, '[ABC]', -1).alias('s')).collect() - [Row(s=[u'one', u'two', u'three', u''])] + >>> df = spark.createDataFrame([('ab12cd',)], ['s',]) + >>> df.select(split(df.s, '[0-9]+').alias('s')).collect() + [Row(s=[u'ab', u'cd'])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.split(_to_java_column(str), pattern, limit)) + return Column(sc._jvm.functions.split(_to_java_column(str), pattern)) @ignore_unicode_prefix @@ -2275,7 +2288,7 @@ def from_json(col, schema, options={}): [Row(json=[Row(a=1)])] >>> schema = schema_of_json(lit('''{"a": 0}''')) >>> df.select(from_json(df.value, schema).alias("json")).collect() - [Row(json=Row(a=None))] + [Row(json=Row(a=1))] >>> data = [(1, '''[1, 2, 3]''')] >>> schema = ArrayType(IntegerType()) >>> df = spark.createDataFrame(data, ("key", "value")) @@ -2335,59 +2348,30 @@ def to_json(col, options={}): @ignore_unicode_prefix @since(2.4) -def schema_of_json(json, options={}): +def schema_of_json(col, options={}): """ - Parses a JSON string and infers its schema in DDL format. + Parses a column containing a JSON string and infers its schema in DDL format. - :param json: a JSON string or a string literal containing a JSON string. + :param col: string column in json format :param options: options to control parsing. accepts the same options as the JSON datasource .. versionchanged:: 3.0 It accepts `options` parameter to control schema inferring. - >>> df = spark.range(1) + >>> from pyspark.sql.types import * + >>> data = [(1, '{"a": 1}')] + >>> df = spark.createDataFrame(data, ("key", "value")) + >>> df.select(schema_of_json(df.value).alias("json")).collect() + [Row(json=u'struct')] >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect() [Row(json=u'struct')] - >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'}) + >>> schema = schema_of_json(lit('{a: 1}'), {'allowUnquotedFieldNames':'true'}) >>> df.select(schema.alias("json")).collect() [Row(json=u'struct')] """ - if isinstance(json, basestring): - col = _create_column_from_literal(json) - elif isinstance(json, Column): - col = _to_java_column(json) - else: - raise TypeError("schema argument should be a column or string") - - sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_json(col, options) - return Column(jc) - - -@ignore_unicode_prefix -@since(3.0) -def schema_of_csv(csv, options={}): - """ - Parses a CSV string and infers its schema in DDL format. - - :param col: a CSV string or a string literal containing a CSV string. - :param options: options to control parsing. accepts the same options as the CSV datasource - - >>> df = spark.range(1) - >>> df.select(schema_of_csv(lit('1|a'), {'sep':'|'}).alias("csv")).collect() - [Row(csv=u'struct<_c0:int,_c1:string>')] - >>> df.select(schema_of_csv('1|a', {'sep':'|'}).alias("csv")).collect() - [Row(csv=u'struct<_c0:int,_c1:string>')] - """ - if isinstance(csv, basestring): - col = _create_column_from_literal(csv) - elif isinstance(csv, Column): - col = _to_java_column(csv) - else: - raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_csv(col, options) + jc = sc._jvm.functions.schema_of_json(_to_java_column(col), options) return Column(jc) @@ -2680,38 +2664,6 @@ def sequence(start, stop, step=None): _to_java_column(start), _to_java_column(stop), _to_java_column(step))) -@ignore_unicode_prefix -@since(3.0) -def from_csv(col, schema, options={}): - """ - Parses a column containing a CSV string to a row with the specified schema. - Returns `null`, in the case of an unparseable string. - - :param col: string column in CSV format - :param schema: a string with schema in DDL format to use when parsing the CSV column. - :param options: options to control parsing. accepts the same options as the CSV datasource - - >>> data = [("1,2,3",)] - >>> df = spark.createDataFrame(data, ("value",)) - >>> df.select(from_csv(df.value, "a INT, b INT, c INT").alias("csv")).collect() - [Row(csv=Row(a=1, b=2, c=3))] - >>> value = data[0][0] - >>> df.select(from_csv(df.value, schema_of_csv(value)).alias("csv")).collect() - [Row(csv=Row(_c0=1, _c1=2, _c2=3))] - """ - - sc = SparkContext._active_spark_context - if isinstance(schema, basestring): - schema = _create_column_from_literal(schema) - elif isinstance(schema, Column): - schema = _to_java_column(schema) - else: - raise TypeError("schema argument should be a column or string") - - jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, options) - return Column(jc) - - # ---------------------------- User Defined Function ---------------------------------- class PandasUDFType(object): @@ -2767,39 +2719,6 @@ def udf(f=None, returnType=StringType()): | 8| JOHN DOE| 22| +----------+--------------+------------+ """ - - # The following table shows most of Python data and SQL type conversions in normal UDFs that - # are not yet visible to the user. Some of behaviors are buggy and might be changed in the near - # future. The table might have to be eventually documented externally. - # Please see SPARK-25666's PR to see the codes in order to generate the table below. - # - # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa - # |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)|1(long)| a(str)| a(unicode)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)| ABC(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa - # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa - # | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa - # | tinyint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa - # | smallint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa - # | int| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa - # | bigint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa - # | string| None| u'true'| u'1'| u'1'| u'a'| u'a'|u'java.util.Grego...| u'java.util.Grego...| u'1.0'| u'[I@24a83055'| u'[1]'|u'[Ljava.lang.Obj...| u'[B@49093632'| u'1'| u'{a=1}'| X| X| # noqa - # | date| None| X| X| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa - # | timestamp| None| X| X| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa - # | float| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa - # | double| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa - # | array| None| None| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa - # | binary| None| None| None| None|bytearray(b'a')|bytearray(b'a')| None| None| None| None| None| None|bytearray(b'ABC')| None| None| X| X| # noqa - # | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa - # | map| None| None| None| None| None| None| None| None| None| None| None| None| None| None| {u'a': 1}| X| X| # noqa - # | struct<_1:int>| None| X| X| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa - # +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa - # - # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be - # used in `returnType`. - # Note: The values inside of the table are generated by `repr`. - # Note: Python 2 is used to generate this table since it is used to check the backward - # compatibility often in practice. - # Note: 'X' means it throws an exception during the conversion. - # decorator @udf, @udf(), @udf(dataType()) if f is None or isinstance(f, (str, DataType)): # If DataType has been passed as a positional argument @@ -3015,49 +2934,7 @@ def pandas_udf(f=None, returnType=None, functionType=None): can fail on special rows, the workaround is to incorporate the condition into the functions. .. note:: The user-defined functions do not take keyword arguments on the calling side. - - .. note:: The data type of returned `pandas.Series` from the user-defined functions should be - matched with defined returnType (see :meth:`types.to_arrow_type` and - :meth:`types.from_arrow_type`). When there is mismatch between them, Spark might do - conversion on returned data. The conversion is not guaranteed to be correct and results - should be checked for accuracy by users. - """ - - # The following table shows most of Pandas data and SQL type conversions in Pandas UDFs that - # are not yet visible to the user. Some of behaviors are buggy and might be changed in the near - # future. The table might have to be eventually documented externally. - # Please see SPARK-25798's PR to see the codes in order to generate the table below. - # - # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa - # |SQL Type \ Pandas Value(Type)|None(object(NoneType))|True(bool)|1(int8)|1(int16)| 1(int32)| 1(int64)|1(uint8)|1(uint16)|1(uint32)|1(uint64)|1.0(float16)|1.0(float32)|1.0(float64)|1970-01-01 00:00:00(datetime64[ns])|1970-01-01 00:00:00-05:00(datetime64[ns, US/Eastern])|a(object(string))| 1(object(Decimal))|[1 2 3](object(array[int32]))|1.0(float128)|(1+0j)(complex64)|(1+0j)(complex128)|A(category)|1 days 00:00:00(timedelta64[ns])| # noqa - # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa - # | boolean| None| True| True| True| True| True| True| True| True| True| False| False| False| False| False| X| X| X| False| False| False| X| False| # noqa - # | tinyint| None| 1| 1| 1| 1| 1| X| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| 0| X| # noqa - # | smallint| None| 1| 1| 1| 1| 1| 1| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa - # | int| None| 1| 1| 1| 1| 1| 1| 1| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa - # | bigint| None| 1| 1| 1| 1| 1| 1| 1| 1| X| 1| 1| 1| 0| 18000000000000| X| X| X| X| X| X| X| X| # noqa - # | float| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X|1.401298464324817...| X| X| X| X| X| X| # noqa - # | double| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X| X| X| X| X| X| X| X| # noqa - # | date| None| X| X| X|datetime.date(197...| X| X| X| X| X| X| X| X| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa - # | timestamp| None| X| X| X| X|datetime.datetime...| X| X| X| X| X| X| X| datetime.datetime...| datetime.datetime...| X| X| X| X| X| X| X| X| # noqa - # | string| None| u''|u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u''| u''| u''| X| X| u'a'| X| X| u''| u''| u''| X| X| # noqa - # | decimal(10,0)| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| Decimal('1')| X| X| X| X| X| X| # noqa - # | array| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| [1, 2, 3]| X| X| X| X| X| # noqa - # | map| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa - # | struct<_1:int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa - # | binary| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa - # +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa - # - # Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be - # used in `returnType`. - # Note: The values inside of the table are generated by `repr`. - # Note: Python 2 is used to generate this table since it is used to check the backward - # compatibility often in practice. - # Note: Pandas 0.19.2 and PyArrow 0.9.0 are used. - # Note: Timezone is Singapore timezone. - # Note: 'X' means it throws an exception during the conversion. - # Note: 'binary' type is only supported with PyArrow 0.10.0+ (SPARK-23555). - + """ # decorator @pandas_udf(returnType, functionType) is_decorator = f is None or isinstance(f, (str, DataType)) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 6f4b32757314d..079af8c05705d 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -192,7 +192,6 @@ def getOrCreate(self): """A class attribute having a :class:`Builder` to construct :class:`SparkSession` instances""" _instantiatedSession = None - _activeSession = None @ignore_unicode_prefix def __init__(self, sparkContext, jsparkSession=None): @@ -234,9 +233,7 @@ def __init__(self, sparkContext, jsparkSession=None): if SparkSession._instantiatedSession is None \ or SparkSession._instantiatedSession._sc._jsc is None: SparkSession._instantiatedSession = self - SparkSession._activeSession = self self._jvm.SparkSession.setDefaultSession(self._jsparkSession) - self._jvm.SparkSession.setActiveSession(self._jsparkSession) def _repr_html_(self): return """ @@ -258,29 +255,6 @@ def newSession(self): """ return self.__class__(self._sc, self._jsparkSession.newSession()) - @classmethod - @since(3.0) - def getActiveSession(cls): - """ - Returns the active SparkSession for the current thread, returned by the builder. - >>> s = SparkSession.getActiveSession() - >>> l = [('Alice', 1)] - >>> rdd = s.sparkContext.parallelize(l) - >>> df = s.createDataFrame(rdd, ['name', 'age']) - >>> df.select("age").collect() - [Row(age=1)] - """ - from pyspark import SparkContext - sc = SparkContext._active_spark_context - if sc is None: - return None - else: - if sc._jvm.SparkSession.getActiveSession().isDefined(): - SparkSession(sc, sc._jvm.SparkSession.getActiveSession().get()) - return SparkSession._activeSession - else: - return None - @property @since(2.0) def sparkContext(self): @@ -697,8 +671,6 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr ... Py4JJavaError: ... """ - SparkSession._activeSession = self - self._jvm.SparkSession.setActiveSession(self._jsparkSession) if isinstance(data, DataFrame): raise TypeError("data is already a DataFrame") @@ -854,9 +826,7 @@ def stop(self): self._sc.stop() # We should clean the default session up. See SPARK-23228. self._jvm.SparkSession.clearDefaultSession() - self._jvm.SparkSession.clearActiveSession() SparkSession._instantiatedSession = None - SparkSession._activeSession = None @since(2.0) def __enter__(self): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 567edcda5a10b..0ed5e24ef6b8f 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -221,63 +221,6 @@ def sql_conf(self, pairs): else: self.spark.conf.set(key, old_value) - @contextmanager - def database(self, *databases): - """ - A convenient context manager to test with some specific databases. This drops the given - databases if exist and sets current database to "default" when it exits. - """ - assert hasattr(self, "spark"), "it should have 'spark' attribute, having a spark session." - - try: - yield - finally: - for db in databases: - self.spark.sql("DROP DATABASE IF EXISTS %s CASCADE" % db) - self.spark.catalog.setCurrentDatabase("default") - - @contextmanager - def table(self, *tables): - """ - A convenient context manager to test with some specific tables. This drops the given tables - if exist when it exits. - """ - assert hasattr(self, "spark"), "it should have 'spark' attribute, having a spark session." - - try: - yield - finally: - for t in tables: - self.spark.sql("DROP TABLE IF EXISTS %s" % t) - - @contextmanager - def tempView(self, *views): - """ - A convenient context manager to test with some specific views. This drops the given views - if exist when it exits. - """ - assert hasattr(self, "spark"), "it should have 'spark' attribute, having a spark session." - - try: - yield - finally: - for v in views: - self.spark.catalog.dropTempView(v) - - @contextmanager - def function(self, *functions): - """ - A convenient context manager to test with some specific functions. This drops the given - functions if exist when it exits. - """ - assert hasattr(self, "spark"), "it should have 'spark' attribute, having a spark session." - - try: - yield - finally: - for f in functions: - self.spark.sql("DROP FUNCTION IF EXISTS %s" % f) - class ReusedSQLTestCase(ReusedPySparkTestCase, SQLTestUtils): @classmethod @@ -385,7 +328,6 @@ class SQLTests(ReusedSQLTestCase): @classmethod def setUpClass(cls): ReusedSQLTestCase.setUpClass() - cls.spark.catalog._reset() cls.tempdir = tempfile.NamedTemporaryFile(delete=False) os.unlink(cls.tempdir.name) cls.testData = [Row(key=i, value=str(i)) for i in range(100)] @@ -401,6 +343,12 @@ def test_sqlcontext_reuses_sparksession(self): sqlContext2 = SQLContext(self.sc) self.assertTrue(sqlContext1.sparkSession is sqlContext2.sparkSession) + def tearDown(self): + super(SQLTests, self).tearDown() + + # tear down test_bucketed_write state + self.spark.sql("DROP TABLE IF EXISTS pyspark_bucket") + def test_row_should_be_read_only(self): row = Row(a=1, b=2) self.assertEqual(1, row.a) @@ -521,12 +469,11 @@ def test_udf(self): self.assertEqual(row[0], 4) def test_udf2(self): - with self.tempView("test"): - self.spark.catalog.registerFunction("strlen", lambda string: len(string), IntegerType()) - self.spark.createDataFrame(self.sc.parallelize([Row(a="test")]))\ - .createOrReplaceTempView("test") - [res] = self.spark.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() - self.assertEqual(4, res[0]) + self.spark.catalog.registerFunction("strlen", lambda string: len(string), IntegerType()) + self.spark.createDataFrame(self.sc.parallelize([Row(a="test")]))\ + .createOrReplaceTempView("test") + [res] = self.spark.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() + self.assertEqual(4, res[0]) def test_udf3(self): two_args = self.spark.catalog.registerFunction( @@ -715,16 +662,14 @@ def test_udf_without_arguments(self): self.assertEqual(row[0], "bar") def test_udf_with_array_type(self): - with self.tempView("test"): - d = [Row(l=list(range(3)), d={"key": list(range(5))})] - rdd = self.sc.parallelize(d) - self.spark.createDataFrame(rdd).createOrReplaceTempView("test") - self.spark.catalog.registerFunction( - "copylist", lambda l: list(l), ArrayType(IntegerType())) - self.spark.catalog.registerFunction("maplen", lambda d: len(d), IntegerType()) - [(l1, l2)] = self.spark.sql("select copylist(l), maplen(d) from test").collect() - self.assertEqual(list(range(3)), l1) - self.assertEqual(1, l2) + d = [Row(l=list(range(3)), d={"key": list(range(5))})] + rdd = self.sc.parallelize(d) + self.spark.createDataFrame(rdd).createOrReplaceTempView("test") + self.spark.catalog.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) + self.spark.catalog.registerFunction("maplen", lambda d: len(d), IntegerType()) + [(l1, l2)] = self.spark.sql("select copylist(l), maplen(d) from test").collect() + self.assertEqual(list(range(3)), l1) + self.assertEqual(1, l2) def test_broadcast_in_udf(self): bar = {"a": "aa", "b": "bb", "c": "abc"} @@ -1112,11 +1057,10 @@ def test_basic_functions(self): self.assertTrue(df.is_cached) self.assertEqual(2, df.count()) - with self.tempView("temp"): - df.createOrReplaceTempView("temp") - df = self.spark.sql("select foo from temp") - df.count() - df.collect() + df.createOrReplaceTempView("temp") + df = self.spark.sql("select foo from temp") + df.count() + df.collect() def test_apply_schema_to_row(self): df = self.spark.read.json(self.sc.parallelize(["""{"a":2}"""])) @@ -1189,90 +1133,17 @@ def test_infer_schema(self): df = self.spark.createDataFrame(rdd) self.assertEqual([], df.rdd.map(lambda r: r.l).first()) self.assertEqual([None, ""], df.rdd.map(lambda r: r.s).collect()) - - with self.tempView("test"): - df.createOrReplaceTempView("test") - result = self.spark.sql("SELECT l[0].a from test where d['key'].d = '2'") - self.assertEqual(1, result.head()[0]) + df.createOrReplaceTempView("test") + result = self.spark.sql("SELECT l[0].a from test where d['key'].d = '2'") + self.assertEqual(1, result.head()[0]) df2 = self.spark.createDataFrame(rdd, samplingRatio=1.0) self.assertEqual(df.schema, df2.schema) self.assertEqual({}, df2.rdd.map(lambda r: r.d).first()) self.assertEqual([None, ""], df2.rdd.map(lambda r: r.s).collect()) - - with self.tempView("test2"): - df2.createOrReplaceTempView("test2") - result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'") - self.assertEqual(1, result.head()[0]) - - def test_infer_schema_specification(self): - from decimal import Decimal - - class A(object): - def __init__(self): - self.a = 1 - - data = [ - True, - 1, - "a", - u"a", - datetime.date(1970, 1, 1), - datetime.datetime(1970, 1, 1, 0, 0), - 1.0, - array.array("d", [1]), - [1], - (1, ), - {"a": 1}, - bytearray(1), - Decimal(1), - Row(a=1), - Row("a")(1), - A(), - ] - - df = self.spark.createDataFrame([data]) - actual = list(map(lambda x: x.dataType.simpleString(), df.schema)) - expected = [ - 'boolean', - 'bigint', - 'string', - 'string', - 'date', - 'timestamp', - 'double', - 'array', - 'array', - 'struct<_1:bigint>', - 'map', - 'binary', - 'decimal(38,18)', - 'struct', - 'struct', - 'struct', - ] - self.assertEqual(actual, expected) - - actual = list(df.first()) - expected = [ - True, - 1, - 'a', - u"a", - datetime.date(1970, 1, 1), - datetime.datetime(1970, 1, 1, 0, 0), - 1.0, - [1.0], - [1], - Row(_1=1), - {"a": 1}, - bytearray(b'\x00'), - Decimal('1.000000000000000000'), - Row(a=1), - Row(a=1), - Row(a=1), - ] - self.assertEqual(actual, expected) + df2.createOrReplaceTempView("test2") + result = self.spark.sql("SELECT l[0].a from test2 where d['key'].d = '2'") + self.assertEqual(1, result.head()[0]) def test_infer_schema_not_enough_names(self): df = self.spark.createDataFrame([["a", "b"]], ["col1"]) @@ -1342,13 +1213,12 @@ def test_apply_schema(self): datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) self.assertEqual(r, results.first()) - with self.tempView("table2"): - df.createOrReplaceTempView("table2") - r = self.spark.sql("SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + - "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + - "float1 + 1.5 as float1 FROM table2").first() + df.createOrReplaceTempView("table2") + r = self.spark.sql("SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + + "short1 + 1 AS short1, short2 - 1 AS short2, int1 - 1 AS int1, " + + "float1 + 1.5 as float1 FROM table2").first() - self.assertEqual((126, -127, -32767, 32766, 2147483646, 2.5), tuple(r)) + self.assertEqual((126, -127, -32767, 32766, 2147483646, 2.5), tuple(r)) def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] @@ -1361,12 +1231,10 @@ def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) df = self.sc.parallelize([row]).toDF() - - with self.tempView("test"): - df.createOrReplaceTempView("test") - row = self.spark.sql("select l, d from test").head() - self.assertEqual(1, row.asDict()["l"][0].a) - self.assertEqual(1.0, row.asDict()['d']['key'].c) + df.createOrReplaceTempView("test") + row = self.spark.sql("select l, d from test").head() + self.assertEqual(1, row.asDict()["l"][0].a) + self.assertEqual(1.0, row.asDict()['d']['key'].c) def test_udt(self): from pyspark.sql.types import _parse_datatype_json_string, _infer_type, _make_type_verifier @@ -1460,22 +1328,18 @@ def test_infer_schema_with_udt(self): schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), ExamplePointUDT) - - with self.tempView("labeled_point"): - df.createOrReplaceTempView("labeled_point") - point = self.spark.sql("SELECT point FROM labeled_point").head().point - self.assertEqual(point, ExamplePoint(1.0, 2.0)) + df.createOrReplaceTempView("labeled_point") + point = self.spark.sql("SELECT point FROM labeled_point").head().point + self.assertEqual(point, ExamplePoint(1.0, 2.0)) row = Row(label=1.0, point=PythonOnlyPoint(1.0, 2.0)) df = self.spark.createDataFrame([row]) schema = df.schema field = [f for f in schema.fields if f.name == "point"][0] self.assertEqual(type(field.dataType), PythonOnlyUDT) - - with self.tempView("labeled_point"): - df.createOrReplaceTempView("labeled_point") - point = self.spark.sql("SELECT point FROM labeled_point").head().point - self.assertEqual(point, PythonOnlyPoint(1.0, 2.0)) + df.createOrReplaceTempView("labeled_point") + point = self.spark.sql("SELECT point FROM labeled_point").head().point + self.assertEqual(point, PythonOnlyPoint(1.0, 2.0)) def test_apply_schema_with_udt(self): from pyspark.sql.tests import ExamplePoint, ExamplePointUDT @@ -3116,199 +2980,187 @@ def test_conf(self): def test_current_database(self): spark = self.spark - with self.database("some_db"): - self.assertEquals(spark.catalog.currentDatabase(), "default") - spark.sql("CREATE DATABASE some_db") - spark.catalog.setCurrentDatabase("some_db") - self.assertEquals(spark.catalog.currentDatabase(), "some_db") - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.setCurrentDatabase("does_not_exist")) + spark.catalog._reset() + self.assertEquals(spark.catalog.currentDatabase(), "default") + spark.sql("CREATE DATABASE some_db") + spark.catalog.setCurrentDatabase("some_db") + self.assertEquals(spark.catalog.currentDatabase(), "some_db") + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.setCurrentDatabase("does_not_exist")) def test_list_databases(self): spark = self.spark - with self.database("some_db"): - databases = [db.name for db in spark.catalog.listDatabases()] - self.assertEquals(databases, ["default"]) - spark.sql("CREATE DATABASE some_db") - databases = [db.name for db in spark.catalog.listDatabases()] - self.assertEquals(sorted(databases), ["default", "some_db"]) + spark.catalog._reset() + databases = [db.name for db in spark.catalog.listDatabases()] + self.assertEquals(databases, ["default"]) + spark.sql("CREATE DATABASE some_db") + databases = [db.name for db in spark.catalog.listDatabases()] + self.assertEquals(sorted(databases), ["default", "some_db"]) def test_list_tables(self): from pyspark.sql.catalog import Table spark = self.spark - with self.database("some_db"): - spark.sql("CREATE DATABASE some_db") - with self.table("tab1", "some_db.tab2"): - with self.tempView("temp_tab"): - self.assertEquals(spark.catalog.listTables(), []) - self.assertEquals(spark.catalog.listTables("some_db"), []) - spark.createDataFrame([(1, 1)]).createOrReplaceTempView("temp_tab") - spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") - spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT) USING parquet") - tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) - tablesDefault = \ - sorted(spark.catalog.listTables("default"), key=lambda t: t.name) - tablesSomeDb = \ - sorted(spark.catalog.listTables("some_db"), key=lambda t: t.name) - self.assertEquals(tables, tablesDefault) - self.assertEquals(len(tables), 2) - self.assertEquals(len(tablesSomeDb), 2) - self.assertEquals(tables[0], Table( - name="tab1", - database="default", - description=None, - tableType="MANAGED", - isTemporary=False)) - self.assertEquals(tables[1], Table( - name="temp_tab", - database=None, - description=None, - tableType="TEMPORARY", - isTemporary=True)) - self.assertEquals(tablesSomeDb[0], Table( - name="tab2", - database="some_db", - description=None, - tableType="MANAGED", - isTemporary=False)) - self.assertEquals(tablesSomeDb[1], Table( - name="temp_tab", - database=None, - description=None, - tableType="TEMPORARY", - isTemporary=True)) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.listTables("does_not_exist")) + spark.catalog._reset() + spark.sql("CREATE DATABASE some_db") + self.assertEquals(spark.catalog.listTables(), []) + self.assertEquals(spark.catalog.listTables("some_db"), []) + spark.createDataFrame([(1, 1)]).createOrReplaceTempView("temp_tab") + spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") + spark.sql("CREATE TABLE some_db.tab2 (name STRING, age INT) USING parquet") + tables = sorted(spark.catalog.listTables(), key=lambda t: t.name) + tablesDefault = sorted(spark.catalog.listTables("default"), key=lambda t: t.name) + tablesSomeDb = sorted(spark.catalog.listTables("some_db"), key=lambda t: t.name) + self.assertEquals(tables, tablesDefault) + self.assertEquals(len(tables), 2) + self.assertEquals(len(tablesSomeDb), 2) + self.assertEquals(tables[0], Table( + name="tab1", + database="default", + description=None, + tableType="MANAGED", + isTemporary=False)) + self.assertEquals(tables[1], Table( + name="temp_tab", + database=None, + description=None, + tableType="TEMPORARY", + isTemporary=True)) + self.assertEquals(tablesSomeDb[0], Table( + name="tab2", + database="some_db", + description=None, + tableType="MANAGED", + isTemporary=False)) + self.assertEquals(tablesSomeDb[1], Table( + name="temp_tab", + database=None, + description=None, + tableType="TEMPORARY", + isTemporary=True)) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.listTables("does_not_exist")) def test_list_functions(self): from pyspark.sql.catalog import Function spark = self.spark - with self.database("some_db"): - spark.sql("CREATE DATABASE some_db") - functions = dict((f.name, f) for f in spark.catalog.listFunctions()) - functionsDefault = dict((f.name, f) for f in spark.catalog.listFunctions("default")) - self.assertTrue(len(functions) > 200) - self.assertTrue("+" in functions) - self.assertTrue("like" in functions) - self.assertTrue("month" in functions) - self.assertTrue("to_date" in functions) - self.assertTrue("to_timestamp" in functions) - self.assertTrue("to_unix_timestamp" in functions) - self.assertTrue("current_database" in functions) - self.assertEquals(functions["+"], Function( - name="+", - description=None, - className="org.apache.spark.sql.catalyst.expressions.Add", - isTemporary=True)) - self.assertEquals(functions, functionsDefault) - - with self.function("func1", "some_db.func2"): - spark.catalog.registerFunction("temp_func", lambda x: str(x)) - spark.sql("CREATE FUNCTION func1 AS 'org.apache.spark.data.bricks'") - spark.sql("CREATE FUNCTION some_db.func2 AS 'org.apache.spark.data.bricks'") - newFunctions = dict((f.name, f) for f in spark.catalog.listFunctions()) - newFunctionsSomeDb = \ - dict((f.name, f) for f in spark.catalog.listFunctions("some_db")) - self.assertTrue(set(functions).issubset(set(newFunctions))) - self.assertTrue(set(functions).issubset(set(newFunctionsSomeDb))) - self.assertTrue("temp_func" in newFunctions) - self.assertTrue("func1" in newFunctions) - self.assertTrue("func2" not in newFunctions) - self.assertTrue("temp_func" in newFunctionsSomeDb) - self.assertTrue("func1" not in newFunctionsSomeDb) - self.assertTrue("func2" in newFunctionsSomeDb) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.listFunctions("does_not_exist")) + spark.catalog._reset() + spark.sql("CREATE DATABASE some_db") + functions = dict((f.name, f) for f in spark.catalog.listFunctions()) + functionsDefault = dict((f.name, f) for f in spark.catalog.listFunctions("default")) + self.assertTrue(len(functions) > 200) + self.assertTrue("+" in functions) + self.assertTrue("like" in functions) + self.assertTrue("month" in functions) + self.assertTrue("to_date" in functions) + self.assertTrue("to_timestamp" in functions) + self.assertTrue("to_unix_timestamp" in functions) + self.assertTrue("current_database" in functions) + self.assertEquals(functions["+"], Function( + name="+", + description=None, + className="org.apache.spark.sql.catalyst.expressions.Add", + isTemporary=True)) + self.assertEquals(functions, functionsDefault) + spark.catalog.registerFunction("temp_func", lambda x: str(x)) + spark.sql("CREATE FUNCTION func1 AS 'org.apache.spark.data.bricks'") + spark.sql("CREATE FUNCTION some_db.func2 AS 'org.apache.spark.data.bricks'") + newFunctions = dict((f.name, f) for f in spark.catalog.listFunctions()) + newFunctionsSomeDb = dict((f.name, f) for f in spark.catalog.listFunctions("some_db")) + self.assertTrue(set(functions).issubset(set(newFunctions))) + self.assertTrue(set(functions).issubset(set(newFunctionsSomeDb))) + self.assertTrue("temp_func" in newFunctions) + self.assertTrue("func1" in newFunctions) + self.assertTrue("func2" not in newFunctions) + self.assertTrue("temp_func" in newFunctionsSomeDb) + self.assertTrue("func1" not in newFunctionsSomeDb) + self.assertTrue("func2" in newFunctionsSomeDb) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.listFunctions("does_not_exist")) def test_list_columns(self): from pyspark.sql.catalog import Column spark = self.spark - with self.database("some_db"): - spark.sql("CREATE DATABASE some_db") - with self.table("tab1", "some_db.tab2"): - spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") - spark.sql( - "CREATE TABLE some_db.tab2 (nickname STRING, tolerance FLOAT) USING parquet") - columns = sorted(spark.catalog.listColumns("tab1"), key=lambda c: c.name) - columnsDefault = \ - sorted(spark.catalog.listColumns("tab1", "default"), key=lambda c: c.name) - self.assertEquals(columns, columnsDefault) - self.assertEquals(len(columns), 2) - self.assertEquals(columns[0], Column( - name="age", - description=None, - dataType="int", - nullable=True, - isPartition=False, - isBucket=False)) - self.assertEquals(columns[1], Column( - name="name", - description=None, - dataType="string", - nullable=True, - isPartition=False, - isBucket=False)) - columns2 = \ - sorted(spark.catalog.listColumns("tab2", "some_db"), key=lambda c: c.name) - self.assertEquals(len(columns2), 2) - self.assertEquals(columns2[0], Column( - name="nickname", - description=None, - dataType="string", - nullable=True, - isPartition=False, - isBucket=False)) - self.assertEquals(columns2[1], Column( - name="tolerance", - description=None, - dataType="float", - nullable=True, - isPartition=False, - isBucket=False)) - self.assertRaisesRegexp( - AnalysisException, - "tab2", - lambda: spark.catalog.listColumns("tab2")) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.listColumns("does_not_exist")) + spark.catalog._reset() + spark.sql("CREATE DATABASE some_db") + spark.sql("CREATE TABLE tab1 (name STRING, age INT) USING parquet") + spark.sql("CREATE TABLE some_db.tab2 (nickname STRING, tolerance FLOAT) USING parquet") + columns = sorted(spark.catalog.listColumns("tab1"), key=lambda c: c.name) + columnsDefault = sorted(spark.catalog.listColumns("tab1", "default"), key=lambda c: c.name) + self.assertEquals(columns, columnsDefault) + self.assertEquals(len(columns), 2) + self.assertEquals(columns[0], Column( + name="age", + description=None, + dataType="int", + nullable=True, + isPartition=False, + isBucket=False)) + self.assertEquals(columns[1], Column( + name="name", + description=None, + dataType="string", + nullable=True, + isPartition=False, + isBucket=False)) + columns2 = sorted(spark.catalog.listColumns("tab2", "some_db"), key=lambda c: c.name) + self.assertEquals(len(columns2), 2) + self.assertEquals(columns2[0], Column( + name="nickname", + description=None, + dataType="string", + nullable=True, + isPartition=False, + isBucket=False)) + self.assertEquals(columns2[1], Column( + name="tolerance", + description=None, + dataType="float", + nullable=True, + isPartition=False, + isBucket=False)) + self.assertRaisesRegexp( + AnalysisException, + "tab2", + lambda: spark.catalog.listColumns("tab2")) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.listColumns("does_not_exist")) def test_cache(self): spark = self.spark - with self.tempView("tab1", "tab2"): - spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab1") - spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab2") - self.assertFalse(spark.catalog.isCached("tab1")) - self.assertFalse(spark.catalog.isCached("tab2")) - spark.catalog.cacheTable("tab1") - self.assertTrue(spark.catalog.isCached("tab1")) - self.assertFalse(spark.catalog.isCached("tab2")) - spark.catalog.cacheTable("tab2") - spark.catalog.uncacheTable("tab1") - self.assertFalse(spark.catalog.isCached("tab1")) - self.assertTrue(spark.catalog.isCached("tab2")) - spark.catalog.clearCache() - self.assertFalse(spark.catalog.isCached("tab1")) - self.assertFalse(spark.catalog.isCached("tab2")) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.isCached("does_not_exist")) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.cacheTable("does_not_exist")) - self.assertRaisesRegexp( - AnalysisException, - "does_not_exist", - lambda: spark.catalog.uncacheTable("does_not_exist")) + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab1") + spark.createDataFrame([(2, 2), (3, 3)]).createOrReplaceTempView("tab2") + self.assertFalse(spark.catalog.isCached("tab1")) + self.assertFalse(spark.catalog.isCached("tab2")) + spark.catalog.cacheTable("tab1") + self.assertTrue(spark.catalog.isCached("tab1")) + self.assertFalse(spark.catalog.isCached("tab2")) + spark.catalog.cacheTable("tab2") + spark.catalog.uncacheTable("tab1") + self.assertFalse(spark.catalog.isCached("tab1")) + self.assertTrue(spark.catalog.isCached("tab2")) + spark.catalog.clearCache() + self.assertFalse(spark.catalog.isCached("tab1")) + self.assertFalse(spark.catalog.isCached("tab2")) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.isCached("does_not_exist")) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.cacheTable("does_not_exist")) + self.assertRaisesRegexp( + AnalysisException, + "does_not_exist", + lambda: spark.catalog.uncacheTable("does_not_exist")) def test_read_text_file_list(self): df = self.spark.read.text(['python/test_support/sql/text-test.txt', @@ -3433,38 +3285,37 @@ def count_bucketed_cols(names, table="pyspark_bucket"): num = len([c for c in cols if c.name in names and c.isBucket]) return num - with self.table("pyspark_bucket"): - # Test write with one bucketing column - df.write.bucketBy(3, "x").mode("overwrite").saveAsTable("pyspark_bucket") - self.assertEqual(count_bucketed_cols(["x"]), 1) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) - - # Test write two bucketing columns - df.write.bucketBy(3, "x", "y").mode("overwrite").saveAsTable("pyspark_bucket") - self.assertEqual(count_bucketed_cols(["x", "y"]), 2) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) - - # Test write with bucket and sort - df.write.bucketBy(2, "x").sortBy("z").mode("overwrite").saveAsTable("pyspark_bucket") - self.assertEqual(count_bucketed_cols(["x"]), 1) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) - - # Test write with a list of columns - df.write.bucketBy(3, ["x", "y"]).mode("overwrite").saveAsTable("pyspark_bucket") - self.assertEqual(count_bucketed_cols(["x", "y"]), 2) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) - - # Test write with bucket and sort with a list of columns - (df.write.bucketBy(2, "x") - .sortBy(["y", "z"]) - .mode("overwrite").saveAsTable("pyspark_bucket")) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) - - # Test write with bucket and sort with multiple columns - (df.write.bucketBy(2, "x") - .sortBy("y", "z") - .mode("overwrite").saveAsTable("pyspark_bucket")) - self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + # Test write with one bucketing column + df.write.bucketBy(3, "x").mode("overwrite").saveAsTable("pyspark_bucket") + self.assertEqual(count_bucketed_cols(["x"]), 1) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + + # Test write two bucketing columns + df.write.bucketBy(3, "x", "y").mode("overwrite").saveAsTable("pyspark_bucket") + self.assertEqual(count_bucketed_cols(["x", "y"]), 2) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + + # Test write with bucket and sort + df.write.bucketBy(2, "x").sortBy("z").mode("overwrite").saveAsTable("pyspark_bucket") + self.assertEqual(count_bucketed_cols(["x"]), 1) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + + # Test write with a list of columns + df.write.bucketBy(3, ["x", "y"]).mode("overwrite").saveAsTable("pyspark_bucket") + self.assertEqual(count_bucketed_cols(["x", "y"]), 2) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + + # Test write with bucket and sort with a list of columns + (df.write.bucketBy(2, "x") + .sortBy(["y", "z"]) + .mode("overwrite").saveAsTable("pyspark_bucket")) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + + # Test write with bucket and sort with multiple columns + (df.write.bucketBy(2, "x") + .sortBy("y", "z") + .mode("overwrite").saveAsTable("pyspark_bucket")) + self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) def _to_pandas(self): from datetime import datetime, date @@ -3748,31 +3599,6 @@ def test_repr_behaviors(self): self.assertEquals(None, df._repr_html_()) self.assertEquals(expected, df.__repr__()) - # SPARK-25591 - def test_same_accumulator_in_udfs(self): - from pyspark.sql.functions import udf - - data_schema = StructType([StructField("a", IntegerType(), True), - StructField("b", IntegerType(), True)]) - data = self.spark.createDataFrame([[1, 2]], schema=data_schema) - - test_accum = self.sc.accumulator(0) - - def first_udf(x): - test_accum.add(1) - return x - - def second_udf(x): - test_accum.add(100) - return x - - func_udf = udf(first_udf, IntegerType()) - func_udf2 = udf(second_udf, IntegerType()) - data = data.withColumn("out1", func_udf(data["a"])) - data = data.withColumn("out2", func_udf2(data["b"])) - data.collect() - self.assertEqual(test_accum.value, 101) - class HiveSparkSubmitTests(SparkSubmitTests): @@ -3913,48 +3739,6 @@ def test_query_execution_listener_on_collect_with_arrow(self): "The callback from the query execution listener should be called after 'toPandas'") -class SparkExtensionsTest(unittest.TestCase): - # These tests are separate because it uses 'spark.sql.extensions' which is - # static and immutable. This can't be set or unset, for example, via `spark.conf`. - - @classmethod - def setUpClass(cls): - import glob - from pyspark.find_spark_home import _find_spark_home - - SPARK_HOME = _find_spark_home() - filename_pattern = ( - "sql/core/target/scala-*/test-classes/org/apache/spark/sql/" - "SparkSessionExtensionSuite.class") - if not glob.glob(os.path.join(SPARK_HOME, filename_pattern)): - raise unittest.SkipTest( - "'org.apache.spark.sql.SparkSessionExtensionSuite' is not " - "available. Will skip the related tests.") - - # Note that 'spark.sql.extensions' is a static immutable configuration. - cls.spark = SparkSession.builder \ - .master("local[4]") \ - .appName(cls.__name__) \ - .config( - "spark.sql.extensions", - "org.apache.spark.sql.MyExtensions") \ - .getOrCreate() - - @classmethod - def tearDownClass(cls): - cls.spark.stop() - - def test_use_custom_class_for_extensions(self): - self.assertTrue( - self.spark._jsparkSession.sessionState().planner().strategies().contains( - self.spark._jvm.org.apache.spark.sql.MySparkStrategy(self.spark._jsparkSession)), - "MySparkStrategy not found in active planner strategies") - self.assertTrue( - self.spark._jsparkSession.sessionState().analyzer().extendedResolutionRules().contains( - self.spark._jvm.org.apache.spark.sql.MyRule(self.spark._jsparkSession)), - "MyRule not found in extended resolution rules") - - class SparkSessionTests(PySparkTestCase): # This test is separate because it's closely related with session's start and stop. @@ -3981,157 +3765,6 @@ def test_jvm_default_session_already_set(self): spark.stop() -class SparkSessionTests2(unittest.TestCase): - - def test_active_session(self): - spark = SparkSession.builder \ - .master("local") \ - .getOrCreate() - try: - activeSession = SparkSession.getActiveSession() - df = activeSession.createDataFrame([(1, 'Alice')], ['age', 'name']) - self.assertEqual(df.collect(), [Row(age=1, name=u'Alice')]) - finally: - spark.stop() - - def test_get_active_session_when_no_active_session(self): - active = SparkSession.getActiveSession() - self.assertEqual(active, None) - spark = SparkSession.builder \ - .master("local") \ - .getOrCreate() - active = SparkSession.getActiveSession() - self.assertEqual(active, spark) - spark.stop() - active = SparkSession.getActiveSession() - self.assertEqual(active, None) - - def test_SparkSession(self): - spark = SparkSession.builder \ - .master("local") \ - .config("some-config", "v2") \ - .getOrCreate() - try: - self.assertEqual(spark.conf.get("some-config"), "v2") - self.assertEqual(spark.sparkContext._conf.get("some-config"), "v2") - self.assertEqual(spark.version, spark.sparkContext.version) - spark.sql("CREATE DATABASE test_db") - spark.catalog.setCurrentDatabase("test_db") - self.assertEqual(spark.catalog.currentDatabase(), "test_db") - spark.sql("CREATE TABLE table1 (name STRING, age INT) USING parquet") - self.assertEqual(spark.table("table1").columns, ['name', 'age']) - self.assertEqual(spark.range(3).count(), 3) - finally: - spark.stop() - - def test_global_default_session(self): - spark = SparkSession.builder \ - .master("local") \ - .getOrCreate() - try: - self.assertEqual(SparkSession.builder.getOrCreate(), spark) - finally: - spark.stop() - - def test_default_and_active_session(self): - spark = SparkSession.builder \ - .master("local") \ - .getOrCreate() - activeSession = spark._jvm.SparkSession.getActiveSession() - defaultSession = spark._jvm.SparkSession.getDefaultSession() - try: - self.assertEqual(activeSession, defaultSession) - finally: - spark.stop() - - def test_config_option_propagated_to_existing_session(self): - session1 = SparkSession.builder \ - .master("local") \ - .config("spark-config1", "a") \ - .getOrCreate() - self.assertEqual(session1.conf.get("spark-config1"), "a") - session2 = SparkSession.builder \ - .config("spark-config1", "b") \ - .getOrCreate() - try: - self.assertEqual(session1, session2) - self.assertEqual(session1.conf.get("spark-config1"), "b") - finally: - session1.stop() - - def test_new_session(self): - session = SparkSession.builder \ - .master("local") \ - .getOrCreate() - newSession = session.newSession() - try: - self.assertNotEqual(session, newSession) - finally: - session.stop() - newSession.stop() - - def test_create_new_session_if_old_session_stopped(self): - session = SparkSession.builder \ - .master("local") \ - .getOrCreate() - session.stop() - newSession = SparkSession.builder \ - .master("local") \ - .getOrCreate() - try: - self.assertNotEqual(session, newSession) - finally: - newSession.stop() - - def test_active_session_with_None_and_not_None_context(self): - from pyspark.context import SparkContext - from pyspark.conf import SparkConf - sc = None - session = None - try: - sc = SparkContext._active_spark_context - self.assertEqual(sc, None) - activeSession = SparkSession.getActiveSession() - self.assertEqual(activeSession, None) - sparkConf = SparkConf() - sc = SparkContext.getOrCreate(sparkConf) - activeSession = sc._jvm.SparkSession.getActiveSession() - self.assertFalse(activeSession.isDefined()) - session = SparkSession(sc) - activeSession = sc._jvm.SparkSession.getActiveSession() - self.assertTrue(activeSession.isDefined()) - activeSession2 = SparkSession.getActiveSession() - self.assertNotEqual(activeSession2, None) - finally: - if session is not None: - session.stop() - if sc is not None: - sc.stop() - - -class SparkSessionTests3(ReusedSQLTestCase): - - def test_get_active_session_after_create_dataframe(self): - session2 = None - try: - activeSession1 = SparkSession.getActiveSession() - session1 = self.spark - self.assertEqual(session1, activeSession1) - session2 = self.spark.newSession() - activeSession2 = SparkSession.getActiveSession() - self.assertEqual(session1, activeSession2) - self.assertNotEqual(session2, activeSession2) - session2.createDataFrame([(1, 'Alice')], ['age', 'name']) - activeSession3 = SparkSession.getActiveSession() - self.assertEqual(session2, activeSession3) - session1.createDataFrame([(1, 'Alice')], ['age', 'name']) - activeSession4 = SparkSession.getActiveSession() - self.assertEqual(session1, activeSession4) - finally: - if session2 is not None: - session2.stop() - - class UDFInitializationTests(unittest.TestCase): def tearDown(self): if SparkSession._instantiatedSession is not None: diff --git a/python/pyspark/sql/window.py b/python/pyspark/sql/window.py index e76563dfaa9c8..d19ced954f04e 100644 --- a/python/pyspark/sql/window.py +++ b/python/pyspark/sql/window.py @@ -16,9 +16,11 @@ # import sys +if sys.version >= '3': + long = int from pyspark import since, SparkContext -from pyspark.sql.column import _to_seq, _to_java_column +from pyspark.sql.column import Column, _to_seq, _to_java_column __all__ = ["Window", "WindowSpec"] @@ -124,20 +126,45 @@ def rangeBetween(start, end): and "5" means the five off after the current row. We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, - and ``Window.currentRow`` to specify special boundary values, rather than using integral - values directly. + ``Window.currentRow``, ``pyspark.sql.functions.unboundedPreceding``, + ``pyspark.sql.functions.unboundedFollowing`` and ``pyspark.sql.functions.currentRow`` + to specify special boundary values, rather than using integral values directly. :param start: boundary start, inclusive. - The frame is unbounded if this is ``Window.unboundedPreceding``, or + The frame is unbounded if this is ``Window.unboundedPreceding``, + a column returned by ``pyspark.sql.functions.unboundedPreceding``, or any value less than or equal to max(-sys.maxsize, -9223372036854775808). :param end: boundary end, inclusive. - The frame is unbounded if this is ``Window.unboundedFollowing``, or + The frame is unbounded if this is ``Window.unboundedFollowing``, + a column returned by ``pyspark.sql.functions.unboundedFollowing``, or any value greater than or equal to min(sys.maxsize, 9223372036854775807). + + >>> from pyspark.sql import functions as F, SparkSession, Window + >>> spark = SparkSession.builder.getOrCreate() + >>> df = spark.createDataFrame( + ... [(1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")], ["id", "category"]) + >>> window = Window.orderBy("id").partitionBy("category").rangeBetween( + ... F.currentRow(), F.lit(1)) + >>> df.withColumn("sum", F.sum("id").over(window)).show() + +---+--------+---+ + | id|category|sum| + +---+--------+---+ + | 1| b| 3| + | 2| b| 5| + | 3| b| 3| + | 1| a| 4| + | 1| a| 4| + | 2| a| 2| + +---+--------+---+ """ - if start <= Window._PRECEDING_THRESHOLD: - start = Window.unboundedPreceding - if end >= Window._FOLLOWING_THRESHOLD: - end = Window.unboundedFollowing + if isinstance(start, (int, long)) and isinstance(end, (int, long)): + if start <= Window._PRECEDING_THRESHOLD: + start = Window.unboundedPreceding + if end >= Window._FOLLOWING_THRESHOLD: + end = Window.unboundedFollowing + elif isinstance(start, Column) and isinstance(end, Column): + start = start._jc + end = end._jc sc = SparkContext._active_spark_context jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rangeBetween(start, end) return WindowSpec(jspec) @@ -212,27 +239,34 @@ def rangeBetween(self, start, end): and "5" means the five off after the current row. We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, - and ``Window.currentRow`` to specify special boundary values, rather than using integral - values directly. + ``Window.currentRow``, ``pyspark.sql.functions.unboundedPreceding``, + ``pyspark.sql.functions.unboundedFollowing`` and ``pyspark.sql.functions.currentRow`` + to specify special boundary values, rather than using integral values directly. :param start: boundary start, inclusive. - The frame is unbounded if this is ``Window.unboundedPreceding``, or + The frame is unbounded if this is ``Window.unboundedPreceding``, + a column returned by ``pyspark.sql.functions.unboundedPreceding``, or any value less than or equal to max(-sys.maxsize, -9223372036854775808). :param end: boundary end, inclusive. - The frame is unbounded if this is ``Window.unboundedFollowing``, or + The frame is unbounded if this is ``Window.unboundedFollowing``, + a column returned by ``pyspark.sql.functions.unboundedFollowing``, or any value greater than or equal to min(sys.maxsize, 9223372036854775807). """ - if start <= Window._PRECEDING_THRESHOLD: - start = Window.unboundedPreceding - if end >= Window._FOLLOWING_THRESHOLD: - end = Window.unboundedFollowing + if isinstance(start, (int, long)) and isinstance(end, (int, long)): + if start <= Window._PRECEDING_THRESHOLD: + start = Window.unboundedPreceding + if end >= Window._FOLLOWING_THRESHOLD: + end = Window.unboundedFollowing + elif isinstance(start, Column) and isinstance(end, Column): + start = start._jc + end = end._jc return WindowSpec(self._jspec.rangeBetween(start, end)) def _test(): import doctest SparkContext('local[4]', 'PythonTest') - (failure_count, test_count) = doctest.testmod() + (failure_count, test_count) = doctest.testmod(optionflags=doctest.NORMALIZE_WHITESPACE) if failure_count: sys.exit(-1) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index e1c194b446504..3fa57ca85b37b 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -343,11 +343,9 @@ def union(self, *dstreams): raise ValueError("All DStreams should have same serializer") if len(set(s._slideDuration for s in dstreams)) > 1: raise ValueError("All DStreams should have same slide duration") - cls = SparkContext._jvm.org.apache.spark.streaming.api.java.JavaDStream - jdstreams = SparkContext._gateway.new_array(cls, len(dstreams)) - for i in range(0, len(dstreams)): - jdstreams[i] = dstreams[i]._jdstream - return DStream(self._jssc.union(jdstreams), self, dstreams[0]._jrdd_deserializer) + first = dstreams[0] + jrest = [d._jdstream for d in dstreams[1:]] + return DStream(self._jssc.union(first._jdstream, jrest), self, first._jrdd_deserializer) def addStreamingListener(self, streamingListener): """ diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index c253e5ce0e72e..ce42a857d0c06 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -45,7 +45,7 @@ class DStream(object): for more details on RDDs). DStreams can either be created from live data (such as, data from TCP - sockets, etc.) using a L{StreamingContext} or it can be + sockets, Kafka, Flume, etc.) using a L{StreamingContext} or it can be generated by transforming existing DStreams using operations such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each DStream periodically generates a RDD, either @@ -626,6 +626,7 @@ def __init__(self, prev, func): # Using type() to avoid folding the functions and compacting the DStreams which is not # not strictly an object of TransformedDStream. + # Changed here is to avoid bug in KafkaTransformedDStream when calling offsetRanges(). if (type(prev) is TransformedDStream and not prev.is_cached and not prev.is_checkpointed): prev_func = prev.func diff --git a/python/pyspark/streaming/flume.py b/python/pyspark/streaming/flume.py new file mode 100644 index 0000000000000..5de448114ece8 --- /dev/null +++ b/python/pyspark/streaming/flume.py @@ -0,0 +1,156 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +if sys.version >= "3": + from io import BytesIO +else: + from StringIO import StringIO +import warnings + +from py4j.protocol import Py4JJavaError + +from pyspark.storagelevel import StorageLevel +from pyspark.serializers import PairDeserializer, NoOpSerializer, UTF8Deserializer, read_int +from pyspark.streaming import DStream + +__all__ = ['FlumeUtils', 'utf8_decoder'] + + +def utf8_decoder(s): + """ Decode the unicode as UTF-8 """ + if s is None: + return None + return s.decode('utf-8') + + +class FlumeUtils(object): + + @staticmethod + def createStream(ssc, hostname, port, + storageLevel=StorageLevel.MEMORY_AND_DISK_2, + enableDecompression=False, + bodyDecoder=utf8_decoder): + """ + Create an input stream that pulls events from Flume. + + :param ssc: StreamingContext object + :param hostname: Hostname of the slave machine to which the flume data will be sent + :param port: Port of the slave machine to which the flume data will be sent + :param storageLevel: Storage level to use for storing the received objects + :param enableDecompression: Should netty server decompress input stream + :param bodyDecoder: A function used to decode body (default is utf8_decoder) + :return: A DStream object + + .. note:: Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. + See SPARK-22142. + """ + warnings.warn( + "Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. " + "See SPARK-22142.", + DeprecationWarning) + jlevel = ssc._sc._getJavaStorageLevel(storageLevel) + helper = FlumeUtils._get_helper(ssc._sc) + jstream = helper.createStream(ssc._jssc, hostname, port, jlevel, enableDecompression) + return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) + + @staticmethod + def createPollingStream(ssc, addresses, + storageLevel=StorageLevel.MEMORY_AND_DISK_2, + maxBatchSize=1000, + parallelism=5, + bodyDecoder=utf8_decoder): + """ + Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + This stream will poll the sink for data and will pull events as they are available. + + :param ssc: StreamingContext object + :param addresses: List of (host, port)s on which the Spark Sink is running. + :param storageLevel: Storage level to use for storing the received objects + :param maxBatchSize: The maximum number of events to be pulled from the Spark sink + in a single RPC call + :param parallelism: Number of concurrent requests this stream should send to the sink. + Note that having a higher number of requests concurrently being pulled + will result in this stream using more threads + :param bodyDecoder: A function used to decode body (default is utf8_decoder) + :return: A DStream object + + .. note:: Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. + See SPARK-22142. + """ + warnings.warn( + "Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. " + "See SPARK-22142.", + DeprecationWarning) + jlevel = ssc._sc._getJavaStorageLevel(storageLevel) + hosts = [] + ports = [] + for (host, port) in addresses: + hosts.append(host) + ports.append(port) + helper = FlumeUtils._get_helper(ssc._sc) + jstream = helper.createPollingStream( + ssc._jssc, hosts, ports, jlevel, maxBatchSize, parallelism) + return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) + + @staticmethod + def _toPythonDStream(ssc, jstream, bodyDecoder): + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + + def func(event): + headersBytes = BytesIO(event[0]) if sys.version >= "3" else StringIO(event[0]) + headers = {} + strSer = UTF8Deserializer() + for i in range(0, read_int(headersBytes)): + key = strSer.loads(headersBytes) + value = strSer.loads(headersBytes) + headers[key] = value + body = bodyDecoder(event[1]) + return (headers, body) + return stream.map(func) + + @staticmethod + def _get_helper(sc): + try: + return sc._jvm.org.apache.spark.streaming.flume.FlumeUtilsPythonHelper() + except TypeError as e: + if str(e) == "'JavaPackage' object is not callable": + FlumeUtils._printErrorMsg(sc) + raise + + @staticmethod + def _printErrorMsg(sc): + print(""" +________________________________________________________________________________________________ + + Spark Streaming's Flume libraries not found in class path. Try one of the following. + + 1. Include the Flume library and its dependencies with in the + spark-submit command as + + $ bin/spark-submit --packages org.apache.spark:spark-streaming-flume:%s ... + + 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, + Group Id = org.apache.spark, Artifact Id = spark-streaming-flume-assembly, Version = %s. + Then, include the jar in the spark-submit command as + + $ bin/spark-submit --jars ... + +________________________________________________________________________________________________ + +""" % (sc.version, sc.version)) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py new file mode 100644 index 0000000000000..ed2e0e7d10fa2 --- /dev/null +++ b/python/pyspark/streaming/kafka.py @@ -0,0 +1,506 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import warnings + +from py4j.protocol import Py4JJavaError + +from pyspark.rdd import RDD +from pyspark.storagelevel import StorageLevel +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer, PairDeserializer, \ + NoOpSerializer +from pyspark.streaming import DStream +from pyspark.streaming.dstream import TransformedDStream +from pyspark.streaming.util import TransformFunction + +__all__ = ['Broker', 'KafkaMessageAndMetadata', 'KafkaUtils', 'OffsetRange', + 'TopicAndPartition', 'utf8_decoder'] + + +def utf8_decoder(s): + """ Decode the unicode as UTF-8 """ + if s is None: + return None + return s.decode('utf-8') + + +class KafkaUtils(object): + + @staticmethod + def createStream(ssc, zkQuorum, groupId, topics, kafkaParams=None, + storageLevel=StorageLevel.MEMORY_AND_DISK_2, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder): + """ + Create an input stream that pulls messages from a Kafka Broker. + + :param ssc: StreamingContext object + :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). + :param groupId: The group id for this consumer. + :param topics: Dict of (topic_name -> numPartitions) to consume. + Each partition is consumed in its own thread. + :param kafkaParams: Additional params for Kafka + :param storageLevel: RDD storage level. + :param keyDecoder: A function used to decode key (default is utf8_decoder) + :param valueDecoder: A function used to decode value (default is utf8_decoder) + :return: A DStream object + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + if kafkaParams is None: + kafkaParams = dict() + kafkaParams.update({ + "zookeeper.connect": zkQuorum, + "group.id": groupId, + "zookeeper.connection.timeout.ms": "10000", + }) + if not isinstance(topics, dict): + raise TypeError("topics should be dict") + jlevel = ssc._sc._getJavaStorageLevel(storageLevel) + helper = KafkaUtils._get_helper(ssc._sc) + jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel) + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) + + @staticmethod + def createDirectStream(ssc, topics, kafkaParams, fromOffsets=None, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder, + messageHandler=None): + """ + Create an input stream that directly pulls messages from a Kafka Broker and specific offset. + + This is not a receiver based Kafka input stream, it directly pulls the message from Kafka + in each batch duration and processed without storing. + + This does not use Zookeeper to store offsets. The consumed offsets are tracked + by the stream itself. For interoperability with Kafka monitoring tools that depend on + Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + You can access the offsets used in each batch from the generated RDDs (see + + To recover from driver failures, you have to enable checkpointing in the StreamingContext. + The information on consumed offset can be recovered from the checkpoint. + See the programming guide for details (constraints, etc.). + + :param ssc: StreamingContext object. + :param topics: list of topic_name to consume. + :param kafkaParams: Additional params for Kafka. + :param fromOffsets: Per-topic/partition Kafka offsets defining the (inclusive) starting + point of the stream (a dictionary mapping `TopicAndPartition` to + integers). + :param keyDecoder: A function used to decode key (default is utf8_decoder). + :param valueDecoder: A function used to decode value (default is utf8_decoder). + :param messageHandler: A function used to convert KafkaMessageAndMetadata. You can assess + meta using messageHandler (default is None). + :return: A DStream object + + .. note:: Experimental + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + if fromOffsets is None: + fromOffsets = dict() + if not isinstance(topics, list): + raise TypeError("topics should be list") + if not isinstance(kafkaParams, dict): + raise TypeError("kafkaParams should be dict") + + def funcWithoutMessageHandler(k_v): + return (keyDecoder(k_v[0]), valueDecoder(k_v[1])) + + def funcWithMessageHandler(m): + m._set_key_decoder(keyDecoder) + m._set_value_decoder(valueDecoder) + return messageHandler(m) + + helper = KafkaUtils._get_helper(ssc._sc) + + jfromOffsets = dict([(k._jTopicAndPartition(helper), + v) for (k, v) in fromOffsets.items()]) + if messageHandler is None: + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + func = funcWithoutMessageHandler + jstream = helper.createDirectStreamWithoutMessageHandler( + ssc._jssc, kafkaParams, set(topics), jfromOffsets) + else: + ser = AutoBatchedSerializer(PickleSerializer()) + func = funcWithMessageHandler + jstream = helper.createDirectStreamWithMessageHandler( + ssc._jssc, kafkaParams, set(topics), jfromOffsets) + + stream = DStream(jstream, ssc, ser).map(func) + return KafkaDStream(stream._jdstream, ssc, stream._jrdd_deserializer) + + @staticmethod + def createRDD(sc, kafkaParams, offsetRanges, leaders=None, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder, + messageHandler=None): + """ + Create an RDD from Kafka using offset ranges for each topic and partition. + + :param sc: SparkContext object + :param kafkaParams: Additional params for Kafka + :param offsetRanges: list of offsetRange to specify topic:partition:[start, end) to consume + :param leaders: Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty + map, in which case leaders will be looked up on the driver. + :param keyDecoder: A function used to decode key (default is utf8_decoder) + :param valueDecoder: A function used to decode value (default is utf8_decoder) + :param messageHandler: A function used to convert KafkaMessageAndMetadata. You can assess + meta using messageHandler (default is None). + :return: An RDD object + + .. note:: Experimental + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + if leaders is None: + leaders = dict() + if not isinstance(kafkaParams, dict): + raise TypeError("kafkaParams should be dict") + if not isinstance(offsetRanges, list): + raise TypeError("offsetRanges should be list") + + def funcWithoutMessageHandler(k_v): + return (keyDecoder(k_v[0]), valueDecoder(k_v[1])) + + def funcWithMessageHandler(m): + m._set_key_decoder(keyDecoder) + m._set_value_decoder(valueDecoder) + return messageHandler(m) + + helper = KafkaUtils._get_helper(sc) + + joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges] + jleaders = dict([(k._jTopicAndPartition(helper), + v._jBroker(helper)) for (k, v) in leaders.items()]) + if messageHandler is None: + jrdd = helper.createRDDWithoutMessageHandler( + sc._jsc, kafkaParams, joffsetRanges, jleaders) + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + rdd = RDD(jrdd, sc, ser).map(funcWithoutMessageHandler) + else: + jrdd = helper.createRDDWithMessageHandler( + sc._jsc, kafkaParams, joffsetRanges, jleaders) + rdd = RDD(jrdd, sc).map(funcWithMessageHandler) + + return KafkaRDD(rdd._jrdd, sc, rdd._jrdd_deserializer) + + @staticmethod + def _get_helper(sc): + try: + return sc._jvm.org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper() + except TypeError as e: + if str(e) == "'JavaPackage' object is not callable": + KafkaUtils._printErrorMsg(sc) + raise + + @staticmethod + def _printErrorMsg(sc): + print(""" +________________________________________________________________________________________________ + + Spark Streaming's Kafka libraries not found in class path. Try one of the following. + + 1. Include the Kafka library and its dependencies with in the + spark-submit command as + + $ bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8:%s ... + + 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, + Group Id = org.apache.spark, Artifact Id = spark-streaming-kafka-0-8-assembly, Version = %s. + Then, include the jar in the spark-submit command as + + $ bin/spark-submit --jars ... + +________________________________________________________________________________________________ + +""" % (sc.version, sc.version)) + + +class OffsetRange(object): + """ + Represents a range of offsets from a single Kafka TopicAndPartition. + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, topic, partition, fromOffset, untilOffset): + """ + Create an OffsetRange to represent range of offsets + :param topic: Kafka topic name. + :param partition: Kafka partition id. + :param fromOffset: Inclusive starting offset. + :param untilOffset: Exclusive ending offset. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + self.topic = topic + self.partition = partition + self.fromOffset = fromOffset + self.untilOffset = untilOffset + + def __eq__(self, other): + if isinstance(other, self.__class__): + return (self.topic == other.topic + and self.partition == other.partition + and self.fromOffset == other.fromOffset + and self.untilOffset == other.untilOffset) + else: + return False + + def __ne__(self, other): + return not self.__eq__(other) + + def __str__(self): + return "OffsetRange(topic: %s, partition: %d, range: [%d -> %d]" \ + % (self.topic, self.partition, self.fromOffset, self.untilOffset) + + def _jOffsetRange(self, helper): + return helper.createOffsetRange(self.topic, self.partition, self.fromOffset, + self.untilOffset) + + +class TopicAndPartition(object): + """ + Represents a specific topic and partition for Kafka. + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, topic, partition): + """ + Create a Python TopicAndPartition to map to the Java related object + :param topic: Kafka topic name. + :param partition: Kafka partition id. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + self._topic = topic + self._partition = partition + + def _jTopicAndPartition(self, helper): + return helper.createTopicAndPartition(self._topic, self._partition) + + def __eq__(self, other): + if isinstance(other, self.__class__): + return (self._topic == other._topic + and self._partition == other._partition) + else: + return False + + def __ne__(self, other): + return not self.__eq__(other) + + def __hash__(self): + return (self._topic, self._partition).__hash__() + + +class Broker(object): + """ + Represent the host and port info for a Kafka broker. + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, host, port): + """ + Create a Python Broker to map to the Java related object. + :param host: Broker's hostname. + :param port: Broker's port. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + self._host = host + self._port = port + + def _jBroker(self, helper): + return helper.createBroker(self._host, self._port) + + +class KafkaRDD(RDD): + """ + A Python wrapper of KafkaRDD, to provide additional information on normal RDD. + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, jrdd, ctx, jrdd_deserializer): + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + RDD.__init__(self, jrdd, ctx, jrdd_deserializer) + + def offsetRanges(self): + """ + Get the OffsetRange of specific KafkaRDD. + :return: A list of OffsetRange + """ + helper = KafkaUtils._get_helper(self.ctx) + joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd()) + ranges = [OffsetRange(o.topic(), o.partition(), o.fromOffset(), o.untilOffset()) + for o in joffsetRanges] + return ranges + + +class KafkaDStream(DStream): + """ + A Python wrapper of KafkaDStream + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, jdstream, ssc, jrdd_deserializer): + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + DStream.__init__(self, jdstream, ssc, jrdd_deserializer) + + def foreachRDD(self, func): + """ + Apply a function to each RDD in this DStream. + """ + if func.__code__.co_argcount == 1: + old_func = func + func = lambda r, rdd: old_func(rdd) + jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) \ + .rdd_wrapper(lambda jrdd, ctx, ser: KafkaRDD(jrdd, ctx, ser)) + api = self._ssc._jvm.PythonDStream + api.callForeachRDD(self._jdstream, jfunc) + + def transform(self, func): + """ + Return a new DStream in which each RDD is generated by applying a function + on each RDD of this DStream. + + `func` can have one argument of `rdd`, or have two arguments of + (`time`, `rdd`) + """ + if func.__code__.co_argcount == 1: + oldfunc = func + func = lambda t, rdd: oldfunc(rdd) + assert func.__code__.co_argcount == 2, "func should take one or two arguments" + + return KafkaTransformedDStream(self, func) + + +class KafkaTransformedDStream(TransformedDStream): + """ + Kafka specific wrapper of TransformedDStream to transform on Kafka RDD. + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, prev, func): + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + TransformedDStream.__init__(self, prev, func) + + @property + def _jdstream(self): + if self._jdstream_val is not None: + return self._jdstream_val + + jfunc = TransformFunction(self._sc, self.func, self.prev._jrdd_deserializer) \ + .rdd_wrapper(lambda jrdd, ctx, ser: KafkaRDD(jrdd, ctx, ser)) + dstream = self._sc._jvm.PythonTransformedDStream(self.prev._jdstream.dstream(), jfunc) + self._jdstream_val = dstream.asJavaDStream() + return self._jdstream_val + + +class KafkaMessageAndMetadata(object): + """ + Kafka message and metadata information. Including topic, partition, offset and message + + .. note:: Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. + See SPARK-21893. + """ + + def __init__(self, topic, partition, offset, key, message): + """ + Python wrapper of Kafka MessageAndMetadata + :param topic: topic name of this Kafka message + :param partition: partition id of this Kafka message + :param offset: Offset of this Kafka message in the specific partition + :param key: key payload of this Kafka message, can be null if this Kafka message has no key + specified, the return data is undecoded bytearry. + :param message: actual message payload of this Kafka message, the return data is + undecoded bytearray. + """ + warnings.warn( + "Deprecated in 2.3.0. Kafka 0.8 support is deprecated as of Spark 2.3.0. " + "See SPARK-21893.", + DeprecationWarning) + self.topic = topic + self.partition = partition + self.offset = offset + self._rawKey = key + self._rawMessage = message + self._keyDecoder = utf8_decoder + self._valueDecoder = utf8_decoder + + def __str__(self): + return "KafkaMessageAndMetadata(topic: %s, partition: %d, offset: %d, key and message...)" \ + % (self.topic, self.partition, self.offset) + + def __repr__(self): + return self.__str__() + + def __reduce__(self): + return (KafkaMessageAndMetadata, + (self.topic, self.partition, self.offset, self._rawKey, self._rawMessage)) + + def _set_key_decoder(self, decoder): + self._keyDecoder = decoder + + def _set_value_decoder(self, decoder): + self._valueDecoder = decoder + + @property + def key(self): + return self._keyDecoder(self._rawKey) + + @property + def message(self): + return self._valueDecoder(self._rawMessage) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 34e3291651eec..15bd389b597f7 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -43,6 +43,8 @@ from pyspark.context import SparkConf, SparkContext, RDD from pyspark.storagelevel import StorageLevel from pyspark.streaming.context import StreamingContext +from pyspark.streaming.kafka import Broker, KafkaUtils, OffsetRange, TopicAndPartition +from pyspark.streaming.flume import FlumeUtils from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream from pyspark.streaming.listener import StreamingListener @@ -1042,6 +1044,401 @@ def check_output(n): self.ssc.stop(True, True) +class KafkaStreamTests(PySparkStreamingTestCase): + timeout = 20 # seconds + duration = 1 + + def setUp(self): + super(KafkaStreamTests, self).setUp() + self._kafkaTestUtils = self.ssc._jvm.org.apache.spark.streaming.kafka.KafkaTestUtils() + self._kafkaTestUtils.setup() + + def tearDown(self): + super(KafkaStreamTests, self).tearDown() + + if self._kafkaTestUtils is not None: + self._kafkaTestUtils.teardown() + self._kafkaTestUtils = None + + def _randomTopic(self): + return "topic-%d" % random.randint(0, 10000) + + def _validateStreamResult(self, sendData, stream): + result = {} + for i in chain.from_iterable(self._collect(stream.map(lambda x: x[1]), + sum(sendData.values()))): + result[i] = result.get(i, 0) + 1 + + self.assertEqual(sendData, result) + + def _validateRddResult(self, sendData, rdd): + result = {} + for i in rdd.map(lambda x: x[1]).collect(): + result[i] = result.get(i, 0) + 1 + self.assertEqual(sendData, result) + + def test_kafka_stream(self): + """Test the Python Kafka stream API.""" + topic = self._randomTopic() + sendData = {"a": 3, "b": 5, "c": 10} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), + "test-streaming-consumer", {topic: 1}, + {"auto.offset.reset": "smallest"}) + self._validateStreamResult(sendData, stream) + + def test_kafka_direct_stream(self): + """Test the Python direct Kafka stream API.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) + self._validateStreamResult(sendData, stream) + + def test_kafka_direct_stream_from_offset(self): + """Test the Python direct Kafka stream API with start offset specified.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + fromOffsets = {TopicAndPartition(topic, 0): long(0)} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) + self._validateStreamResult(sendData, stream) + + def test_kafka_rdd(self): + """Test the Python direct Kafka RDD API.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) + self._validateRddResult(sendData, rdd) + + def test_kafka_rdd_with_leaders(self): + """Test the Python direct Kafka RDD API with leaders.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + address = self._kafkaTestUtils.brokerAddress().split(":") + leaders = {TopicAndPartition(topic, 0): Broker(address[0], int(address[1]))} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) + self._validateRddResult(sendData, rdd) + + def test_kafka_rdd_get_offsetRanges(self): + """Test Python direct Kafka RDD get OffsetRanges.""" + topic = self._randomTopic() + sendData = {"a": 3, "b": 4, "c": 5} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) + self.assertEqual(offsetRanges, rdd.offsetRanges()) + + def test_kafka_direct_stream_foreach_get_offsetRanges(self): + """Test the Python direct Kafka stream foreachRDD get offsetRanges.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) + + offsetRanges = [] + + def getOffsetRanges(_, rdd): + for o in rdd.offsetRanges(): + offsetRanges.append(o) + + stream.foreachRDD(getOffsetRanges) + self.ssc.start() + self.wait_for(offsetRanges, 1) + + self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))]) + + def test_kafka_direct_stream_transform_get_offsetRanges(self): + """Test the Python direct Kafka stream transform get offsetRanges.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) + + offsetRanges = [] + + def transformWithOffsetRanges(rdd): + for o in rdd.offsetRanges(): + offsetRanges.append(o) + return rdd + + # Test whether it is ok mixing KafkaTransformedDStream and TransformedDStream together, + # only the TransformedDstreams can be folded together. + stream.transform(transformWithOffsetRanges).map(lambda kv: kv[1]).count().pprint() + self.ssc.start() + self.wait_for(offsetRanges, 1) + + self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))]) + + def test_topic_and_partition_equality(self): + topic_and_partition_a = TopicAndPartition("foo", 0) + topic_and_partition_b = TopicAndPartition("foo", 0) + topic_and_partition_c = TopicAndPartition("bar", 0) + topic_and_partition_d = TopicAndPartition("foo", 1) + + self.assertEqual(topic_and_partition_a, topic_and_partition_b) + self.assertNotEqual(topic_and_partition_a, topic_and_partition_c) + self.assertNotEqual(topic_and_partition_a, topic_and_partition_d) + + def test_kafka_direct_stream_transform_with_checkpoint(self): + """Test the Python direct Kafka stream transform with checkpoint correctly recovered.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + offsetRanges = [] + + def transformWithOffsetRanges(rdd): + for o in rdd.offsetRanges(): + offsetRanges.append(o) + return rdd + + self.ssc.stop(False) + self.ssc = None + tmpdir = "checkpoint-test-%d" % random.randint(0, 10000) + + def setup(): + ssc = StreamingContext(self.sc, 0.5) + ssc.checkpoint(tmpdir) + stream = KafkaUtils.createDirectStream(ssc, [topic], kafkaParams) + stream.transform(transformWithOffsetRanges).count().pprint() + return ssc + + try: + ssc1 = StreamingContext.getOrCreate(tmpdir, setup) + ssc1.start() + self.wait_for(offsetRanges, 1) + self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))]) + + # To make sure some checkpoint is written + time.sleep(3) + ssc1.stop(False) + ssc1 = None + + # Restart again to make sure the checkpoint is recovered correctly + ssc2 = StreamingContext.getOrCreate(tmpdir, setup) + ssc2.start() + ssc2.awaitTermination(3) + ssc2.stop(stopSparkContext=False, stopGraceFully=True) + ssc2 = None + finally: + shutil.rmtree(tmpdir) + + def test_kafka_rdd_message_handler(self): + """Test Python direct Kafka RDD MessageHandler.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 1, "c": 2} + offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))] + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()} + + def getKeyAndDoubleMessage(m): + return m and (m.key, m.message * 2) + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, + messageHandler=getKeyAndDoubleMessage) + self._validateRddResult({"aa": 1, "bb": 1, "cc": 2}, rdd) + + def test_kafka_direct_stream_message_handler(self): + """Test the Python direct Kafka stream MessageHandler.""" + topic = self._randomTopic() + sendData = {"a": 1, "b": 2, "c": 3} + kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(), + "auto.offset.reset": "smallest"} + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, sendData) + + def getKeyAndDoubleMessage(m): + return m and (m.key, m.message * 2) + + stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, + messageHandler=getKeyAndDoubleMessage) + self._validateStreamResult({"aa": 1, "bb": 2, "cc": 3}, stream) + + +class FlumeStreamTests(PySparkStreamingTestCase): + timeout = 20 # seconds + duration = 1 + + def setUp(self): + super(FlumeStreamTests, self).setUp() + self._utils = self.ssc._jvm.org.apache.spark.streaming.flume.FlumeTestUtils() + + def tearDown(self): + if self._utils is not None: + self._utils.close() + self._utils = None + + super(FlumeStreamTests, self).tearDown() + + def _startContext(self, n, compressed): + # Start the StreamingContext and also collect the result + dstream = FlumeUtils.createStream(self.ssc, "localhost", self._utils.getTestPort(), + enableDecompression=compressed) + result = [] + + def get_output(_, rdd): + for event in rdd.collect(): + if len(result) < n: + result.append(event) + dstream.foreachRDD(get_output) + self.ssc.start() + return result + + def _validateResult(self, input, result): + # Validate both the header and the body + header = {"test": "header"} + self.assertEqual(len(input), len(result)) + for i in range(0, len(input)): + self.assertEqual(header, result[i][0]) + self.assertEqual(input[i], result[i][1]) + + def _writeInput(self, input, compressed): + # Try to write input to the receiver until success or timeout + start_time = time.time() + while True: + try: + self._utils.writeInput(input, compressed) + break + except: + if time.time() - start_time < self.timeout: + time.sleep(0.01) + else: + raise + + def test_flume_stream(self): + input = [str(i) for i in range(1, 101)] + result = self._startContext(len(input), False) + self._writeInput(input, False) + self.wait_for(result, len(input)) + self._validateResult(input, result) + + def test_compressed_flume_stream(self): + input = [str(i) for i in range(1, 101)] + result = self._startContext(len(input), True) + self._writeInput(input, True) + self.wait_for(result, len(input)) + self._validateResult(input, result) + + +class FlumePollingStreamTests(PySparkStreamingTestCase): + timeout = 20 # seconds + duration = 1 + maxAttempts = 5 + + def setUp(self): + self._utils = self.sc._jvm.org.apache.spark.streaming.flume.PollingFlumeTestUtils() + + def tearDown(self): + if self._utils is not None: + self._utils.close() + self._utils = None + + def _writeAndVerify(self, ports): + # Set up the streaming context and input streams + ssc = StreamingContext(self.sc, self.duration) + try: + addresses = [("localhost", port) for port in ports] + dstream = FlumeUtils.createPollingStream( + ssc, + addresses, + maxBatchSize=self._utils.eventsPerBatch(), + parallelism=5) + outputBuffer = [] + + def get_output(_, rdd): + for e in rdd.collect(): + outputBuffer.append(e) + + dstream.foreachRDD(get_output) + ssc.start() + self._utils.sendDataAndEnsureAllDataHasBeenReceived() + + self.wait_for(outputBuffer, self._utils.getTotalEvents()) + outputHeaders = [event[0] for event in outputBuffer] + outputBodies = [event[1] for event in outputBuffer] + self._utils.assertOutput(outputHeaders, outputBodies) + finally: + ssc.stop(False) + + def _testMultipleTimes(self, f): + attempt = 0 + while True: + try: + f() + break + except: + attempt += 1 + if attempt >= self.maxAttempts: + raise + else: + import traceback + traceback.print_exc() + + def _testFlumePolling(self): + try: + port = self._utils.startSingleSink() + self._writeAndVerify([port]) + self._utils.assertChannelsAreEmpty() + finally: + self._utils.close() + + def _testFlumePollingMultipleHosts(self): + try: + port = self._utils.startSingleSink() + self._writeAndVerify([port]) + self._utils.assertChannelsAreEmpty() + finally: + self._utils.close() + + def test_flume_polling(self): + self._testMultipleTimes(self._testFlumePolling) + + def test_flume_polling_multiple_hosts(self): + self._testMultipleTimes(self._testFlumePollingMultipleHosts) + + class KinesisStreamTests(PySparkStreamingTestCase): def test_kinesis_stream_api(self): @@ -1113,6 +1510,40 @@ def search_jar(dir, name_prefix): return [jar for jar in jars if not jar.endswith(ignored_jar_suffixes)] +def search_kafka_assembly_jar(): + SPARK_HOME = os.environ["SPARK_HOME"] + kafka_assembly_dir = os.path.join(SPARK_HOME, "external/kafka-0-8-assembly") + jars = search_jar(kafka_assembly_dir, "spark-streaming-kafka-0-8-assembly") + if not jars: + raise Exception( + ("Failed to find Spark Streaming kafka assembly jar in %s. " % kafka_assembly_dir) + + "You need to build Spark with " + "'build/sbt -Pkafka-0-8 assembly/package streaming-kafka-0-8-assembly/assembly' or " + "'build/mvn -DskipTests -Pkafka-0-8 package' before running this test.") + elif len(jars) > 1: + raise Exception(("Found multiple Spark Streaming Kafka assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) + else: + return jars[0] + + +def search_flume_assembly_jar(): + SPARK_HOME = os.environ["SPARK_HOME"] + flume_assembly_dir = os.path.join(SPARK_HOME, "external/flume-assembly") + jars = search_jar(flume_assembly_dir, "spark-streaming-flume-assembly") + if not jars: + raise Exception( + ("Failed to find Spark Streaming Flume assembly jar in %s. " % flume_assembly_dir) + + "You need to build Spark with " + "'build/sbt -Pflume assembly/package streaming-flume-assembly/assembly' or " + "'build/mvn -DskipTests -Pflume package' before running this test.") + elif len(jars) > 1: + raise Exception(("Found multiple Spark Streaming Flume assembly JARs: %s; please " + "remove all but one") % (", ".join(jars))) + else: + return jars[0] + + def _kinesis_asl_assembly_dir(): SPARK_HOME = os.environ["SPARK_HOME"] return os.path.join(SPARK_HOME, "external/kinesis-asl-assembly") @@ -1129,26 +1560,50 @@ def search_kinesis_asl_assembly_jar(): return jars[0] +# Must be same as the variable and condition defined in modules.py +flume_test_environ_var = "ENABLE_FLUME_TESTS" +are_flume_tests_enabled = os.environ.get(flume_test_environ_var) == '1' +# Must be same as the variable and condition defined in modules.py +kafka_test_environ_var = "ENABLE_KAFKA_0_8_TESTS" +are_kafka_tests_enabled = os.environ.get(kafka_test_environ_var) == '1' # Must be same as the variable and condition defined in KinesisTestUtils.scala and modules.py kinesis_test_environ_var = "ENABLE_KINESIS_TESTS" are_kinesis_tests_enabled = os.environ.get(kinesis_test_environ_var) == '1' if __name__ == "__main__": from pyspark.streaming.tests import * + kafka_assembly_jar = search_kafka_assembly_jar() + flume_assembly_jar = search_flume_assembly_jar() kinesis_asl_assembly_jar = search_kinesis_asl_assembly_jar() if kinesis_asl_assembly_jar is None: kinesis_jar_present = False - jars_args = "" + jars = "%s,%s" % (kafka_assembly_jar, flume_assembly_jar) else: kinesis_jar_present = True - jars_args = "--jars %s" % kinesis_asl_assembly_jar + jars = "%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, kinesis_asl_assembly_jar) existing_args = os.environ.get("PYSPARK_SUBMIT_ARGS", "pyspark-shell") + jars_args = "--jars %s" % jars os.environ["PYSPARK_SUBMIT_ARGS"] = " ".join([jars_args, existing_args]) testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, CheckpointTests, StreamingListenerTests] + if are_flume_tests_enabled: + testcases.append(FlumeStreamTests) + testcases.append(FlumePollingStreamTests) + else: + sys.stderr.write( + "Skipped test_flume_stream (enable by setting environment variable %s=1" + % flume_test_environ_var) + + if are_kafka_tests_enabled: + testcases.append(KafkaStreamTests) + else: + sys.stderr.write( + "Skipped test_kafka_stream (enable by setting environment variable %s=1" + % kafka_test_environ_var) + if kinesis_jar_present is True: testcases.append(KinesisStreamTests) elif are_kinesis_tests_enabled is False: diff --git a/python/setup.py b/python/setup.py index 7da67a4109ed1..c447f2d40343d 100644 --- a/python/setup.py +++ b/python/setup.py @@ -201,7 +201,7 @@ def _supports_symlinks(): 'pyspark.examples.src.main.python': ['*.py', '*/*.py']}, scripts=scripts, license='http://www.apache.org/licenses/LICENSE-2.0', - install_requires=['py4j==0.10.8.1'], + install_requires=['py4j==0.10.7'], setup_requires=['pypandoc'], extras_require={ 'ml': ['numpy>=1.7'], diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index b89ea383bf872..90bac19cba019 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -29,7 +29,7 @@ Spark Project Kubernetes kubernetes - 4.1.0 + 3.0.0 diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 0a38de4a5141e..abfcfd81172e9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -247,45 +247,19 @@ private[spark] object Config extends Logging { .stringConf .checkValue(pv => List("2", "3").contains(pv), "Ensure that major Python version is either Python2 or Python3") - .createWithDefault("3") + .createWithDefault("2") - val KUBERNETES_KERBEROS_KRB5_FILE = - ConfigBuilder("spark.kubernetes.kerberos.krb5.path") - .doc("Specify the local location of the krb5.conf file to be mounted on the driver " + - "and executors for Kerberos. Note: The KDC defined needs to be " + - "visible from inside the containers ") - .stringConf - .createOptional - - val KUBERNETES_KERBEROS_KRB5_CONFIG_MAP = - ConfigBuilder("spark.kubernetes.kerberos.krb5.configMapName") - .doc("Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted " + - "on the driver and executors for Kerberos. Note: The KDC defined" + - "needs to be visible from inside the containers ") - .stringConf - .createOptional - - val KUBERNETES_HADOOP_CONF_CONFIG_MAP = - ConfigBuilder("spark.kubernetes.hadoop.configMapName") - .doc("Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, " + - "to be mounted on the driver and executors for custom Hadoop configuration.") - .stringConf - .createOptional - - val KUBERNETES_KERBEROS_DT_SECRET_NAME = - ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.name") - .doc("Specify the name of the secret where your existing delegation tokens are stored. " + - "This removes the need for the job user to provide any keytab for launching a job") + val KUBERNETES_DRIVER_PODTEMPLATE_FILE = + ConfigBuilder("spark.kubernetes.driver.podTemplateFile") + .doc("File containing a template pod spec for the driver") .stringConf .createOptional - val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = - ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.itemKey") - .doc("Specify the item key of the data where your existing delegation tokens are stored. " + - "This removes the need for the job user to provide any keytab for launching a job") + val KUBERNETES_EXECUTOR_PODTEMPLATE_FILE = + ConfigBuilder("spark.kubernetes.executor.podTemplateFile") + .doc("File containing a template pod spec for executors") .stringConf .createOptional - val APP_RESOURCE_TYPE = ConfigBuilder("spark.kubernetes.resource.type") .doc("This sets the resource type internally") @@ -302,30 +276,6 @@ private[spark] object Config extends Logging { .booleanConf .createWithDefault(false) - val KUBERNETES_DRIVER_PODTEMPLATE_FILE = - ConfigBuilder("spark.kubernetes.driver.podTemplateFile") - .doc("File containing a template pod spec for the driver") - .stringConf - .createOptional - - val KUBERNETES_EXECUTOR_PODTEMPLATE_FILE = - ConfigBuilder("spark.kubernetes.executor.podTemplateFile") - .doc("File containing a template pod spec for executors") - .stringConf - .createOptional - - val KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME = - ConfigBuilder("spark.kubernetes.driver.podTemplateContainerName") - .doc("container name to be used as a basis for the driver in the given pod template") - .stringConf - .createOptional - - val KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME = - ConfigBuilder("spark.kubernetes.executor.podTemplateContainerName") - .doc("container name to be used as a basis for executors in the given pod template") - .stringConf - .createOptional - val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 6bb39d8f01bbd..4534721c66cf6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -64,14 +64,12 @@ private[spark] object Constants { val ENV_MOUNTED_FILES_FROM_SECRET_DIR = "SPARK_MOUNTED_FILES_FROM_SECRET_DIR" val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR" - val ENV_SPARK_USER = "SPARK_USER" // Spark app configs for containers val MOUNTED_FILES_SECRET_DIR = "/var/data/spark-submitted-files" val SPARK_CONF_VOLUME = "spark-conf-volume" val SPARK_CONF_DIR_INTERNAL = "/opt/spark/conf" val SPARK_CONF_FILE_NAME = "spark.properties" val SPARK_CONF_PATH = s"$SPARK_CONF_DIR_INTERNAL/$SPARK_CONF_FILE_NAME" - val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" // BINDINGS val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" @@ -83,8 +81,8 @@ private[spark] object Constants { // Pod spec templates val EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME = "pod-spec-template.yml" - val EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH = "/opt/spark/pod-template" - val POD_TEMPLATE_VOLUME = "pod-template-volume" + val EXECUTOR_POD_SPEC_TEMPLATE_MOUNTHPATH = "/opt/spark/pod-template" + val POD_TEMPLATE_VOLUME = "podspec-volume" val POD_TEMPLATE_CONFIGMAP = "podspec-configmap" val POD_TEMPLATE_KEY = "podspec-configmap-key" @@ -93,29 +91,4 @@ private[spark] object Constants { val DEFAULT_DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" val DEFAULT_EXECUTOR_CONTAINER_NAME = "spark-kubernetes-executor" val MEMORY_OVERHEAD_MIN_MIB = 384L - - // Hadoop Configuration - val HADOOP_FILE_VOLUME = "hadoop-properties" - val KRB_FILE_VOLUME = "krb5-file" - val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" - val KRB_FILE_DIR_PATH = "/etc" - val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" - val HADOOP_CONFIG_MAP_NAME = - "spark.kubernetes.executor.hadoopConfigMapName" - val KRB5_CONFIG_MAP_NAME = - "spark.kubernetes.executor.krb5ConfigMapName" - - // Kerberos Configuration - val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens" - val KERBEROS_DT_SECRET_NAME = - "spark.kubernetes.kerberos.dt-secret-name" - val KERBEROS_DT_SECRET_KEY = - "spark.kubernetes.kerberos.dt-secret-key" - val KERBEROS_SPARK_USER_NAME = - "spark.kubernetes.kerberos.spark-user-name" - val KERBEROS_SECRET_KEY = "hadoop-tokens" - - // Hadoop credentials secrets for the Spark app. - val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" - val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index da9281dc235ca..38a55b4eb0b25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -19,12 +19,10 @@ package org.apache.spark.deploy.k8s import scala.collection.mutable import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferenceBuilder, Pod} -import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry @@ -50,13 +48,6 @@ private[spark] case class KubernetesExecutorSpecificConf( driverPod: Option[Pod]) extends KubernetesRoleSpecificConf -/* - * Structure containing metadata for HADOOP_CONF_DIR customization - */ -private[spark] case class HadoopConfSpec( - hadoopConfDir: Option[String], - hadoopConfigMapName: Option[String]) - /** * Structure containing metadata for Kubernetes logic to build Spark pods. */ @@ -72,15 +63,7 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleSecretEnvNamesToKeyRefs: Map[String, String], roleEnvs: Map[String, String], roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], - sparkFiles: Seq[String], - hadoopConfSpec: Option[HadoopConfSpec]) { - - def hadoopConfigMapName: String = s"$appResourceNamePrefix-hadoop-config" - - def krbConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - - def tokenManager(conf: SparkConf, hConf: Configuration): KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager(conf, hConf) + sparkFiles: Seq[String]) { def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) @@ -135,8 +118,7 @@ private[spark] object KubernetesConf { mainAppResource: Option[MainAppResource], mainClass: String, appArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfDir: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { + maybePyFiles: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { val sparkConfWithMainAppJar = sparkConf.clone() val additionalFiles = mutable.ArrayBuffer.empty[String] mainAppResource.foreach { @@ -215,19 +197,6 @@ private[spark] object KubernetesConf { .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) ++ additionalFiles - val hadoopConfigMapName = sparkConf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) - KubernetesUtils.requireNandDefined( - hadoopConfDir, - hadoopConfigMapName, - "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + - "as the creation of an additional ConfigMap, when one is already specified is extraneous" ) - val hadoopConfSpec = - if (hadoopConfDir.isDefined || hadoopConfigMapName.isDefined) { - Some(HadoopConfSpec(hadoopConfDir, hadoopConfigMapName)) - } else { - None - } - KubernetesConf( sparkConfWithMainAppJar, KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs), @@ -240,8 +209,7 @@ private[spark] object KubernetesConf { driverSecretEnvNamesToKeyRefs, driverEnvs, driverVolumes, - sparkFiles, - hadoopConfSpec) + sparkFiles) } def createExecutorConf( @@ -298,7 +266,6 @@ private[spark] object KubernetesConf { executorEnvSecrets, executorEnv, executorVolumes, - Seq.empty[String], - None) + Seq.empty[String]) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index 437bcd822f7e8..5aaa7a1caa749 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -18,10 +18,9 @@ package org.apache.spark.deploy.k8s import java.io.File -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, PodBuilder} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, PodBuilder, Time} import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging @@ -43,27 +42,8 @@ private[spark] object KubernetesUtils extends Logging { sparkConf.getAllWithPrefix(prefix).toMap } - def requireBothOrNeitherDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenFirstIsMissing: String, - errMessageWhenSecondIsMissing: String): Unit = { - requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) - requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) - } - - def requireSecondIfFirstIsDefined( - opt1: Option[_], - opt2: Option[_], - errMessageWhenSecondIsMissing: String): Unit = { - opt1.foreach { _ => - require(opt2.isDefined, errMessageWhenSecondIsMissing) - } - } - def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { opt1.foreach { _ => require(opt2.isEmpty, errMessage) } - opt2.foreach { _ => require(opt1.isEmpty, errMessage) } } /** @@ -99,11 +79,19 @@ private[spark] object KubernetesUtils extends Logging { def loadPodFromTemplate( kubernetesClient: KubernetesClient, - templateFile: File, - containerName: Option[String]): SparkPod = { + templateFile: File): SparkPod = { try { val pod = kubernetesClient.pods().load(templateFile).get() - selectSparkContainer(pod, containerName) + pod.getSpec.getContainers.asScala.toList match { + case first :: rest => SparkPod( + new PodBuilder(pod) + .editSpec() + .withContainers(rest.asJava) + .endSpec() + .build(), + first) + case Nil => SparkPod(pod, new ContainerBuilder().build()) + } } catch { case e: Exception => logError( @@ -112,32 +100,6 @@ private[spark] object KubernetesUtils extends Logging { } } - def selectSparkContainer(pod: Pod, containerName: Option[String]): SparkPod = { - def selectNamedContainer( - containers: List[Container], name: String): Option[(Container, List[Container])] = - containers.partition(_.getName == name) match { - case (sparkContainer :: Nil, rest) => Some((sparkContainer, rest)) - case _ => - logWarning( - s"specified container ${name} not found on pod template, " + - s"falling back to taking the first container") - Option.empty - } - val containers = pod.getSpec.getContainers.asScala.toList - containerName - .flatMap(selectNamedContainer(containers, _)) - .orElse(containers.headOption.map((_, containers.tail))) - .map { - case (sparkContainer: Container, rest: List[Container]) => SparkPod( - new PodBuilder(pod) - .editSpec() - .withContainers(rest.asJava) - .endSpec() - .build(), - sparkContainer) - }.getOrElse(SparkPod(pod, new ContainerBuilder().build())) - } - def parseMasterUrl(url: String): String = url.substring("k8s://".length) def formatPairsBundle(pairs: Seq[(String, String)], indent: Int = 1) : String = { @@ -213,7 +175,7 @@ private[spark] object KubernetesUtils extends Logging { }.getOrElse(Seq(("container state", "N/A"))) } - def formatTime(time: String): String = { - if (time != null) time else "N/A" + def formatTime(time: Time): String = { + if (time != null) time.getTime else "N/A" } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 77bd66b608e7c..c47e78cbf19e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -42,9 +42,6 @@ private[spark] object SparkKubernetesClientFactory { sparkConf: SparkConf, defaultServiceAccountToken: Option[File], defaultServiceAccountCaCert: Option[File]): KubernetesClient = { - - // TODO [SPARK-25887] Support configurable context - val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) @@ -66,8 +63,6 @@ private[spark] object SparkKubernetesClientFactory { .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") val dispatcher = new Dispatcher( ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) - - // TODO [SPARK-25887] Create builder in a way that respects configurable context val config = new ConfigBuilder() .withApiVersion("v1") .withMasterUrl(master) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala deleted file mode 100644 index fd09de2a918a1..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for bootstraping the container with ConfigMaps - * containing Hadoop config files mounted as volumes and an ENV variable - * pointed to the mounted file directory. - */ -private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val hadoopConfDirCMapName = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - require(hadoopConfDirCMapName.isDefined, - "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + - " using pre-existing ConfigMaps") - logInfo("HADOOP_CONF_DIR defined") - HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala deleted file mode 100644 index 5b6a6d5a7db45..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for setting ENV_SPARK_USER when HADOOP_FILES are detected - * however, this step would not be run if Kerberos is enabled, as Kerberos sets SPARK_USER - */ -private[spark] class HadoopSparkUserExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) - HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala deleted file mode 100644 index ce47933b7f700..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils._ -import org.apache.spark.internal.Logging - -/** - * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the - * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. - */ -private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - require(kubernetesConf.hadoopConfSpec.isDefined, - "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") - private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get - private val conf = kubernetesConf.sparkConf - private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) - private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val existingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val existingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) - private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) - private val isKerberosEnabled = - (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || - (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (krb5File.isDefined || krb5CMap.isDefined)) - require(keytab.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Keytab") - - require(existingSecretName.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Secret") - - KubernetesUtils.requireNandDefined( - krb5File, - krb5CMap, - "Do not specify both a Krb5 local file and the ConfigMap as the creation " + - "of an additional ConfigMap, when one is already specified, is extraneous") - - KubernetesUtils.requireBothOrNeitherDefined( - keytab, - principal, - "If a Kerberos principal is specified you must also specify a Kerberos keytab", - "If a Kerberos keytab is specified you must also specify a Kerberos principal") - - KubernetesUtils.requireBothOrNeitherDefined( - existingSecretName, - existingSecretItemKey, - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret", - "If a secret storing a Kerberos Delegation Token is specified you must also" + - " specify the item-key where the data is stored") - - private val hadoopConfigurationFiles = hadoopConfDirSpec.hadoopConfDir.map { hConfDir => - HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) - } - private val newHadoopConfigMapName = - if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { - Some(kubernetesConf.hadoopConfigMapName) - } else { - None - } - - // Either use pre-existing secret or login to create new Secret with DT stored within - private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { - secretName <- existingSecretName - secretItemKey <- existingSecretItemKey - } yield { - KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) - }).orElse( - if (isKerberosEnabled) { - Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) - } else { - None - } - ) - - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDirSpec.hadoopConfDir, - newHadoopConfigMapName, - hadoopConfDirSpec.hadoopConfigMapName, - pod) - kerberosConfSpec.map { hSpec => - HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - krb5File, - Some(kubernetesConf.krbConfigMapName), - krb5CMap, - hadoopBasedSparkPod) - }.getOrElse( - HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, - hadoopBasedSparkPod)) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = kerberosConfSpec.map { hSpec => - Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) - }.getOrElse( - Map(KERBEROS_SPARK_USER_NAME -> - kubeTokenManager.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse( - kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues - } - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val hadoopConfConfigMap = for { - hName <- newHadoopConfigMapName - hFiles <- hadoopConfigurationFiles - } yield { - HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles) - } - - val krb5ConfigMap = krb5File.map { fileLocation => - HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.krbConfigMapName, - fileLocation) - } - - val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) - - hadoopConfConfigMap.toSeq ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala deleted file mode 100644 index 06a88b6c229f7..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for mounting the DT secret for the executors - */ -private[spark] class KerberosConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) - require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") - - override def configurePod(pod: SparkPod): SparkPod = { - logInfo(s"Mounting Resources for Kerberos") - HadoopBootstrapUtil.bootstrapKerberosPod( - sparkConf.get(KERBEROS_DT_SECRET_NAME), - sparkConf.get(KERBEROS_DT_SECRET_KEY), - sparkConf.get(KERBEROS_SPARK_USER_NAME), - None, - None, - maybeKrb5CMap, - pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty[HasMetadata] -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index e60259c4a9b5a..bb0e2b3128efd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -56,9 +56,8 @@ private[spark] class MountVolumesFeatureStep( val volumeBuilder = spec.volumeConf match { case KubernetesHostPathVolumeConf(hostPath) => - /* "" means that no checks will be performed before mounting the hostPath volume */ new VolumeBuilder() - .withHostPath(new HostPathVolumeSource(hostPath, "")) + .withHostPath(new HostPathVolumeSource(hostPath)) case KubernetesPVCVolumeConf(claimName) => new VolumeBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala index 96a8013246b74..52baed636c3f4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -48,7 +48,7 @@ private[spark] class PodTemplateConfigMapStep( val containerWithVolume = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(POD_TEMPLATE_VOLUME) - .withMountPath(EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) + .withMountPath(EXECUTOR_POD_SPEC_TEMPLATE_MOUNTHPATH) .endVolumeMount() .build() SparkPod(podWithVolume, containerWithVolume) @@ -56,7 +56,7 @@ private[spark] class PodTemplateConfigMapStep( def getAdditionalPodSystemProperties(): Map[String, String] = Map[String, String]( KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key -> - (EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) + (EXECUTOR_POD_SPEC_TEMPLATE_MOUNTHPATH + "/" + EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) def getAdditionalKubernetesResources(): Seq[HasMetadata] = { require(conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala deleted file mode 100644 index 5bee766caf2be..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ /dev/null @@ -1,283 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import java.io.File -import java.nio.charset.StandardCharsets - -import scala.collection.JavaConverters._ - -import com.google.common.io.Files -import io.fabric8.kubernetes.api.model._ - -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.SparkPod -import org.apache.spark.internal.Logging - -private[spark] object HadoopBootstrapUtil extends Logging { - - /** - * Mounting the DT secret for both the Driver and the executors - * - * @param dtSecretName Name of the secret that stores the Delegation Token - * @param dtSecretItemKey Name of the Item Key storing the Delegation Token - * @param userName Name of the SparkUser to set SPARK_USER - * @param fileLocation Optional Location of the krb5 file - * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 - * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - fileLocation: Option[String], - newKrb5ConfName: Option[String], - existingKrb5ConfName: Option[String], - pod: SparkPod): SparkPod = { - - val preConfigMapVolume = existingKrb5ConfName.map { kconf => - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(kconf) - .endConfigMap() - .build() - } - - val createConfigMapVolume = for { - fLocation <- fileLocation - krb5ConfName <- newKrb5ConfName - } yield { - val krb5File = new File(fLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .build() - } - - // Breaking up Volume creation for clarity - val configMapVolume = preConfigMapVolume.orElse(createConfigMapVolume) - if (configMapVolume.isEmpty) { - logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + - "Make sure that you have the krb5.conf locally on the Driver and Executor images") - } - - val kerberizedPodWithDTSecret = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(dtSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() - - // Optionally add the krb5.conf ConfigMap - val kerberizedPod = configMapVolume.map { cmVolume => - new PodBuilder(kerberizedPodWithDTSecret) - .editSpec() - .addNewVolumeLike(cmVolume) - .endVolume() - .endSpec() - .build() - }.getOrElse(kerberizedPodWithDTSecret) - - val kerberizedContainerWithMounts = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") - .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(userName) - .endEnv() - .build() - - // Optionally add the krb5.conf Volume Mount - val kerberizedContainer = - if (configMapVolume.isDefined) { - new ContainerBuilder(kerberizedContainerWithMounts) - .addNewVolumeMount() - .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") - .withSubPath("krb5.conf") - .endVolumeMount() - .build() - } else { - kerberizedContainerWithMounts - } - - SparkPod(kerberizedPod, kerberizedContainer) - } - - /** - * setting ENV_SPARK_USER when HADOOP_FILES are detected - * - * @param sparkUserName Name of the SPARK_USER - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() - .build() - SparkPod(pod.pod, envModifiedContainer) - } - - /** - * Grabbing files in the HADOOP_CONF_DIR - * - * @param path location of HADOOP_CONF_DIR - * @return a list of File object - */ - def getHadoopConfFiles(path: String): Seq[File] = { - val dir = new File(path) - if (dir.isDirectory) { - dir.listFiles.filter(_.isFile).toSeq - } else { - Seq.empty[File] - } - } - - /** - * Bootstraping the container with ConfigMaps that store - * Hadoop configuration files - * - * @param hadoopConfDir directory location of HADOOP_CONF_DIR env - * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR - * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapHadoopConfDir( - hadoopConfDir: Option[String], - newHadoopConfigMapName: Option[String], - existingHadoopConfigMapName: Option[String], - pod: SparkPod): SparkPod = { - val preConfigMapVolume = existingHadoopConfigMapName.map { hConf => - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConf) - .endConfigMap() - .build() } - - val createConfigMapVolume = for { - dirLocation <- hadoopConfDir - hConfName <- newHadoopConfigMapName - } yield { - val hadoopConfigFiles = getHadoopConfFiles(dirLocation) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() - } - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConfName) - .withItems(keyPaths.asJava) - .endConfigMap() - .build() - } - - // Breaking up Volume Creation for clarity - val configMapVolume = preConfigMapVolume.getOrElse(createConfigMapVolume.get) - - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolumeLike(configMapVolume) - .endVolume() - .endSpec() - .build() - - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) - } - - /** - * Builds ConfigMap given the file location of the - * krb5.conf file - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ - def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String): ConfigMap = { - val file = new File(fileLocation) - new ConfigMapBuilder() - .withNewMetadata() - .withName(configMapName) - .endMetadata() - .addToData(Map(file.toPath.getFileName.toString -> - Files.toString(file, StandardCharsets.UTF_8)).asJava) - .build() - } - - /** - * Builds ConfigMap given the ConfigMap name - * and a list of Hadoop Conf files - * - * @param hadoopConfigMapName name of hadoopConfigMap - * @param hadoopConfFiles list of hadoopFiles - * @return a ConfigMap - */ - def buildHadoopConfigMap( - hadoopConfigMapName: String, - hadoopConfFiles: Seq[File]): ConfigMap = { - new ConfigMapBuilder() - .withNewMetadata() - .withName(hadoopConfigMapName) - .endMetadata() - .addToData(hadoopConfFiles.map { file => - (file.toPath.getFileName.toString, - Files.toString(file, StandardCharsets.UTF_8)) - }.toMap.asJava) - .build() - } - -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala deleted file mode 100644 index 0022d8f242a72..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.SecretBuilder -import org.apache.commons.codec.binary.Base64 - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager - -/** - * This logic does all the heavy lifting for Delegation Token creation. This step - * assumes that the job user has either specified a principal and keytab or ran - * $kinit before running spark-submit. By running UGI.getCurrentUser we are able - * to obtain the current user, either signed in via $kinit or keytab. With the - * Job User principal you then retrieve the delegation token from the NameNode - * and store values in DelegationToken. Lastly, the class puts the data into - * a secret. All this is defined in a KerberosConfigSpec. - */ -private[spark] object HadoopKerberosLogin { - def buildSpec( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix: String, - tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { - // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal - // The login happens in the SparkSubmit so login logic is not necessary to include - val jobUserUGI = tokenManager.getCurrentUser - val originalCredentials = jobUserUGI.getCredentials - tokenManager.obtainDelegationTokens(originalCredentials) - - val tokenData = SparkHadoopUtil.get.serialize(originalCredentials) - - val initialTokenDataKeyName = KERBEROS_SECRET_KEY - val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(newSecretName) - .endMetadata() - .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) - .build() - KerberosConfigSpec( - dtSecret = Some(secretDT), - dtSecretName = newSecretName, - dtSecretItemKey = initialTokenDataKeyName, - jobUserName = jobUserUGI.getShortUserName) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 4b58f8ba3c9bd..c79290a1ca68e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -18,12 +18,9 @@ package org.apache.spark.deploy.k8s.submit import java.io.StringWriter import java.util.{Collections, Locale, Properties, UUID} -import java.util.{Collections, UUID} -import java.util.Properties import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.hadoop.security.UserGroupInformation import scala.collection.mutable import scala.util.control.NonFatal @@ -47,8 +44,7 @@ private[spark] case class ClientArguments( mainAppResource: Option[MainAppResource], mainClass: String, driverArgs: Array[String], - maybePyFiles: Option[String], - hadoopConfigDir: Option[String]) + maybePyFiles: Option[String]) private[spark] object ClientArguments { @@ -82,8 +78,7 @@ private[spark] object ClientArguments { mainAppResource, mainClass.get, driverArgs.toArray, - maybePyFiles, - sys.env.get(ENV_HADOOP_CONF_DIR)) + maybePyFiles) } } @@ -226,8 +221,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { clientArguments.mainAppResource, clientArguments.mainClass, clientArguments.driverArgs, - clientArguments.maybePyFiles, - clientArguments.hadoopConfigDir) + clientArguments.maybePyFiles) val namespace = kubernetesConf.namespace() // The master URL has been checked for validity already in SparkSubmit. // We just need to get rid of the "k8s://" prefix here. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index b0e70a3cd2908..33beab49d7ca4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -21,9 +21,9 @@ import java.io.File import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{Config, KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.features._ -import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} +import org.apache.spark.deploy.k8s.features.bindings._ import org.apache.spark.util.Utils private[spark] class KubernetesDriverBuilder( @@ -49,22 +49,15 @@ private[spark] class KubernetesDriverBuilder( provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - providePythonStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => PythonDriverFeatureStep) = - new PythonDriverFeatureStep(_), - provideRStep: ( - KubernetesConf[KubernetesDriverSpecificConf] + providePythonStep: (KubernetesConf[KubernetesDriverSpecificConf] + => PythonDriverFeatureStep) = + new PythonDriverFeatureStep(_), + provideRStep: (KubernetesConf[KubernetesDriverSpecificConf] => RDriverFeatureStep) = new RDriverFeatureStep(_), - provideJavaStep: ( - KubernetesConf[KubernetesDriverSpecificConf] + provideJavaStep: (KubernetesConf[KubernetesDriverSpecificConf] => JavaDriverFeatureStep) = new JavaDriverFeatureStep(_), - provideHadoopGlobalStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => KerberosConfDriverFeatureStep) = - new KerberosConfDriverFeatureStep(_), providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => PodTemplateConfigMapStep) = new PodTemplateConfigMapStep(_), @@ -103,10 +96,6 @@ private[spark] class KubernetesDriverBuilder( provideRStep(kubernetesConf)} .getOrElse(provideJavaStep(kubernetesConf)) - val maybeHadoopConfigStep = - kubernetesConf.hadoopConfSpec.map { _ => - provideHadoopGlobalStep(kubernetesConf)} - val localFiles = KubernetesUtils.submitterLocalFiles(kubernetesConf.sparkFiles) .map(new File(_)) require(localFiles.forall(_.isFile), s"All submitted local files must be present and not" + @@ -121,15 +110,12 @@ private[spark] class KubernetesDriverBuilder( Seq(provideMountLocalFilesStep(kubernetesConf)) } else Nil - val allFeatures: Seq[KubernetesFeatureConfigStep] = - (baseFeatures :+ bindingsStep) ++ - secretFeature ++ envSecretFeature ++ volumesFeature ++ - maybeHadoopConfigStep.toSeq ++ podTemplateFeature ++ - providedLocalFilesFeature + val allFeatures = (baseFeatures :+ bindingsStep) ++ secretFeature ++ + envSecretFeature ++ volumesFeature ++ providedLocalFilesFeature ++ podTemplateFeature var spec = KubernetesDriverSpec( provideInitialPod(), - driverKubernetesResources = Seq.empty, + Seq.empty, kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { val configuredPod = feature.configurePod(spec.pod) @@ -153,10 +139,7 @@ private[spark] object KubernetesDriverBuilder { conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE) .map(new File(_)) .map(file => new KubernetesDriverBuilder(provideInitialPod = () => - KubernetesUtils.loadPodFromTemplate( - kubernetesClient, - file, - conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME)) + KubernetesUtils.loadPodFromTemplate(kubernetesClient, file) )) .getOrElse(new KubernetesDriverBuilder()) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index a2430c05e2568..1889fe5eb3e9b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -18,10 +18,13 @@ package org.apache.spark.deploy.k8s.submit import java.util.concurrent.{CountDownLatch, TimeUnit} -import io.fabric8.kubernetes.api.model.Pod +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, Time} import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index 4bb9586d4c8b2..ed060ef059be0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -122,8 +122,8 @@ private[spark] class ExecutorPodsLifecycleManager( execId: Long, schedulerBackend: KubernetesClusterSchedulerBackend, execIdsRemovedInRound: mutable.Set[Long]): Unit = { - removeExecutorFromSpark(schedulerBackend, podState, execId) removeExecutorFromK8s(podState.pod) + removeExecutorFromSpark(schedulerBackend, podState, execId) execIdsRemovedInRound += execId } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index ce10f766334ff..547ee3fa83854 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -72,8 +72,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit if (sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) { KubernetesUtils.loadPodFromTemplate( kubernetesClient, - new File(sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get), - sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) + new File(sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get)) } val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 992fbec2f2cf0..d515dc56c4567 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -22,7 +22,6 @@ import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep} @@ -45,26 +44,10 @@ private[spark] class KubernetesExecutorBuilder( provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideHadoopConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopConfExecutorFeatureStep) = - new HadoopConfExecutorFeatureStep(_), - provideKerberosConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => KerberosConfExecutorFeatureStep) = - new KerberosConfExecutorFeatureStep(_), - provideHadoopSparkUserStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopSparkUserExecutorFeatureStep) = - new HadoopSparkUserExecutorFeatureStep(_), provideInitialPod: () => SparkPod = SparkPod.initialPod) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) - val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { @@ -80,24 +63,8 @@ private[spark] class KubernetesExecutorBuilder( Seq(provideMountLocalFilesStep(kubernetesConf)) } else Nil - val maybeHadoopConfFeatureSteps = maybeHadoopConfigMap.map { _ => - val maybeKerberosStep = - if (maybeDTSecretName.isDefined && maybeDTDataItem.isDefined) { - provideKerberosConfStep(kubernetesConf) - } else { - provideHadoopSparkUserStep(kubernetesConf) - } - Seq(provideHadoopConfStep(kubernetesConf)) :+ - maybeKerberosStep - }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) - - val allFeatures: Seq[KubernetesFeatureConfigStep] = - baseFeatures ++ - secretFeature ++ - secretEnvFeature ++ - volumesFeature ++ - maybeHadoopConfFeatureSteps ++ - localFilesFeature + val allFeatures = baseFeatures ++ + secretFeature ++ secretEnvFeature ++ volumesFeature ++ localFilesFeature var executorPod = provideInitialPod() for (feature <- allFeatures) { @@ -112,10 +79,7 @@ private[spark] object KubernetesExecutorBuilder { conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) .map(new File(_)) .map(file => new KubernetesExecutorBuilder(provideInitialPod = () => - KubernetesUtils.loadPodFromTemplate( - kubernetesClient, - file, - conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) + KubernetesUtils.loadPodFromTemplate(kubernetesClient, file) )) .getOrElse(new KubernetesExecutorBuilder()) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index bb2b94f9976e2..e3c19cdb81567 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -59,8 +59,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + maybePyFiles = None) assert(conf.appId === APP_ID) assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) @@ -82,8 +81,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppJar, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + maybePyFiles = None) assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") .split(",") === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) @@ -95,8 +93,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + maybePyFiles = None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) @@ -117,8 +114,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - Some(inputPyFiles.mkString(",")), - hadoopConfDir = None) + Some(inputPyFiles.mkString(","))) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) @@ -140,8 +136,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + maybePyFiles = None) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) @@ -163,8 +158,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + None) assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } @@ -195,8 +189,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource = None, MAIN_CLASS, APP_ARGS, - maybePyFiles = None, - hadoopConfDir = None) + maybePyFiles = None) assert(conf.roleLabels === Map( SPARK_APP_ID_LABEL -> APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala deleted file mode 100644 index 7c231586af935..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.k8s - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, PodBuilder} - -import org.apache.spark.SparkFunSuite - -class KubernetesUtilsSuite extends SparkFunSuite { - private val HOST = "test-host" - private val POD = new PodBuilder() - .withNewSpec() - .withHostname(HOST) - .withContainers( - new ContainerBuilder().withName("first").build(), - new ContainerBuilder().withName("second").build()) - .endSpec() - .build() - - test("Selects the given container as spark container.") { - val sparkPod = KubernetesUtils.selectSparkContainer(POD, Some("second")) - assert(sparkPod.pod.getSpec.getHostname == HOST) - assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("first")) - assert(sparkPod.container.getName == "second") - } - - test("Selects the first container if no container name is given.") { - val sparkPod = KubernetesUtils.selectSparkContainer(POD, Option.empty) - assert(sparkPod.pod.getSpec.getHostname == HOST) - assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("second")) - assert(sparkPod.container.getName == "first") - } - - test("Falls back to the first container if given container name does not exist.") { - val sparkPod = KubernetesUtils.selectSparkContainer(POD, Some("does-not-exist")) - assert(sparkPod.pod.getSpec.getHostname == HOST) - assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("second")) - assert(sparkPod.container.getName == "first") - } - - test("constructs spark pod correctly with pod template with no containers") { - val noContainersPod = new PodBuilder(POD).editSpec().withContainers().endSpec().build() - val sparkPod = KubernetesUtils.selectSparkContainer(noContainersPod, Some("does-not-exist")) - assert(sparkPod.pod.getSpec.getHostname == HOST) - assert(sparkPod.container.getName == null) - val sparkPodWithNoContainerName = - KubernetesUtils.selectSparkContainer(noContainersPod, Option.empty) - assert(sparkPodWithNoContainerName.pod.getSpec.getHostname == HOST) - assert(sparkPodWithNoContainerName.container.getName == null) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 24ba1d600c23d..13e4dd7a088a1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -78,8 +78,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, DRIVER_ENVS, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -141,7 +140,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { .set(CONTAINER_IMAGE, "spark-driver:latest") val pythonSparkConf = new SparkConf() .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") - .set(CONTAINER_IMAGE, "spark-driver-py:latest") + .set(CONTAINER_IMAGE, "spark-driver:latest") val javaKubernetesConf = KubernetesConf( javaSparkConf, KubernetesDriverSpecificConf( @@ -158,9 +157,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, DRIVER_ENVS, Nil, - Seq.empty[String], - hadoopConfSpec = None) - + Seq.empty[String]) val pythonKubernetesConf = KubernetesConf( pythonSparkConf, KubernetesDriverSpecificConf( @@ -177,15 +174,12 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, DRIVER_ENVS, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() val configuredJavaPod = javaFeatureStep.configurePod(basePod) val configuredPythonPod = pythonFeatureStep.configurePod(basePod) - assert(configuredJavaPod.container.getImage === "spark-driver:latest") - assert(configuredPythonPod.container.getImage === "spark-driver-py:latest") } test("Additional system properties resolve jars and set cluster-mode confs.") { @@ -208,8 +202,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { Map.empty, DRIVER_ENVS, Nil, - allFiles, - hadoopConfSpec = None) + allFiles) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 995ccc242f434..217e7f7c0417c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -92,8 +92,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. @@ -134,8 +133,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } @@ -157,8 +155,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Map("qux" -> "quux"), Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, @@ -186,8 +183,7 @@ class BasicExecutorFeatureStepSuite Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala index 263b4e81dcf31..402a54db4ce01 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala @@ -63,8 +63,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -97,8 +96,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) @@ -138,8 +136,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index ef33a1b1edce2..f54f7215fa68c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -69,8 +69,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -103,8 +102,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" @@ -127,8 +125,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None)) + Seq.empty[String])) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -160,8 +157,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None), + Seq.empty[String]), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -190,8 +186,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None), + Seq.empty[String]), clock) fail("The driver bind address should not be allowed.") } catch { @@ -218,8 +213,7 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None), + Seq.empty[String]), clock) fail("The driver host address should not be allowed.") } catch { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala index 921a0d569fef6..349f911ce448f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala @@ -47,8 +47,7 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ envVarsToKeys, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val step = new EnvSecretsFeatureStep(kubernetesConf) val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala index 4dfa9363453ed..51767fa65a0f8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala @@ -49,8 +49,7 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) } test("Resolve to default local dir if neither env nor configuration are set") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountLocalFilesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountLocalFilesFeatureStepSuite.scala index c35fcdea869cb..99149e257248b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountLocalFilesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountLocalFilesFeatureStepSuite.scala @@ -18,15 +18,14 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import scala.collection.JavaConverters._ - import com.google.common.base.Charsets import com.google.common.io.{BaseEncoding, Files} import io.fabric8.kubernetes.api.model.Secret import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesExecutorSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.Utils @@ -66,8 +65,7 @@ class MountLocalFilesFeatureStepSuite extends SparkFunSuite with BeforeAndAfter Map.empty, Map.empty, Seq.empty, - sparkFiles, - None) + sparkFiles) stepUnderTest = new MountLocalFilesFeatureStep(kubernetesConf) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index e0394558bf167..409b84ac88e65 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -45,8 +45,7 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) val step = new MountSecretsFeatureStep(kubernetesConf) val driverPodWithSecretsMounted = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index 00c3b13057ede..d2e1af887811e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -37,8 +37,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, roleVolumes = Nil, - sparkFiles = Nil, - hadoopConfSpec = None) + sparkFiles = Nil) test("Mounts hostPath volumes") { val volumeConf = KubernetesVolumeSpec( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala index 95927ea39f809..61ab844777685 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala @@ -49,8 +49,7 @@ class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - Option.empty) + Seq.empty[String]) templateFile = Files.createTempFile("pod-template", "yml").toFile templateFile.deleteOnExit() Mockito.doReturn(Option(templateFile.getAbsolutePath)).when(sparkConf) @@ -76,7 +75,7 @@ class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { assert(configuredPod.container.getVolumeMounts.size() === 1) val volumeMount = configuredPod.container.getVolumeMounts.get(0) - assert(volumeMount.getMountPath === Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) + assert(volumeMount.getMountPath === Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTHPATH) assert(volumeMount.getName === Constants.POD_TEMPLATE_VOLUME) val resources = step.getAdditionalKubernetesResources() @@ -92,7 +91,7 @@ class PodTemplateConfigMapStepSuite extends SparkFunSuite with BeforeAndAfter { assert(systemProperties.size === 1) assert(systemProperties.contains(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key)) assert(systemProperties.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key).get === - (Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + + (Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTHPATH + "/" + Constants.EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala index 912f56812806c..60836b8c4239c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -44,8 +44,7 @@ class JavaDriverFeatureStepSuite extends SparkFunSuite { roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, roleVolumes = Nil, - sparkFiles = Seq.empty[String], - hadoopConfSpec = None) + sparkFiles = Seq.empty[String]) val step = new JavaDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 289f3b45fcb37..4bad952bd05c2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -54,8 +54,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, roleVolumes = Nil, - sparkFiles = Seq.empty[String], - hadoopConfSpec = None) + sparkFiles = Seq.empty[String]) val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod @@ -93,8 +92,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, roleVolumes = Nil, - sparkFiles = Seq.empty[String], - hadoopConfSpec = None) + sparkFiles = Seq.empty[String]) val step = new PythonDriverFeatureStep(kubernetesConf) val driverContainerwithPySpark = step.configurePod(baseDriverPod).container val args = driverContainerwithPySpark diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala index 5473ae77ba397..29f63cc8aa904 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala @@ -48,8 +48,7 @@ class RDriverFeatureStepSuite extends SparkFunSuite { roleSecretEnvNamesToKeyRefs = Map.empty, roleEnvs = Map.empty, roleVolumes = Seq.empty, - sparkFiles = Seq.empty[String], - hadoopConfSpec = None) + sparkFiles = Seq.empty[String]) val step = new RDriverFeatureStep(kubernetesConf) val driverContainerwithR = step.configurePod(baseDriverPod).container diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 6ddd2292c7eaa..75215bfa83e52 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -143,8 +143,7 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index c768d61e8da46..5ce80ff8be577 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config.{CONTAINER_IMAGE, KUBERNETES_DRIVER_PODTEMPLATE_FILE, KUBERNETES_EXECUTOR_PODTEMPLATE_FILE} import org.apache.spark.deploy.k8s.features._ -import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} +import org.apache.spark.deploy.k8s.features.bindings._ import org.apache.spark.util.Utils class KubernetesDriverBuilderSuite extends SparkFunSuite { @@ -40,10 +40,9 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val SECRETS_STEP_TYPE = "mount-secrets" private val MOUNT_LOCAL_FILES_STEP_TYPE = "mount-local-files" private val JAVA_STEP_TYPE = "java-bindings" - private val R_STEP_TYPE = "r-bindings" private val PYSPARK_STEP_TYPE = "pyspark-bindings" + private val R_STEP_TYPE = "r-bindings" private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val HADOOP_GLOBAL_STEP_TYPE = "hadoop-global" private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" private val TEMPLATE_VOLUME_STEP_TYPE = "template-volume" @@ -77,9 +76,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_GLOBAL_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) @@ -100,7 +96,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => pythonStep, _ => rStep, _ => javaStep, - _ => hadoopGlobalStep, _ => templateVolumeStep) test("Apply fundamental steps all the time.") { @@ -120,8 +115,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -148,8 +142,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map("EnvName" -> "SecretName:secretKey"), Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -178,8 +171,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -206,8 +198,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -243,8 +234,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Seq.empty, - allFiles, - hadoopConfSpec = None) + allFiles) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -277,8 +267,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, volumeSpec :: Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -309,8 +298,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -338,8 +326,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - hadoopConfSpec = None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -349,72 +336,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { R_STEP_TYPE) } - test("Apply HadoopSteps if HADOOP_CONF_DIR is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - None, - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - None, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String], - hadoopConfSpec = Some( - HadoopConfSpec( - Some("/var/hadoop-conf"), - None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - JAVA_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - - test("Apply HadoopSteps if HADOOP_CONF ConfigMap is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - None, - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - None, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String], - hadoopConfSpec = Some( - HadoopConfSpec( - None, - Some("pre-defined-configMapName")))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - JAVA_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) - : Unit = { + : Unit = { assert(resolvedSpec.systemProperties.size === stepTypes.size) stepTypes.foreach { stepType => assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) @@ -451,8 +374,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - Option.empty) + Seq.empty[String]) val driverSpec = KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) @@ -485,8 +407,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - Option.empty) + Seq.empty[String]) val exception = intercept[SparkException] { KubernetesDriverBuilder .apply(kubernetesClient, sparkConf) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 2e883623a4b1c..c6b667ed85e8c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -82,7 +82,7 @@ object ExecutorLifecycleTestUtils { def deletedExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewMetadata() - .withDeletionTimestamp("523012521") + .withNewDeletionTimestamp("523012521") .endMetadata() .build() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 2f984e5d89808..c62555b001861 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -162,7 +162,6 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { } else { val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] val executorSpecificConf = k8sConf.roleSpecificConf - // TODO: HADOOP_CONF_DIR val expectedK8sConf = KubernetesConf.createExecutorConf( conf, executorSpecificConf.executorId, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index 477353f21cf09..a552a93ccfaa3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -22,7 +22,6 @@ import org.mockito.Mockito.{mock, never, verify} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features._ import org.apache.spark.deploy.k8s.submit.PodBuilderSuiteUtils @@ -32,9 +31,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val MOUNT_LOCAL_FILES_STEP_TYPE = "mount-local-files" private val ENV_SECRETS_STEP_TYPE = "env-secrets" private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val HADOOP_CONF_STEP_TYPE = "hadoop-conf-step" - private val HADOOP_SPARK_USER_STEP_TYPE = "hadoop-spark-user" - private val KERBEROS_CONF_STEP_TYPE = "kerberos-step" private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( @@ -47,12 +43,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_CONF_STEP_TYPE, classOf[HadoopConfExecutorFeatureStep]) - private val hadoopSparkUser = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_SPARK_USER_STEP_TYPE, classOf[HadoopSparkUserExecutorFeatureStep]) - private val kerberosConf = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfExecutorFeatureStep]) private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) @@ -62,10 +52,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { _ => mountLocalFilesStep, _ => envSecretsStep, _ => localDirsStep, - _ => mountVolumesStep, - _ => hadoopConfStep, - _ => kerberosConf, - _ => hadoopSparkUser) + _ => mountVolumesStep) test("Basic steps are consistently applied.") { val conf = KubernetesConf( @@ -81,8 +68,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) } @@ -101,8 +87,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map("secret-name" -> "secret-key"), Map.empty, Nil, - Seq.empty[String], - None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -125,8 +110,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Seq.empty, - Seq.empty[String], - None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -153,8 +137,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, volumeSpec :: Nil, - Seq.empty[String], - None) + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -163,66 +146,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { MOUNT_VOLUMES_STEP_TYPE) } - test("Apply basicHadoop step if HADOOP_CONF_DIR is defined") { - // HADOOP_DELEGATION_TOKEN - val HADOOP_CREDS_PREFIX = "spark.security.credentials." - val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" - val conf = KubernetesConf( - new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(HADOOPFS_PROVIDER, "true"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - None, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String], - Some(HadoopConfSpec(Some("/var/hadoop-conf"), None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - HADOOP_SPARK_USER_STEP_TYPE) - } - - test("Apply kerberos step if DT secrets created") { - val conf = KubernetesConf( - new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(KERBEROS_DT_SECRET_NAME, "dt-secret") - .set(KERBEROS_DT_SECRET_KEY, "dt-key"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - None, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String], - Some(HadoopConfSpec(None, Some("pre-defined-onfigMapName")))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - KERBEROS_CONF_STEP_TYPE) - } - private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) stepTypes.foreach { stepType => @@ -259,8 +182,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Nil, - Seq.empty[String], - Option.empty) + Seq.empty[String]) val sparkPod = KubernetesExecutorBuilder .apply(kubernetesClient, sparkConf) .buildFromFeatures(kubernetesConf) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index a5850dad21d06..23865e21a78e7 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -18,9 +18,7 @@ FROM openjdk:8-alpine ARG spark_jars=jars -ARG example_jars=examples/jars ARG img_path=kubernetes/dockerfiles -ARG k8s_tests=kubernetes/tests # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. @@ -31,7 +29,7 @@ ARG k8s_tests=kubernetes/tests RUN set -ex && \ apk upgrade --no-cache && \ - apk add --no-cache bash tini krb5 krb5-libs && \ + apk add --no-cache bash tini && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/work-dir && \ touch /opt/spark/RELEASE && \ diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 17a88320afaba..f45321a4dab7f 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -86,10 +86,6 @@ elif [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "3" ]; then export PYSPARK_DRIVER_PYTHON="python3" fi -if ! [ -z ${HADOOP_CONF_DIR+x} ]; then - SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; -fi - case "$SPARK_K8S_CMD" in driver) CMD=( diff --git a/resource-managers/kubernetes/integration-tests/README.md b/resource-managers/kubernetes/integration-tests/README.md index 64f8e77597eba..b3863e6b7d1af 100644 --- a/resource-managers/kubernetes/integration-tests/README.md +++ b/resource-managers/kubernetes/integration-tests/README.md @@ -8,59 +8,26 @@ title: Spark on Kubernetes Integration Tests Note that the integration test framework is currently being heavily revised and is subject to change. Note that currently the integration tests only run with Java 8. -The simplest way to run the integration tests is to install and run Minikube, then run the following from this -directory: +The simplest way to run the integration tests is to install and run Minikube, then run the following: dev/dev-run-integration-tests.sh The minimum tested version of Minikube is 0.23.0. The kube-dns addon must be enabled. Minikube should -run with a minimum of 4 CPUs and 6G of memory: +run with a minimum of 3 CPUs and 4G of memory: - minikube start --cpus 4 --memory 6144 + minikube start --cpus 3 --memory 4096 You can download Minikube [here](https://github.com/kubernetes/minikube/releases). # Integration test customization -Configuration of the integration test runtime is done through passing different arguments to the test script. -The main useful options are outlined below. - -## Using a different backend - -The integration test backend i.e. the K8S cluster used for testing is controlled by the `--deploy-mode` option. By -default this is set to `minikube`, the available backends are their perequisites are as follows. - -### `minikube` - -Uses the local `minikube` cluster, this requires that `minikube` 0.23.0 or greater be installed and that it be allocated -at least 4 CPUs and 6GB memory (some users have reported success with as few as 3 CPUs and 4GB memory). The tests will -check if `minikube` is started and abort early if it isn't currently running. - -### `docker-for-desktop` - -Since July 2018 Docker for Desktop provide an optional Kubernetes cluster that can be enabled as described in this -[blog post](https://blog.docker.com/2018/07/kubernetes-is-now-available-in-docker-desktop-stable-channel/). Assuming -this is enabled using this backend will auto-configure itself from the `docker-for-desktop` context that Docker creates -in your `~/.kube/config` file. If your config file is in a different location you should set the `KUBECONFIG` -environment variable appropriately. - -### `cloud` - -These cloud backend configures the tests to use an arbitrary Kubernetes cluster running in the cloud or otherwise. - -The `cloud` backend auto-configures the cluster to use from your K8S config file, this is assumed to be `~/.kube/config` -unless the `KUBECONFIG` environment variable is set to override this location. By default this will use whatever your -current context is in the config file, to use an alternative context from your config file you can specify the -`--context ` flag with the desired context. - -You can optionally use a different K8S master URL than the one your K8S config file specified, this should be supplied -via the `--spark-master ` flag. +Configuration of the integration test runtime is done through passing different arguments to the test script. The main useful options are outlined below. ## Re-using Docker Images By default, the test framework will build new Docker images on every test execution. A unique image tag is generated, -and it is written to file at `target/imageTag.txt`. To reuse the images built in a previous run, or to use a Docker -image tag that you have built by other means already, pass the tag to the test script: +and it is written to file at `target/imageTag.txt`. To reuse the images built in a previous run, or to use a Docker image tag +that you have built by other means already, pass the tag to the test script: dev/dev-run-integration-tests.sh --image-tag @@ -70,140 +37,16 @@ where if you still want to use images that were built before by the test framewo ## Spark Distribution Under Test -The Spark code to test is handed to the integration test system via a tarball. Here is the option that is used to -specify the tarball: +The Spark code to test is handed to the integration test system via a tarball. Here is the option that is used to specify the tarball: * `--spark-tgz ` - set `` to point to a tarball containing the Spark distribution to test. -This Tarball should be created by first running `dev/make-distribution.sh` passing the `--tgz` flag and `-Pkubernetes` -as one of the options to ensure that Kubernetes support is included in the distribution. For more details on building a -runnable distribution please see the -[Building Spark](https://spark.apache.org/docs/latest/building-spark.html#building-a-runnable-distribution) -documentation. - -**TODO:** Don't require the packaging of the built Spark artifacts into this tarball, just read them out of the current -tree. +TODO: Don't require the packaging of the built Spark artifacts into this tarball, just read them out of the current tree. ## Customizing the Namespace and Service Account -If no namespace is specified then a temporary namespace will be created and deleted during the test run. Similarly if -no service account is specified then the `default` service account for the namespace will be used. - -Using the `--namespace ` flag sets `` to the namespace in which the tests should be run. If this -is supplied then the tests assume this namespace exists in the K8S cluster and will not attempt to create it. -Additionally this namespace must have an appropriately authorized service account which can be customised via the -`--service-account` flag. - -The `--service-account ` flag sets `` to the name of the Kubernetes service -account to use in the namespace specified by the `--namespace` flag. The service account is expected to have permissions -to get, list, watch, and create pods. For clusters with RBAC turned on, it's important that the right permissions are -granted to the service account in the namespace through an appropriate role and role binding. A reference RBAC -configuration is provided in `dev/spark-rbac.yaml`. - -# Running the Test Directly - -If you prefer to run just the integration tests directly, then you can customise the behaviour via passing system -properties to Maven. For example: - - mvn integration-test -am -pl :spark-kubernetes-integration-tests_2.11 \ - -Pkubernetes -Pkubernetes-integration-tests \ - -Phadoop-2.7 -Dhadoop.version=2.7.3 \ - -Dspark.kubernetes.test.sparkTgz=spark-3.0.0-SNAPSHOT-bin-example.tgz \ - -Dspark.kubernetes.test.imageTag=sometag \ - -Dspark.kubernetes.test.imageRepo=docker.io/somerepo \ - -Dspark.kubernetes.test.namespace=spark-int-tests \ - -Dspark.kubernetes.test.deployMode=docker-for-desktop \ - -Dtest.include.tags=k8s - - -## Available Maven Properties - -The following are the available Maven properties that can be passed. For the most part these correspond to flags passed -to the wrapper scripts and using the wrapper scripts will simply set these appropriately behind the scenes. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    PropertyDescriptionDefault
    spark.kubernetes.test.sparkTgz - A runnable Spark distribution to test. -
    spark.kubernetes.test.unpackSparkDir - The directory where the runnable Spark distribution will be unpacked. - ${project.build.directory}/spark-dist-unpacked
    spark.kubernetes.test.deployMode - The integration test backend to use. Acceptable values are minikube, - docker-for-desktop and cloud. - minikube
    spark.kubernetes.test.kubeConfigContext - When using the cloud backend specifies the context from the users K8S config file that should be used - as the target cluster for integration testing. If not set and using the cloud backend then your - current context will be used. -
    spark.kubernetes.test.master - When using the cloud-url backend must be specified to indicate the K8S master URL to communicate - with. -
    spark.kubernetes.test.imageTag - A specific image tag to use, when set assumes images with those tags are already built and available in the - specified image repository. When set to N/A (the default) fresh images will be built. - N/A -
    spark.kubernetes.test.imageTagFile - A file containing the image tag to use, if no specific image tag is set then fresh images will be built with a - generated tag and that tag written to this file. - ${project.build.directory}/imageTag.txt
    spark.kubernetes.test.imageRepo - The Docker image repository that contains the images to be used if a specific image tag is set or to which the - images will be pushed to if fresh images are being built. - docker.io/kubespark
    spark.kubernetes.test.namespace - A specific Kubernetes namespace to run the tests in. If specified then the tests assume that this namespace - already exists. When not specified a temporary namespace for the tests will be created and deleted as part of the - test run. -
    spark.kubernetes.test.serviceAccountName - A specific Kubernetes service account to use for running the tests. If not specified then the namespaces default - service account will be used and that must have sufficient permissions or the tests will fail. -
    +* `--namespace ` - set `` to the namespace in which the tests should be run. +* `--service-account ` - set `` to the name of the Kubernetes service account to +use in the namespace specified by the `--namespace`. The service account is expected to have permissions to get, list, watch, +and create pods. For clusters with RBAC turned on, it's important that the right permissions are granted to the service account +in the namespace through an appropriate role and role binding. A reference RBAC configuration is provided in `dev/spark-rbac.yaml`. diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index 3c7cc9369047a..b28b8b82ca016 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -26,10 +26,8 @@ IMAGE_TAG="N/A" SPARK_MASTER= NAMESPACE= SERVICE_ACCOUNT= -CONTEXT= INCLUDE_TAGS="k8s" EXCLUDE_TAGS= -SCALA_VERSION="$($TEST_ROOT_DIR/build/mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=scala.binary.version | grep -v '\[' )" # Parse arguments while (( "$#" )); do @@ -62,10 +60,6 @@ while (( "$#" )); do SERVICE_ACCOUNT="$2" shift ;; - --context) - CONTEXT="$2" - shift - ;; --include-tags) INCLUDE_TAGS="k8s,$2" shift @@ -99,11 +93,6 @@ then properties=( ${properties[@]} -Dspark.kubernetes.test.serviceAccountName=$SERVICE_ACCOUNT ) fi -if [ -n $CONTEXT ]; -then - properties=( ${properties[@]} -Dspark.kubernetes.test.kubeConfigContext=$CONTEXT ) -fi - if [ -n $SPARK_MASTER ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.master=$SPARK_MASTER ) @@ -114,4 +103,4 @@ then properties=( ${properties[@]} -Dtest.exclude.tags=$EXCLUDE_TAGS ) fi -$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -Pkubernetes -Pkubernetes-integration-tests ${properties[@]} +$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pkubernetes -Phadoop-2.7 ${properties[@]} diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 07288c97bd527..23453c8957b28 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -29,24 +29,15 @@ 1.3.0 1.4.0 - 4.1.0 + 3.0.0 3.2.2 1.0 kubernetes-integration-tests - - - - ${project.build.directory}/spark-dist-unpacked N/A ${project.build.directory}/imageTag.txt minikube docker.io/kubespark - - - - - @@ -144,7 +135,6 @@ ${spark.kubernetes.test.unpackSparkDir} ${spark.kubernetes.test.imageRepo} ${spark.kubernetes.test.deployMode} - ${spark.kubernetes.test.kubeConfigContext} ${spark.kubernetes.test.master} ${spark.kubernetes.test.namespace} ${spark.kubernetes.test.serviceAccountName} diff --git a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh index a4a9f5b7da131..ccfb8e767c529 100755 --- a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh +++ b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh @@ -71,36 +71,19 @@ if [[ $IMAGE_TAG == "N/A" ]]; then IMAGE_TAG=$(uuidgen); cd $UNPACKED_SPARK_TGZ - - case $DEPLOY_MODE in - cloud) - # Build images - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG build - - # Push images appropriately - if [[ $IMAGE_REPO == gcr.io* ]] ; - then - gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG - else - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push - fi - ;; - - docker-for-desktop) - # Only need to build as this will place it in our local Docker repo which is all - # we need for Docker for Desktop to work so no need to also push - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG build - ;; - - minikube) - # Only need to build and if we do this with the -m option for minikube we will - # build the images directly using the minikube Docker daemon so no need to push - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG build - ;; - *) - echo "Unrecognized deploy mode $DEPLOY_MODE" && exit 1 - ;; - esac + if [[ $DEPLOY_MODE == cloud ]] ; + then + $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG build + if [[ $IMAGE_REPO == gcr.io* ]] ; + then + gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG + else + $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push + fi + else + # -m option for minikube. + $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG build + fi cd - fi diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml index 0282e23a39bd2..b2087b72963ae 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml @@ -17,6 +17,7 @@ apiVersion: v1 Kind: Pod metadata: + name: template-pod labels: template-label-key: executor-template-label-value spec: diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index cb439861da8d0..7e0a5d7fb7b2b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -33,7 +33,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s.integrationtest.TestConfig._ -import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory} import org.apache.spark.internal.Logging import org.apache.spark.launcher.SparkLauncher @@ -79,7 +78,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite System.clearProperty(key) } - val sparkDirProp = System.getProperty(CONFIG_KEY_UNPACK_DIR) + val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir") require(sparkDirProp != null, "Spark home directory must be provided in system properties.") sparkHomeDir = Paths.get(sparkDirProp) require(sparkHomeDir.toFile.isDirectory, @@ -290,21 +289,21 @@ private[spark] class KubernetesSuite extends SparkFunSuite protected def doBasicExecutorPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === image) - assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === baseMemory) } protected def doBasicExecutorPyPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === pyImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } protected def doBasicExecutorRPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === rImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index c0b435efb8c9c..5615d6173eebd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -25,16 +25,15 @@ import scala.collection.mutable import io.fabric8.kubernetes.client.DefaultKubernetesClient import org.scalatest.concurrent.Eventually -import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.internal.Logging private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { - val namespaceOption = Option(System.getProperty(CONFIG_KEY_KUBE_NAMESPACE)) + val namespaceOption = Option(System.getProperty("spark.kubernetes.test.namespace")) val hasUserSpecifiedNamespace = namespaceOption.isDefined val namespace = namespaceOption.getOrElse(UUID.randomUUID().toString.replaceAll("-", "")) val serviceAccountName = - Option(System.getProperty(CONFIG_KEY_KUBE_SVC_ACCOUNT)) + Option(System.getProperty("spark.kubernetes.test.serviceAccountName")) .getOrElse("default") val kubernetesClient = defaultClient.inNamespace(namespace) val clientConfig = kubernetesClient.getConfiguration diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala index e5a847e7210cb..fde4fe055d9e7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala @@ -39,6 +39,7 @@ private[spark] trait PodTemplateSuite { k8sSuite: KubernetesSuite => assert(driverPod.getMetadata.getLabels.get(LABEL_KEY) === "driver-template-label-value") }, executorPodChecker = (executorPod: Pod) => { + assert(executorPod.getMetadata.getName === "template-pod") assert(executorPod.getSpec.getContainers.get(0).getImage === image) assert(executorPod.getSpec.getContainers.get(0).getName === "test-executor-container") assert(executorPod.getMetadata.getLabels.containsKey(LABEL_KEY)) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 7da0127857017..d8f3a6cec05c3 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -28,9 +28,7 @@ object ProcessUtils extends Logging { * executeProcess is used to run a command and return the output if it * completes within timeout seconds. */ - def executeProcess(fullCommand: Array[String], - timeout: Long, - dumpErrors: Boolean = false): Seq[String] = { + def executeProcess(fullCommand: Array[String], timeout: Long): Seq[String] = { val pb = new ProcessBuilder().command(fullCommand: _*) pb.redirectErrorStream(true) val proc = pb.start() @@ -42,9 +40,7 @@ object ProcessUtils extends Logging { }) assert(proc.waitFor(timeout, TimeUnit.SECONDS), s"Timed out while executing ${fullCommand.mkString(" ")}") - assert(proc.exitValue == 0, - s"Failed to execute ${fullCommand.mkString(" ")}" + - s"${if (dumpErrors) "\n" + outputLines.mkString("\n")}") + assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}") outputLines } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala index 363ec0a6016bb..5a49e0779160c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala @@ -21,11 +21,9 @@ import java.io.File import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ - object TestConfig { def getTestImageTag: String = { - val imageTagFileProp = System.getProperty(CONFIG_KEY_IMAGE_TAG_FILE) + val imageTagFileProp = System.getProperty("spark.kubernetes.test.imageTagFile") require(imageTagFileProp != null, "Image tag file must be provided in system properties.") val imageTagFile = new File(imageTagFileProp) require(imageTagFile.isFile, s"No file found for image tag at ${imageTagFile.getAbsolutePath}.") @@ -33,7 +31,7 @@ object TestConfig { } def getTestImageRepo: String = { - val imageRepo = System.getProperty(CONFIG_KEY_IMAGE_REPO) + val imageRepo = System.getProperty("spark.kubernetes.test.imageRepo") require(imageRepo != null, "Image repo must be provided in system properties.") imageRepo } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala index eeae70cd68571..8595d0eab1126 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala @@ -17,17 +17,6 @@ package org.apache.spark.deploy.k8s.integrationtest object TestConstants { - val BACKEND_MINIKUBE = "minikube" - val BACKEND_DOCKER_FOR_DESKTOP = "docker-for-desktop" - val BACKEND_CLOUD = "cloud" - - val CONFIG_KEY_DEPLOY_MODE = "spark.kubernetes.test.deployMode" - val CONFIG_KEY_KUBE_CONFIG_CONTEXT = "spark.kubernetes.test.kubeConfigContext" - val CONFIG_KEY_KUBE_MASTER_URL = "spark.kubernetes.test.master" - val CONFIG_KEY_KUBE_NAMESPACE = "spark.kubernetes.test.namespace" - val CONFIG_KEY_KUBE_SVC_ACCOUNT = "spark.kubernetes.test.serviceAccountName" - val CONFIG_KEY_IMAGE_TAG = "spark.kubernetes.test.imageTagF" - val CONFIG_KEY_IMAGE_TAG_FILE = "spark.kubernetes.test.imageTagFile" - val CONFIG_KEY_IMAGE_REPO = "spark.kubernetes.test.imageRepo" - val CONFIG_KEY_UNPACK_DIR = "spark.kubernetes.test.unpackSparkDir" + val MINIKUBE_TEST_BACKEND = "minikube" + val GCE_TEST_BACKEND = "gce" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index bc80cd79482e2..284712c6d250e 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -19,9 +19,6 @@ package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ -import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend -import org.apache.spark.deploy.k8s.integrationtest.backend.docker.DockerForDesktopBackend import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend private[spark] trait IntegrationTestBackend { @@ -31,16 +28,16 @@ private[spark] trait IntegrationTestBackend { } private[spark] object IntegrationTestBackendFactory { + val deployModeConfigKey = "spark.kubernetes.test.deployMode" + def getTestBackend: IntegrationTestBackend = { - val deployMode = Option(System.getProperty(CONFIG_KEY_DEPLOY_MODE)) - .getOrElse(BACKEND_MINIKUBE) - deployMode match { - case BACKEND_MINIKUBE => MinikubeTestBackend - case BACKEND_CLOUD => new KubeConfigBackend( - System.getProperty(CONFIG_KEY_KUBE_CONFIG_CONTEXT)) - case BACKEND_DOCKER_FOR_DESKTOP => DockerForDesktopBackend - case _ => throw new IllegalArgumentException("Invalid " + - CONFIG_KEY_DEPLOY_MODE + ": " + deployMode) + val deployMode = Option(System.getProperty(deployModeConfigKey)) + .getOrElse("minikube") + if (deployMode == "minikube") { + MinikubeTestBackend + } else { + throw new IllegalArgumentException( + "Invalid " + deployModeConfigKey + ": " + deployMode) } } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala deleted file mode 100644 index 9a4def106e0b0..0000000000000 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.integrationtest.backend.cloud - -import io.fabric8.kubernetes.client.{Config, DefaultKubernetesClient} -import io.fabric8.kubernetes.client.utils.Utils -import org.apache.commons.lang3.StringUtils - -import org.apache.spark.deploy.k8s.integrationtest.TestConstants -import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils.checkAndGetK8sMasterUrl - -private[spark] class KubeConfigBackend(var context: String) - extends IntegrationTestBackend with Logging { - logInfo(s"K8S Integration tests will run against " + - s"${if (context != null) s"context ${context}" else "default context"}" + - s" from users K8S config file") - - private var defaultClient: DefaultKubernetesClient = _ - - override def initialize(): Unit = { - // Auto-configure K8S client from K8S config file - if (Utils.getSystemPropertyOrEnvVar(Config.KUBERNETES_KUBECONFIG_FILE, null: String) == null) { - // Fabric 8 client will automatically assume a default location in this case - logWarning( - s"No explicit KUBECONFIG specified, will assume .kube/config under your home directory") - } - val config = Config.autoConfigure(context) - - // If an explicit master URL was specified then override that detected from the - // K8S config if it is different - var masterUrl = Option(System.getProperty(TestConstants.CONFIG_KEY_KUBE_MASTER_URL)) - .getOrElse(null) - if (StringUtils.isNotBlank(masterUrl)) { - // Clean up master URL which would have been specified in Spark format into a normal - // K8S master URL - masterUrl = checkAndGetK8sMasterUrl(masterUrl).replaceFirst("k8s://", "") - if (!StringUtils.equals(config.getMasterUrl, masterUrl)) { - logInfo(s"Overriding K8S master URL ${config.getMasterUrl} from K8S config file " + - s"with user specified master URL ${masterUrl}") - config.setMasterUrl(masterUrl) - } - } - - defaultClient = new DefaultKubernetesClient(config) - } - - override def cleanUp(): Unit = { - super.cleanUp() - } - - override def getKubernetesClient: DefaultKubernetesClient = { - defaultClient - } -} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index f5866651dc90b..bac0246b7ddc5 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -26,12 +26,12 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future +import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException, TaskState} import org.apache.spark.deploy.mesos.config._ -import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config import org.apache.spark.internal.config.EXECUTOR_HEARTBEAT_INTERVAL import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} @@ -60,6 +60,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler with MesosSchedulerUtils { + private lazy val hadoopDelegationTokenManager: MesosHadoopDelegationTokenManager = + new MesosHadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint) + // Blacklist a slave after this many failures private val MAX_SLAVE_FAILURES = 2 @@ -675,7 +678,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( launcherBackend.close() } - private def stopSchedulerBackend(): Unit = { + private def stopSchedulerBackend() { // Make sure we're not launching tasks during shutdown stateLock.synchronized { if (stopCalled) { @@ -774,10 +777,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } - override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { - Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) - } - private def numExecutors(): Int = { slaves.values.map(_.taskIDs.size).sum } @@ -790,6 +789,14 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( offer.getHostname } } + + override def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { + if (UserGroupInformation.isSecurityEnabled) { + Some(hadoopDelegationTokenManager.getTokens()) + } else { + None + } + } } private class Slave(val hostname: String) { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala new file mode 100644 index 0000000000000..a1bf4f0c048fe --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala @@ -0,0 +1,160 @@ +/* + * 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.scheduler.cluster.mesos + +import java.security.PrivilegedExceptionAction +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.ThreadUtils + + +/** + * The MesosHadoopDelegationTokenManager fetches and updates Hadoop delegation tokens on the behalf + * of the MesosCoarseGrainedSchedulerBackend. It is modeled after the YARN AMCredentialRenewer, + * and similarly will renew the Credentials when 75% of the renewal interval has passed. + * The principal difference is that instead of writing the new credentials to HDFS and + * incrementing the timestamp of the file, the new credentials (called Tokens when they are + * serialized) are broadcast to all running executors. On the executor side, when new Tokens are + * received they overwrite the current credentials. + */ +private[spark] class MesosHadoopDelegationTokenManager( + conf: SparkConf, + hadoopConfig: Configuration, + driverEndpoint: RpcEndpointRef) + extends Logging { + + require(driverEndpoint != null, "DriverEndpoint is not initialized") + + private val credentialRenewerThread: ScheduledExecutorService = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") + + private val tokenManager: HadoopDelegationTokenManager = + new HadoopDelegationTokenManager(conf, hadoopConfig) + + private val principal: String = conf.get(config.PRINCIPAL).orNull + + private var (tokens: Array[Byte], timeOfNextRenewal: Long) = { + try { + val creds = UserGroupInformation.getCurrentUser.getCredentials + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) + logInfo(s"Initialized tokens: ${SparkHadoopUtil.get.dumpTokens(creds)}") + (SparkHadoopUtil.get.serialize(creds), SparkHadoopUtil.nextCredentialRenewalTime(rt, conf)) + } catch { + case e: Exception => + logError(s"Failed to fetch Hadoop delegation tokens $e") + throw e + } + } + + private val keytabFile: Option[String] = conf.get(config.KEYTAB) + + scheduleTokenRenewal() + + private def scheduleTokenRenewal(): Unit = { + if (keytabFile.isDefined) { + require(principal != null, "Principal is required for Keytab-based authentication") + logInfo(s"Using keytab: ${keytabFile.get} and principal $principal") + } else { + logInfo("Using ticket cache for Kerberos authentication, no token renewal.") + return + } + + def scheduleRenewal(runnable: Runnable): Unit = { + val remainingTime = timeOfNextRenewal - System.currentTimeMillis() + if (remainingTime <= 0) { + logInfo("Credentials have expired, creating new ones now.") + runnable.run() + } else { + logInfo(s"Scheduling login from keytab in $remainingTime millis.") + credentialRenewerThread.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS) + } + } + + val credentialRenewerRunnable = + new Runnable { + override def run(): Unit = { + try { + getNewDelegationTokens() + broadcastDelegationTokens(tokens) + } catch { + case e: Exception => + // Log the error and try to write new tokens back in an hour + val delay = TimeUnit.SECONDS.toMillis(conf.get(config.CREDENTIALS_RENEWAL_RETRY_WAIT)) + logWarning( + s"Couldn't broadcast tokens, trying again in ${UIUtils.formatDuration(delay)}", e) + credentialRenewerThread.schedule(this, delay, TimeUnit.MILLISECONDS) + return + } + scheduleRenewal(this) + } + } + scheduleRenewal(credentialRenewerRunnable) + } + + private def getNewDelegationTokens(): Unit = { + logInfo(s"Attempting to login to KDC with principal ${principal}") + // Get new delegation tokens by logging in with a new UGI inspired by AMCredentialRenewer.scala + // Don't protect against keytabFile being empty because it's guarded above. + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabFile.get) + logInfo("Successfully logged into KDC") + val tempCreds = ugi.getCredentials + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val nextRenewalTime = ugi.doAs(new PrivilegedExceptionAction[Long] { + override def run(): Long = { + tokenManager.obtainDelegationTokens(hadoopConf, tempCreds) + } + }) + + val currTime = System.currentTimeMillis() + timeOfNextRenewal = if (nextRenewalTime <= currTime) { + logWarning(s"Next credential renewal time ($nextRenewalTime) is earlier than " + + s"current time ($currTime), which is unexpected, please check your credential renewal " + + "related configurations in the target services.") + currTime + } else { + SparkHadoopUtil.nextCredentialRenewalTime(nextRenewalTime, conf) + } + logInfo(s"Time of next renewal is in ${timeOfNextRenewal - System.currentTimeMillis()} ms") + + // Add the temp credentials back to the original ones. + UserGroupInformation.getCurrentUser.addCredentials(tempCreds) + // update tokens for late or dynamically added executors + tokens = SparkHadoopUtil.get.serialize(tempCreds) + } + + private def broadcastDelegationTokens(tokens: Array[Byte]) = { + logInfo("Sending new tokens to all executors") + driverEndpoint.send(UpdateDelegationTokens(tokens)) + } + + def getTokens(): Array[Byte] = { + tokens + } +} + diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index c1f3211bcab29..8f94e3f731007 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -41,7 +41,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager +import org.apache.spark.deploy.yarn.security.AMCredentialRenewer import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.metrics.MetricsSystem @@ -99,18 +99,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private val tokenManager: Option[YARNHadoopDelegationTokenManager] = { - sparkConf.get(KEYTAB).map { _ => - new YARNHadoopDelegationTokenManager(sparkConf, yarnConf) - } + private val credentialRenewer: Option[AMCredentialRenewer] = sparkConf.get(KEYTAB).map { _ => + new AMCredentialRenewer(sparkConf, yarnConf) } - private val ugi = tokenManager match { - case Some(tm) => + private val ugi = credentialRenewer match { + case Some(cr) => // Set the context class loader so that the token renewer has access to jars distributed // by the user. - Utils.withContextClassLoader(userClassLoader) { - tm.start() + val currentLoader = Thread.currentThread().getContextClassLoader() + Thread.currentThread().setContextClassLoader(userClassLoader) + try { + cr.start() + } finally { + Thread.currentThread().setContextClassLoader(currentLoader) } case _ => @@ -378,7 +380,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends userClassThread.interrupt() } if (!inShutdown) { - tokenManager.foreach(_.stop()) + credentialRenewer.foreach(_.stop()) } } } @@ -438,7 +440,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends securityMgr, localResources) - tokenManager.foreach(_.setDriverRef(driverRef)) + credentialRenewer.foreach(_.setDriverRef(driverRef)) // Initialize the AM endpoint *after* the allocator has been initialized. This ensures // that when the driver sends an initial executor request (e.g. after an AM restart), diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 1bf5ee2084021..220594f606498 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -154,8 +154,6 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { - ResourceRequestHelper.validateResources(sparkConf) - var appId: ApplicationId = null try { launcherBackend.connect() @@ -236,13 +234,6 @@ private[spark] class Client( def createApplicationSubmissionContext( newApp: YarnClientApplication, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val amResources = - if (isClusterMode) { - sparkConf.getAllWithPrefix(config.YARN_DRIVER_RESOURCE_TYPES_PREFIX).toMap - } else { - sparkConf.getAllWithPrefix(config.YARN_AM_RESOURCE_TYPES_PREFIX).toMap - } - logDebug(s"AM resources: $amResources") val appContext = newApp.getApplicationSubmissionContext appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark")) appContext.setQueue(sparkConf.get(QUEUE_NAME)) @@ -265,10 +256,6 @@ private[spark] class Client( val capability = Records.newRecord(classOf[Resource]) capability.setMemory(amMemory + amMemoryOverhead) capability.setVirtualCores(amCores) - if (amResources.nonEmpty) { - ResourceRequestHelper.setResourceRequests(amResources, capability) - } - logDebug(s"Created resource capability for AM request: $capability") sparkConf.get(AM_NODE_LABEL_EXPRESSION) match { case Some(expr) => @@ -286,10 +273,19 @@ private[spark] class Client( sparkConf.get(ROLLED_LOG_INCLUDE_PATTERN).foreach { includePattern => try { val logAggregationContext = Records.newRecord(classOf[LogAggregationContext]) - logAggregationContext.setRolledLogsIncludePattern(includePattern) + + // These two methods were added in Hadoop 2.6.4, so we still need to use reflection to + // avoid compile error when building against Hadoop 2.6.0 ~ 2.6.3. + val setRolledLogsIncludePatternMethod = + logAggregationContext.getClass.getMethod("setRolledLogsIncludePattern", classOf[String]) + setRolledLogsIncludePatternMethod.invoke(logAggregationContext, includePattern) + sparkConf.get(ROLLED_LOG_EXCLUDE_PATTERN).foreach { excludePattern => - logAggregationContext.setRolledLogsExcludePattern(excludePattern) + val setRolledLogsExcludePatternMethod = + logAggregationContext.getClass.getMethod("setRolledLogsExcludePattern", classOf[String]) + setRolledLogsExcludePatternMethod.invoke(logAggregationContext, excludePattern) } + appContext.setLogAggregationContext(logAggregationContext) } catch { case NonFatal(e) => @@ -310,7 +306,7 @@ private[spark] class Client( private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val credentials = UserGroupInformation.getCurrentUser().getCredentials() val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - credentialManager.obtainDelegationTokens(credentials) + credentialManager.obtainDelegationTokens(hadoopConf, credentials) // When using a proxy user, copy the delegation tokens to the user's credentials. Avoid // that for regular users, since in those case the user already has access to the TGT, @@ -1169,7 +1165,7 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.10.8.1-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.7-src.zip") require(py4jFile.exists(), s"$py4jFile not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala deleted file mode 100644 index 012268ea856f5..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.lang.{Long => JLong} -import java.lang.reflect.InvocationTargetException - -import scala.collection.mutable -import scala.util.Try - -import org.apache.hadoop.yarn.api.records.Resource - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.util.{CausedBy, Utils} - -/** - * This helper class uses some of Hadoop 3 methods from the YARN API, - * so we need to use reflection to avoid compile error when building against Hadoop 2.x - */ -private object ResourceRequestHelper extends Logging { - private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r - private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" - - /** - * Validates sparkConf and throws a SparkException if any of standard resources (memory or cores) - * is defined with the property spark.yarn.x.resource.y - * Need to reject all combinations of AM / Driver / Executor and memory / CPU cores resources, as - * Spark has its own names for them (memory, cores), - * but YARN have its names too: (memory, memory-mb, mb) and (cores, vcores, cpu-vcores). - * We need to disable every possible way YARN could receive the resource definitions above. - */ - def validateResources(sparkConf: SparkConf): Unit = { - val resourceDefinitions = Seq[(String, String)]( - (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory"), - (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory"), - (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory"), - (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "mb"), - (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "mb"), - (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb"), - (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory-mb"), - (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory-mb"), - (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb"), - (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cores"), - (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cores"), - (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cores"), - (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "vcores"), - (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "vcores"), - (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores"), - (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cpu-vcores"), - (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cpu-vcores"), - (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cpu-vcores")) - val errorMessage = new mutable.StringBuilder() - - resourceDefinitions.foreach { case (sparkName, resourceRequest) => - if (sparkConf.contains(resourceRequest)) { - errorMessage.append(s"Error: Do not use $resourceRequest, " + - s"please use $sparkName instead!\n") - } - } - - if (errorMessage.nonEmpty) { - throw new SparkException(errorMessage.toString()) - } - } - - /** - * Sets resource amount with the corresponding unit to the passed resource object. - * @param resources resource values to set - * @param resource resource object to update - */ - def setResourceRequests( - resources: Map[String, String], - resource: Resource): Unit = { - require(resource != null, "Resource parameter should not be null!") - - logDebug(s"Custom resources requested: $resources") - if (!isYarnResourceTypesAvailable()) { - if (resources.nonEmpty) { - logWarning("Ignoring custom resource requests because " + - "the version of YARN does not support it!") - } - return - } - - val resInfoClass = Utils.classForName(RESOURCE_INFO_CLASS) - val setResourceInformationMethod = - resource.getClass.getMethod("setResourceInformation", classOf[String], resInfoClass) - resources.foreach { case (name, rawAmount) => - try { - val AMOUNT_AND_UNIT_REGEX(amountPart, unitPart) = rawAmount - val amount = amountPart.toLong - val unit = unitPart match { - case "g" => "G" - case "t" => "T" - case "p" => "P" - case _ => unitPart - } - logDebug(s"Registering resource with name: $name, amount: $amount, unit: $unit") - val resourceInformation = createResourceInformation(name, amount, unit, resInfoClass) - setResourceInformationMethod.invoke( - resource, name, resourceInformation.asInstanceOf[AnyRef]) - } catch { - case _: MatchError => - throw new IllegalArgumentException(s"Resource request for '$name' ('$rawAmount') " + - s"does not match pattern $AMOUNT_AND_UNIT_REGEX.") - case CausedBy(e: IllegalArgumentException) => - throw new IllegalArgumentException(s"Invalid request for $name: ${e.getMessage}") - case e: InvocationTargetException if e.getCause != null => throw e.getCause - } - } - } - - private def createResourceInformation( - resourceName: String, - amount: Long, - unit: String, - resInfoClass: Class[_]): Any = { - val resourceInformation = - if (unit.nonEmpty) { - val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", - classOf[String], classOf[String], JLong.TYPE) - resInfoNewInstanceMethod.invoke(null, resourceName, unit, amount.asInstanceOf[JLong]) - } else { - val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", - classOf[String], JLong.TYPE) - resInfoNewInstanceMethod.invoke(null, resourceName, amount.asInstanceOf[JLong]) - } - resourceInformation - } - - /** - * Checks whether Hadoop 2.x or 3 is used as a dependency. - * In case of Hadoop 3 and later, the ResourceInformation class - * should be available on the classpath. - */ - def isYarnResourceTypesAvailable(): Boolean = { - Try(Utils.classForName(RESOURCE_INFO_CLASS)).isSuccess - } -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index ebdcf45603cea..8a7551de7c088 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -140,18 +140,10 @@ private[yarn] class YarnAllocator( } // Number of cores per executor. protected val executorCores = sparkConf.get(EXECUTOR_CORES) - - private val executorResourceRequests = - sparkConf.getAllWithPrefix(config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX).toMap - - // Resource capability requested for each executor - private[yarn] val resource: Resource = { - val resource = Resource.newInstance( - executorMemory + memoryOverhead + pysparkWorkerMemory, executorCores) - ResourceRequestHelper.setResourceRequests(executorResourceRequests, resource) - logDebug(s"Created resource capability: $resource") - resource - } + // Resource capability requested for each executors + private[yarn] val resource = Resource.newInstance( + executorMemory + memoryOverhead + pysparkWorkerMemory, + executorCores) private val launcherPool = ThreadUtils.newDaemonCachedThreadPool( "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS)) @@ -296,16 +288,9 @@ private[yarn] class YarnAllocator( s"executorsStarting: ${numExecutorsStarting.get}") if (missing > 0) { - if (log.isInfoEnabled()) { - var requestContainerMessage = s"Will request $missing executor container(s), each with " + - s"${resource.getVirtualCores} core(s) and " + - s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)" - if (ResourceRequestHelper.isYarnResourceTypesAvailable() && - executorResourceRequests.nonEmpty) { - requestContainerMessage ++= s" with custom resources: " + resource.toString - } - logInfo(requestContainerMessage) - } + logInfo(s"Will request $missing executor container(s), each with " + + s"${resource.getVirtualCores} core(s) and " + + s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)") // Split the pending container request into three groups: locality matched list, locality // unmatched list and non-locality list. Take the locality matched container request into @@ -471,20 +456,13 @@ private[yarn] class YarnAllocator( // memory, but use the asked vcore count for matching, effectively disabling matching on vcore // count. val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, - resource.getVirtualCores) - - ResourceRequestHelper.setResourceRequests(executorResourceRequests, matchingResource) - - logDebug(s"Calling amClient.getMatchingRequests with parameters: " + - s"priority: ${allocatedContainer.getPriority}, " + - s"location: $location, resource: $matchingResource") + resource.getVirtualCores) val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, matchingResource) // Match the allocation to a request if (!matchingRequests.isEmpty) { val containerRequest = matchingRequests.get(0).iterator.next - logDebug(s"Removing container request via AM client: $containerRequest") amClient.removeContainerRequest(containerRequest) containersToUse += allocatedContainer } else { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index b257d8fdd3b1a..ab8273bd6321d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -325,6 +325,10 @@ package object config { .stringConf .createOptional + private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.yarn.kerberos.relogin.period") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("1m") + // The list of cache-related config entries. This is used by Client and the AM to clean // up the environment so that these settings do not appear on the web UI. private[yarn] val CACHE_CONFIGS = Seq( @@ -341,8 +345,4 @@ package object config { .booleanConf .createWithDefault(false) - private[yarn] val YARN_EXECUTOR_RESOURCE_TYPES_PREFIX = "spark.yarn.executor.resource." - private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." - private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." - } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala new file mode 100644 index 0000000000000..bc8d47dbd54c6 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.yarn.security + +import java.security.PrivilegedExceptionAction +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} +import java.util.concurrent.atomic.AtomicReference + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.ThreadUtils + +/** + * A manager tasked with periodically updating delegation tokens needed by the application. + * + * This manager is meant to make sure long-running apps (such as Spark Streaming apps) can run + * without interruption while accessing secured services. It periodically logs in to the KDC with + * user-provided credentials, and contacts all the configured secure services to obtain delegation + * tokens to be distributed to the rest of the application. + * + * This class will manage the kerberos login, by renewing the TGT when needed. Because the UGI API + * does not expose the TTL of the TGT, a configuration controls how often to check that a relogin is + * necessary. This is done reasonably often since the check is a no-op when the relogin is not yet + * needed. The check period can be overridden in the configuration. + * + * New delegation tokens are created once 75% of the renewal interval of the original tokens has + * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. + * The driver is tasked with distributing the tokens to other processes that might need them. + */ +private[yarn] class AMCredentialRenewer( + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { + + private val principal = sparkConf.get(PRINCIPAL).get + private val keytab = sparkConf.get(KEYTAB).get + private val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) + + private val renewalExecutor: ScheduledExecutorService = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Refresh Thread") + + private val driverRef = new AtomicReference[RpcEndpointRef]() + + private val renewalTask = new Runnable() { + override def run(): Unit = { + updateTokensTask() + } + } + + def setDriverRef(ref: RpcEndpointRef): Unit = { + driverRef.set(ref) + } + + /** + * Start the token renewer. Upon start, the renewer will: + * + * - log in the configured user, and set up a task to keep that user's ticket renewed + * - obtain delegation tokens from all available providers + * - schedule a periodic task to update the tokens when needed. + * + * @return The newly logged in user. + */ + def start(): UserGroupInformation = { + val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() + val ugi = doLogin() + + val tgtRenewalTask = new Runnable() { + override def run(): Unit = { + ugi.checkTGTAndReloginFromKeytab() + } + } + val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) + renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, + TimeUnit.SECONDS) + + val creds = obtainTokensAndScheduleRenewal(ugi) + ugi.addCredentials(creds) + + // Transfer the original user's tokens to the new user, since that's needed to connect to + // YARN. Explicitly avoid overwriting tokens that already exist in the current user's + // credentials, since those were freshly obtained above (see SPARK-23361). + val existing = ugi.getCredentials() + existing.mergeAll(originalCreds) + ugi.addCredentials(existing) + + ugi + } + + def stop(): Unit = { + renewalExecutor.shutdown() + } + + private def scheduleRenewal(delay: Long): Unit = { + val _delay = math.max(0, delay) + logInfo(s"Scheduling login from keytab in ${UIUtils.formatDuration(delay)}.") + renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) + } + + /** + * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself + * to fetch the next set of tokens when needed. + */ + private def updateTokensTask(): Unit = { + try { + val freshUGI = doLogin() + val creds = obtainTokensAndScheduleRenewal(freshUGI) + val tokens = SparkHadoopUtil.get.serialize(creds) + + val driver = driverRef.get() + if (driver != null) { + logInfo("Updating delegation tokens.") + driver.send(UpdateDelegationTokens(tokens)) + } else { + // This shouldn't really happen, since the driver should register way before tokens expire + // (or the AM should time out the application). + logWarning("Delegation tokens close to expiration but no driver has registered yet.") + SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) + } + } catch { + case e: Exception => + val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) + logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + + " If this happens too often tasks will fail.", e) + scheduleRenewal(delay) + } + } + + /** + * Obtain new delegation tokens from the available providers. Schedules a new task to fetch + * new tokens before the new set expires. + * + * @return Credentials containing the new tokens. + */ + private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { + ugi.doAs(new PrivilegedExceptionAction[Credentials]() { + override def run(): Credentials = { + val creds = new Credentials() + val nextRenewal = credentialManager.obtainDelegationTokens(hadoopConf, creds) + + val timeToWait = SparkHadoopUtil.nextCredentialRenewalTime(nextRenewal, sparkConf) - + System.currentTimeMillis() + scheduleRenewal(timeToWait) + creds + } + }) + } + + private def doLogin(): UserGroupInformation = { + logInfo(s"Attempting to login to KDC using principal: $principal") + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + logInfo("Successfully logged into KDC.") + ugi + } + +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala index 2d9a3f0c83fd2..26a2e5d730218 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala @@ -22,13 +22,12 @@ import java.util.ServiceLoader import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil -import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** @@ -37,25 +36,27 @@ import org.apache.spark.util.Utils * in [[HadoopDelegationTokenManager]]. */ private[yarn] class YARNHadoopDelegationTokenManager( - _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { + sparkConf: SparkConf, + hadoopConf: Configuration) extends Logging { - private val credentialProviders = { - ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) - .asScala - .toList - .filter { p => isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap - } + private val delegationTokenManager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, + conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) + + // public for testing + val credentialProviders = getCredentialProviders if (credentialProviders.nonEmpty) { logDebug("Using the following YARN-specific credential providers: " + s"${credentialProviders.keys.mkString(", ")}.") } - override def obtainDelegationTokens(creds: Credentials): Long = { - val superInterval = super.obtainDelegationTokens(creds) + /** + * Writes delegation tokens to creds. Delegation tokens are fetched from all registered + * providers. + * + * @return Time after which the fetched delegation tokens should be renewed. + */ + def obtainDelegationTokens(hadoopConf: Configuration, creds: Credentials): Long = { + val superInterval = delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { @@ -68,13 +69,18 @@ private[yarn] class YARNHadoopDelegationTokenManager( }.foldLeft(superInterval)(math.min) } - // For testing. - override def isProviderLoaded(serviceName: String): Boolean = { - credentialProviders.contains(serviceName) || super.isProviderLoaded(serviceName) - } + private def getCredentialProviders: Map[String, ServiceCredentialProvider] = { + val providers = loadCredentialProviders - override protected def fileSystemsToAccess(): Set[FileSystem] = { - YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, hadoopConf) + providers. + filter { p => delegationTokenManager.isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap } + private def loadCredentialProviders: List[ServiceCredentialProvider] = { + ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) + .asScala + .toList + } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 67c36aac49266..63bea3e7a5003 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -19,14 +19,16 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.atomic.{AtomicBoolean} +import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.Future import scala.util.{Failure, Success} import scala.util.control.NonFatal +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} import org.apache.spark.SparkContext +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ @@ -268,6 +270,7 @@ private[spark] abstract class YarnSchedulerBackend( case u @ UpdateDelegationTokens(tokens) => // Add the tokens to the current user and send a message to the scheduler so that it // notifies all registered executors of the new tokens. + SparkHadoopUtil.get.addDelegationTokens(tokens, sc.conf) driverEndpoint.send(u) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index b3286e8fd824e..26013a109c42b 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -23,7 +23,6 @@ import java.util.Properties import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap => MutableHashMap} -import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -358,35 +357,6 @@ class ClientSuite extends SparkFunSuite with Matchers { sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName))) } - Seq( - "client" -> YARN_AM_RESOURCE_TYPES_PREFIX, - "cluster" -> YARN_DRIVER_RESOURCE_TYPES_PREFIX - ).foreach { case (deployMode, prefix) => - test(s"custom resource request ($deployMode mode)") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) - val resources = Map("fpga" -> 2, "gpu" -> 3) - ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) - - val conf = new SparkConf().set("spark.submit.deployMode", deployMode) - resources.foreach { case (name, v) => - conf.set(prefix + name, v.toString) - } - - val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) - val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) - val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) - - val client = new Client(new ClientArguments(Array()), conf) - client.createApplicationSubmissionContext( - new YarnClientApplication(getNewApplicationResponse, appContext), - containerLaunchContext) - - resources.foreach { case (name, value) => - ResourceRequestTestHelper.getRequestedValue(appContext.getResource, name) should be (value) - } - } - } - private val matching = Seq( ("files URI match test1", "file:///file1", "file:///file2"), ("files URI match test2", "file:///c:file1", "file://c:file2"), @@ -462,4 +432,5 @@ class ClientSuite extends SparkFunSuite with Matchers { populateClasspath(null, new Configuration(), client.sparkConf, env) classpath(env) } + } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala deleted file mode 100644 index 8032213602c95..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import org.apache.hadoop.yarn.api.records.Resource -import org.apache.hadoop.yarn.util.Records -import org.scalatest.Matchers - -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.config.{DRIVER_MEMORY, EXECUTOR_MEMORY} - -class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { - - private val CUSTOM_RES_1 = "custom-resource-type-1" - private val CUSTOM_RES_2 = "custom-resource-type-2" - private val MEMORY = "memory" - private val CORES = "cores" - private val NEW_CONFIG_EXECUTOR_MEMORY = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_EXECUTOR_CORES = YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_AM_MEMORY = YARN_AM_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_AM_CORES = YARN_AM_RESOURCE_TYPES_PREFIX + CORES - private val NEW_CONFIG_DRIVER_MEMORY = YARN_DRIVER_RESOURCE_TYPES_PREFIX + MEMORY - private val NEW_CONFIG_DRIVER_CORES = YARN_DRIVER_RESOURCE_TYPES_PREFIX + CORES - - test("empty SparkConf should be valid") { - val sparkConf = new SparkConf() - ResourceRequestHelper.validateResources(sparkConf) - } - - test("just normal resources are defined") { - val sparkConf = new SparkConf() - sparkConf.set(DRIVER_MEMORY.key, "3G") - sparkConf.set(DRIVER_CORES.key, "4") - sparkConf.set(EXECUTOR_MEMORY.key, "4G") - sparkConf.set(EXECUTOR_CORES.key, "2") - ResourceRequestHelper.validateResources(sparkConf) - } - - Seq( - "value with unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 2, "G")), - "value without unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "")), - "multiple resources" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "m"), - ResourceInformation(CUSTOM_RES_2, 10, "G")) - ).foreach { case (name, resources) => - test(s"valid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) - val resourceDefs = resources.map { r => r.name } - val requests = resources.map { r => (r.name, r.value.toString + r.unit) }.toMap - - ResourceRequestTestHelper.initializeResourceTypes(resourceDefs) - - val resource = createResource() - ResourceRequestHelper.setResourceRequests(requests, resource) - - resources.foreach { r => - val requested = ResourceRequestTestHelper.getResourceInformationByName(resource, r.name) - assert(requested === r) - } - } - } - - Seq( - ("value does not match pattern", CUSTOM_RES_1, "**@#"), - ("only unit defined", CUSTOM_RES_1, "m"), - ("invalid unit", CUSTOM_RES_1, "123ppp") - ).foreach { case (name, key, value) => - test(s"invalid request: $name") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) - ResourceRequestTestHelper.initializeResourceTypes(Seq(key)) - - val resource = createResource() - val thrown = intercept[IllegalArgumentException] { - ResourceRequestHelper.setResourceRequests(Map(key -> value), resource) - } - thrown.getMessage should include (key) - } - } - - Seq( - NEW_CONFIG_EXECUTOR_MEMORY -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb" -> "30G", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb" -> "30G", - NEW_CONFIG_EXECUTOR_CORES -> "5", - YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores" -> "5", - NEW_CONFIG_AM_MEMORY -> "1G", - NEW_CONFIG_DRIVER_MEMORY -> "1G", - NEW_CONFIG_AM_CORES -> "3", - NEW_CONFIG_DRIVER_CORES -> "1G" - ).foreach { case (key, value) => - test(s"disallowed resource request: $key") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) - val conf = new SparkConf(false).set(key, value) - val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(conf) - } - thrown.getMessage should include (key) - } - } - - test("multiple disallowed resources in config") { - val sparkConf = new SparkConf() - sparkConf.set(DRIVER_MEMORY.key, "2G") - sparkConf.set(DRIVER_CORES.key, "2") - sparkConf.set(EXECUTOR_MEMORY.key, "2G") - sparkConf.set(EXECUTOR_CORES.key, "4") - sparkConf.set(AM_MEMORY.key, "3G") - sparkConf.set(NEW_CONFIG_EXECUTOR_MEMORY, "3G") - sparkConf.set(NEW_CONFIG_AM_MEMORY, "2G") - sparkConf.set(NEW_CONFIG_DRIVER_MEMORY, "2G") - - val thrown = intercept[SparkException] { - ResourceRequestHelper.validateResources(sparkConf) - } - thrown.getMessage should ( - include(NEW_CONFIG_EXECUTOR_MEMORY) and - include(NEW_CONFIG_AM_MEMORY) and - include(NEW_CONFIG_DRIVER_MEMORY)) - } - - private def createResource(): Resource = { - val resource = Records.newRecord(classOf[Resource]) - resource.setMemory(512) - resource.setVirtualCores(2) - resource - } -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala deleted file mode 100644 index 953d447bf4b6e..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.yarn.api.records.Resource - -import org.apache.spark.util.Utils - -object ResourceRequestTestHelper { - def initializeResourceTypes(resourceTypes: Seq[String]): Unit = { - if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { - throw new IllegalStateException("This method should not be invoked " + - "since YARN resource types is not available because of old Hadoop version!" ) - } - - // ResourceUtils.reinitializeResources() is the YARN-way - // to specify resources for the execution of the tests. - // This method should receive standard resources with names of memory-mb and vcores. - // Without specifying the standard resources or specifying them - // with different names e.g. memory, YARN would throw various exceptions - // because it relies on that standard resources are always specified. - val defaultResourceTypes = List( - createResourceTypeInfo("memory-mb"), - createResourceTypeInfo("vcores")) - val customResourceTypes = resourceTypes.map(createResourceTypeInfo) - val allResourceTypes = defaultResourceTypes ++ customResourceTypes - - val resourceUtilsClass = - Utils.classForName("org.apache.hadoop.yarn.util.resource.ResourceUtils") - val reinitializeResourcesMethod = resourceUtilsClass.getMethod("reinitializeResources", - classOf[java.util.List[AnyRef]]) - reinitializeResourcesMethod.invoke(null, allResourceTypes.asJava) - } - - private def createResourceTypeInfo(resourceName: String): AnyRef = { - val resTypeInfoClass = Utils.classForName("org.apache.hadoop.yarn.api.records.ResourceTypeInfo") - val resTypeInfoNewInstanceMethod = resTypeInfoClass.getMethod("newInstance", classOf[String]) - resTypeInfoNewInstanceMethod.invoke(null, resourceName) - } - - def getRequestedValue(res: Resource, rtype: String): AnyRef = { - val resourceInformation = getResourceInformation(res, rtype) - invokeMethod(resourceInformation, "getValue") - } - - def getResourceInformationByName(res: Resource, nameParam: String): ResourceInformation = { - val resourceInformation: AnyRef = getResourceInformation(res, nameParam) - val name = invokeMethod(resourceInformation, "getName").asInstanceOf[String] - val value = invokeMethod(resourceInformation, "getValue").asInstanceOf[Long] - val units = invokeMethod(resourceInformation, "getUnits").asInstanceOf[String] - ResourceInformation(name, value, units) - } - - private def getResourceInformation(res: Resource, name: String): AnyRef = { - if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { - throw new IllegalStateException("assertResourceTypeValue() should not be invoked " + - "since yarn resource types is not available because of old Hadoop version!") - } - - val getResourceInformationMethod = res.getClass.getMethod("getResourceInformation", - classOf[String]) - val resourceInformation = getResourceInformationMethod.invoke(res, name) - resourceInformation - } - - private def invokeMethod(resourceInformation: AnyRef, methodName: String): AnyRef = { - val getValueMethod = resourceInformation.getClass.getMethod(methodName) - getValueMethod.invoke(resourceInformation) - } - - case class ResourceInformation(name: String, value: Long, unit: String) -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 35299166d9814..3f783baed110d 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.mockito.ArgumentCaptor import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} @@ -87,8 +86,7 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter def createAllocator( maxExecutors: Int = 5, - rmClient: AMRMClient[ContainerRequest] = rmClient, - additionalConfigs: Map[String, String] = Map()): YarnAllocator = { + rmClient: AMRMClient[ContainerRequest] = rmClient): YarnAllocator = { val args = Array( "--jar", "somejar.jar", "--class", "SomeClass") @@ -97,11 +95,6 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter .set("spark.executor.instances", maxExecutors.toString) .set("spark.executor.cores", "5") .set("spark.executor.memory", "2048") - - for ((name, value) <- additionalConfigs) { - sparkConfClone.set(name, value) - } - new YarnAllocator( "not used", mock(classOf[RpcEndpointRef]), @@ -115,12 +108,12 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock) } - def createContainer(host: String, resource: Resource = containerResource): Container = { + def createContainer(host: String): Container = { // When YARN 2.6+ is required, avoid deprecation by using version with long second arg val containerId = ContainerId.newInstance(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) - Container.newInstance(containerId, nodeId, "", resource, RM_REQUEST_PRIORITY, null) + Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null) } test("single container allocated") { @@ -141,29 +134,6 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter size should be (0) } - test("custom resource requested from yarn") { - assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) - ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) - - val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) - val handler = createAllocator(1, mockAmClient, - Map(YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "gpu" -> "2G")) - - handler.updateResourceRequests() - val container = createContainer("host1", handler.resource) - handler.handleAllocatedContainers(Array(container)) - - // get amount of memory and vcores from resource, so effectively skipping their validation - val expectedResources = Resource.newInstance(handler.resource.getMemory(), - handler.resource.getVirtualCores) - ResourceRequestHelper.setResourceRequests(Map("gpu" -> "2G"), expectedResources) - val captor = ArgumentCaptor.forClass(classOf[ContainerRequest]) - - verify(mockAmClient).addContainerRequest(captor.capture()) - val containerRequest: ContainerRequest = captor.getValue - assert(containerRequest.getCapability === expectedResources) - } - test("container should not be created if requested number if met") { // request a single container and receive it val handler = createAllocator(1) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 6bf273c717aec..c2c2976bea151 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -311,7 +311,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.8.1-src.zip", + s"$sparkHome/python/lib/py4j-0.10.7-src.zip", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), @@ -350,7 +350,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.8.1-src.zip", + s"$sparkHome/python/lib/py4j-0.10.7-src.zip", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala index 98315e4235741..9fa749b14c98c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.Credentials +import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite { +class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { private var credentialManager: YARNHadoopDelegationTokenManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -35,7 +36,7 @@ class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite { test("Correctly loads credential providers") { credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - assert(credentialManager.isProviderLoaded("yarn-test")) + credentialManager.credentialProviders.get("yarn-test") should not be (None) } } diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 0771e2a044757..bf3da18c3706e 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.8.1-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 71dace47767cb..38a43b98c3992 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -28,22 +28,7 @@ if [ -z "${SPARK_HOME}" ]; then export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" fi -# NOTE: This exact class name is matched downstream by SparkSubmit. -# Any changes need to be reflected there. -CLASS="org.apache.spark.deploy.history.HistoryServer" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-history-server.sh [options]" - pattern="Usage:" - pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Started daemon with process name" - pattern+="\|Registered signal handler for" - - "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 - exit 1 -fi - . "${SPARK_HOME}/sbin/spark-config.sh" . "${SPARK_HOME}/bin/load-spark-env.sh" -exec "${SPARK_HOME}/sbin"/spark-daemon.sh start $CLASS 1 "$@" +exec "${SPARK_HOME}/sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 "$@" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index b6a566e4daf4b..97ee32159b6de 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -31,8 +31,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./sbin/start-master.sh [options]" pattern="Usage:" pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Started daemon with process name" - pattern+="\|Registered signal handler for" + pattern+="\|Registered signal handlers for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 exit 1 diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 247c9e20395ea..8c268b8859155 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -43,8 +43,7 @@ if [[ $# -lt 1 ]] || [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./sbin/start-slave.sh [options] " pattern="Usage:" pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Started daemon with process name" - pattern+="\|Registered signal handler for" + pattern+="\|Registered signal handlers for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 exit 1 diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index b1d38713218b7..f02f31793e346 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -39,10 +39,6 @@ function usage { pattern+="\|Spark Command: " pattern+="\|=======" pattern+="\|--help" - pattern+="\|Using Spark's default log4j profile:" - pattern+="\|^log4j:" - pattern+="\|Started daemon with process name" - pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo @@ -52,7 +48,7 @@ function usage { if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage - exit 1 + exit 0 fi export SUBMIT_USAGE_FUNCTION=usage diff --git a/spark-docker-image-generator/src/test/resources/ExpectedDockerfile b/spark-docker-image-generator/src/test/resources/ExpectedDockerfile index 2e0613cd2a826..b9b6114752f9f 100644 --- a/spark-docker-image-generator/src/test/resources/ExpectedDockerfile +++ b/spark-docker-image-generator/src/test/resources/ExpectedDockerfile @@ -18,9 +18,7 @@ FROM fabric8/java-centos-openjdk8-jdk:latest ARG spark_jars=jars -ARG example_jars=examples/jars ARG img_path=kubernetes/dockerfiles -ARG k8s_tests=kubernetes/tests # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. @@ -31,7 +29,7 @@ ARG k8s_tests=kubernetes/tests RUN set -ex && \ apk upgrade --no-cache && \ - apk add --no-cache bash tini krb5 krb5-libs && \ + apk add --no-cache bash tini && \ mkdir -p /opt/spark && \ mkdir -p /opt/spark/work-dir && \ touch /opt/spark/RELEASE && \ diff --git a/sql/catalyst/benchmarks/HashBenchmark-results.txt b/sql/catalyst/benchmarks/HashBenchmark-results.txt deleted file mode 100644 index 2459b35c75bb5..0000000000000 --- a/sql/catalyst/benchmarks/HashBenchmark-results.txt +++ /dev/null @@ -1,70 +0,0 @@ -================================================================================================ -single ints -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For single ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 5615 / 5616 95.6 10.5 1.0X -codegen version 8400 / 8407 63.9 15.6 0.7X -codegen version 64-bit 8139 / 8145 66.0 15.2 0.7X -codegen HiveHash version 7213 / 7348 74.4 13.4 0.8X - - -================================================================================================ -single longs -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For single longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 6053 / 6054 88.7 11.3 1.0X -codegen version 9367 / 9369 57.3 17.4 0.6X -codegen version 64-bit 8041 / 8051 66.8 15.0 0.8X -codegen HiveHash version 7546 / 7575 71.1 14.1 0.8X - - -================================================================================================ -normal -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 3181 / 3182 0.7 1517.0 1.0X -codegen version 2403 / 2403 0.9 1145.7 1.3X -codegen version 64-bit 915 / 916 2.3 436.2 3.5X -codegen HiveHash version 4505 / 4527 0.5 2148.3 0.7X - - -================================================================================================ -array -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 1828 / 1844 0.1 13946.1 1.0X -codegen version 3678 / 3804 0.0 28058.2 0.5X -codegen version 64-bit 2925 / 2931 0.0 22317.8 0.6X -codegen HiveHash version 1216 / 1217 0.1 9280.0 1.5X - - -================================================================================================ -map -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -interpreted version 0 / 0 44.3 22.6 1.0X -codegen version 176 / 176 0.0 42978.8 0.0X -codegen version 64-bit 173 / 175 0.0 42214.3 0.0X -codegen HiveHash version 44 / 44 0.1 10659.9 0.0X - - diff --git a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt b/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt deleted file mode 100644 index a4304ee3b5f60..0000000000000 --- a/sql/catalyst/benchmarks/HashByteArrayBenchmark-results.txt +++ /dev/null @@ -1,77 +0,0 @@ -================================================================================================ -Benchmark for MurMurHash 3 and xxHash64 -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 8: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 16 / 16 127.7 7.8 1.0X -xxHash 64-bit 23 / 23 90.7 11.0 0.7X -HiveHasher 16 / 16 134.8 7.4 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 16: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 26 / 26 79.5 12.6 1.0X -xxHash 64-bit 26 / 27 79.3 12.6 1.0X -HiveHasher 30 / 30 70.1 14.3 0.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 24: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 36 / 36 58.1 17.2 1.0X -xxHash 64-bit 30 / 30 70.2 14.2 1.2X -HiveHasher 45 / 45 46.4 21.5 0.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 31: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 50 / 50 41.8 23.9 1.0X -xxHash 64-bit 43 / 43 49.3 20.3 1.2X -HiveHasher 58 / 58 35.9 27.8 0.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 95: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 132 / 132 15.9 62.7 1.0X -xxHash 64-bit 79 / 79 26.7 37.5 1.7X -HiveHasher 198 / 199 10.6 94.6 0.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 287: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 334 / 334 6.3 159.3 1.0X -xxHash 64-bit 126 / 126 16.7 59.9 2.7X -HiveHasher 633 / 634 3.3 302.0 0.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 1055: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 1149 / 1149 1.8 547.9 1.0X -xxHash 64-bit 327 / 327 6.4 155.9 3.5X -HiveHasher 2338 / 2346 0.9 1114.6 0.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 2079: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 2215 / 2216 0.9 1056.1 1.0X -xxHash 64-bit 554 / 554 3.8 264.0 4.0X -HiveHasher 4609 / 4609 0.5 2197.5 0.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Hash byte arrays with length 8223: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Murmur3_x86_32 8633 / 8643 0.2 4116.3 1.0X -xxHash 64-bit 1891 / 1892 1.1 901.6 4.6X -HiveHasher 18206 / 18206 0.1 8681.3 0.5X - - diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 16ecebf159c1f..2e7df4fd14042 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -103,12 +103,6 @@ commons-codec commons-codec
    - - com.univocity - univocity-parsers - 2.7.3 - jar - target/scala-${scala.binary.version}/classes diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index e2d34d1650ddc..16665eb0d7374 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -162,8 +162,7 @@ statement tableIdentifier partitionSpec? describeColName? #describeTable | REFRESH TABLE tableIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource - | CACHE LAZY? TABLE tableIdentifier - (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable + | CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE @@ -469,7 +468,7 @@ joinType joinCriteria : ON booleanExpression - | USING identifierList + | USING '(' identifier (',' identifier)* ')' ; sample diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java index 460513816dfd9..551443a11298b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java @@ -16,7 +16,6 @@ */ package org.apache.spark.sql.catalyst.expressions; -import java.io.Closeable; import java.io.IOException; import org.apache.spark.memory.MemoryConsumer; @@ -46,7 +45,7 @@ * page requires an average size for key value pairs to be larger than 1024 bytes. * */ -public abstract class RowBasedKeyValueBatch extends MemoryConsumer implements Closeable { +public abstract class RowBasedKeyValueBatch extends MemoryConsumer { protected final Logger logger = LoggerFactory.getLogger(RowBasedKeyValueBatch.class); private static final int DEFAULT_CAPACITY = 1 << 16; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 8a30c81912fe9..b47ec0b72c638 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -203,10 +203,12 @@ object Encoders { validatePublicClass[T]() ExpressionEncoder[T]( - objSerializer = + schema = new StructType().add("value", BinaryType), + flat = true, + serializer = Seq( EncodeUsingSerializer( - BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo), - objDeserializer = + BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo)), + deserializer = DecodeUsingSerializer[T]( Cast(GetColumnByOrdinal(0, BinaryType), BinaryType), classTag[T], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 8ef8b2be6939c..3ecc137c8cd7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -26,7 +26,7 @@ import scala.language.existentials import com.google.common.reflect.TypeToken -import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} @@ -187,23 +187,26 @@ object JavaTypeInference { } /** - * Returns an expression that can be used to deserialize a Spark SQL representation to an object - * of java bean `T` with a compatible schema. The Spark SQL representation is located at ordinal - * 0 of a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed - * using `UnresolvedExtractValue`. + * Returns an expression that can be used to deserialize an internal row to an object of java bean + * `T` with a compatible schema. Fields of the row will be extracted using UnresolvedAttributes + * of the same name as the constructor arguments. Nested classes will have their fields accessed + * using UnresolvedExtractValue. */ def deserializerFor(beanClass: Class[_]): Expression = { - val typeToken = TypeToken.of(beanClass) - deserializerFor(typeToken, GetColumnByOrdinal(0, inferDataType(typeToken)._1)) + deserializerFor(TypeToken.of(beanClass), None) } - private def deserializerFor(typeToken: TypeToken[_], path: Expression): Expression = { + private def deserializerFor(typeToken: TypeToken[_], path: Option[Expression]): Expression = { /** Returns the current path with a sub-field extracted. */ - def addToPath(part: String): Expression = UnresolvedExtractValue(path, - expressions.Literal(part)) + def addToPath(part: String): Expression = path + .map(p => UnresolvedExtractValue(p, expressions.Literal(part))) + .getOrElse(UnresolvedAttribute(part)) + + /** Returns the current path or `GetColumnByOrdinal`. */ + def getPath: Expression = path.getOrElse(GetColumnByOrdinal(0, inferDataType(typeToken)._1)) typeToken.getRawType match { - case c if !inferExternalType(c).isInstanceOf[ObjectType] => path + case c if !inferExternalType(c).isInstanceOf[ObjectType] => getPath case c if c == classOf[java.lang.Short] || c == classOf[java.lang.Integer] || @@ -216,7 +219,7 @@ object JavaTypeInference { c, ObjectType(c), "valueOf", - path :: Nil, + getPath :: Nil, returnNullable = false) case c if c == classOf[java.sql.Date] => @@ -224,7 +227,7 @@ object JavaTypeInference { DateTimeUtils.getClass, ObjectType(c), "toJavaDate", - path :: Nil, + getPath :: Nil, returnNullable = false) case c if c == classOf[java.sql.Timestamp] => @@ -232,14 +235,14 @@ object JavaTypeInference { DateTimeUtils.getClass, ObjectType(c), "toJavaTimestamp", - path :: Nil, + getPath :: Nil, returnNullable = false) case c if c == classOf[java.lang.String] => - Invoke(path, "toString", ObjectType(classOf[String])) + Invoke(getPath, "toString", ObjectType(classOf[String])) case c if c == classOf[java.math.BigDecimal] => - Invoke(path, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal])) + Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal])) case c if c.isArray => val elementType = c.getComponentType @@ -255,12 +258,12 @@ object JavaTypeInference { } primitiveMethod.map { method => - Invoke(path, method, ObjectType(c)) + Invoke(getPath, method, ObjectType(c)) }.getOrElse { Invoke( MapObjects( - p => deserializerFor(typeToken.getComponentType, p), - path, + p => deserializerFor(typeToken.getComponentType, Some(p)), + getPath, inferDataType(elementType)._1), "array", ObjectType(c)) @@ -268,27 +271,32 @@ object JavaTypeInference { case c if listType.isAssignableFrom(typeToken) => val et = elementType(typeToken) - UnresolvedMapObjects( - p => deserializerFor(et, p), - path, + MapObjects( + p => deserializerFor(et, Some(p)), + getPath, + inferDataType(et)._1, customCollectionCls = Some(c)) case _ if mapType.isAssignableFrom(typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) + val keyDataType = inferDataType(keyType)._1 + val valueDataType = inferDataType(valueType)._1 val keyData = Invoke( - UnresolvedMapObjects( - p => deserializerFor(keyType, p), - MapKeys(path)), + MapObjects( + p => deserializerFor(keyType, Some(p)), + Invoke(getPath, "keyArray", ArrayType(keyDataType)), + keyDataType), "array", ObjectType(classOf[Array[Any]])) val valueData = Invoke( - UnresolvedMapObjects( - p => deserializerFor(valueType, p), - MapValues(path)), + MapObjects( + p => deserializerFor(valueType, Some(p)), + Invoke(getPath, "valueArray", ArrayType(valueDataType)), + valueDataType), "array", ObjectType(classOf[Array[Any]])) @@ -304,7 +312,7 @@ object JavaTypeInference { other, ObjectType(other), "valueOf", - Invoke(path, "toString", ObjectType(classOf[String]), returnNullable = false) :: Nil, + Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) :: Nil, returnNullable = false) case other => @@ -313,7 +321,7 @@ object JavaTypeInference { val fieldName = p.getName val fieldType = typeToken.method(p.getReadMethod).getReturnType val (_, nullable) = inferDataType(fieldType) - val constructor = deserializerFor(fieldType, addToPath(fieldName)) + val constructor = deserializerFor(fieldType, Some(addToPath(fieldName))) val setter = if (nullable) { constructor } else { @@ -325,23 +333,28 @@ object JavaTypeInference { val newInstance = NewInstance(other, Nil, ObjectType(other), propagateNull = false) val result = InitializeJavaBean(newInstance, setters) - expressions.If( - IsNull(path), - expressions.Literal.create(null, ObjectType(other)), + if (path.nonEmpty) { + expressions.If( + IsNull(getPath), + expressions.Literal.create(null, ObjectType(other)), + result + ) + } else { result - ) + } } } /** - * Returns an expression for serializing an object of the given type to a Spark SQL - * representation. The input object is located at ordinal 0 of a row, i.e., - * `BoundReference(0, _)`. + * Returns an expression for serializing an object of the given type to an internal row. */ - def serializerFor(beanClass: Class[_]): Expression = { + def serializerFor(beanClass: Class[_]): CreateNamedStruct = { val inputObject = BoundReference(0, ObjectType(beanClass), nullable = true) val nullSafeInput = AssertNotNull(inputObject, Seq("top level input bean")) - serializerFor(nullSafeInput, TypeToken.of(beanClass)) + serializerFor(nullSafeInput, TypeToken.of(beanClass)) match { + case expressions.If(_, _, s: CreateNamedStruct) => s + case other => CreateNamedStruct(expressions.Literal("value") :: other :: Nil) + } } private def serializerFor(inputObject: Expression, typeToken: TypeToken[_]): Expression = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 912744eab6a3a..0238d57de2446 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.catalyst import java.lang.reflect.Constructor -import scala.util.Properties - import org.apache.commons.lang3.reflect.ConstructorUtils -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, MapData} @@ -129,43 +126,21 @@ object ScalaReflection extends ScalaReflection { } /** - * When we build the `deserializer` for an encoder, we set up a lot of "unresolved" stuff - * and lost the required data type, which may lead to runtime error if the real type doesn't - * match the encoder's schema. - * For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type - * is [a: int, b: long], then we will hit runtime error and say that we can't construct class - * `Data` with int and long, because we lost the information that `b` should be a string. - * - * This method help us "remember" the required data type by adding a `UpCast`. Note that we - * only need to do this for leaf nodes. - */ - private def upCastToExpectedType(expr: Expression, expected: DataType, - walkedTypePath: Seq[String]): Expression = expected match { - case _: StructType => expr - case _: ArrayType => expr - case _: MapType => expr - case _ => UpCast(expr, expected, walkedTypePath) - } - - /** - * Returns an expression that can be used to deserialize a Spark SQL representation to an object - * of type `T` with a compatible schema. The Spark SQL representation is located at ordinal 0 of - * a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed using - * `UnresolvedExtractValue`. + * Returns an expression that can be used to deserialize an input row to an object of type `T` + * with a compatible schema. Fields of the row will be extracted using UnresolvedAttributes + * of the same name as the constructor arguments. Nested classes will have their fields accessed + * using UnresolvedExtractValue. * - * The returned expression is used by `ExpressionEncoder`. The encoder will resolve and bind this - * deserializer expression when using it. + * When used on a primitive type, the constructor will instead default to extracting the value + * from ordinal 0 (since there are no names to map to). The actual location can be moved by + * calling resolve/bind with a new schema. */ - def deserializerForType(tpe: `Type`): Expression = { + def deserializerFor[T : TypeTag]: Expression = { + val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "$clsName"""" :: Nil - val Schema(dataType, nullable) = schemaFor(tpe) - - // Assumes we are deserializing the first column of a row. - val input = upCastToExpectedType( - GetColumnByOrdinal(0, dataType), dataType, walkedTypePath) - - val expr = deserializerFor(tpe, input, walkedTypePath) + val expr = deserializerFor(tpe, None, walkedTypePath) + val Schema(_, nullable) = schemaFor(tpe) if (nullable) { expr } else { @@ -173,22 +148,16 @@ object ScalaReflection extends ScalaReflection { } } - /** - * Returns an expression that can be used to deserialize an input expression to an object of type - * `T` with a compatible schema. - * - * @param tpe The `Type` of deserialized object. - * @param path The expression which can be used to extract serialized value. - * @param walkedTypePath The paths from top to bottom to access current field when deserializing. - */ private def deserializerFor( tpe: `Type`, - path: Expression, + path: Option[Expression], walkedTypePath: Seq[String]): Expression = cleanUpReflectionObjects { /** Returns the current path with a sub-field extracted. */ def addToPath(part: String, dataType: DataType, walkedTypePath: Seq[String]): Expression = { - val newPath = UnresolvedExtractValue(path, expressions.Literal(part)) + val newPath = path + .map(p => UnresolvedExtractValue(p, expressions.Literal(part))) + .getOrElse(UnresolvedAttribute.quoted(part)) upCastToExpectedType(newPath, dataType, walkedTypePath) } @@ -197,12 +166,46 @@ object ScalaReflection extends ScalaReflection { ordinal: Int, dataType: DataType, walkedTypePath: Seq[String]): Expression = { - val newPath = GetStructField(path, ordinal) + val newPath = path + .map(p => GetStructField(p, ordinal)) + .getOrElse(GetColumnByOrdinal(ordinal, dataType)) upCastToExpectedType(newPath, dataType, walkedTypePath) } + /** Returns the current path or `GetColumnByOrdinal`. */ + def getPath: Expression = { + val dataType = schemaFor(tpe).dataType + if (path.isDefined) { + path.get + } else { + upCastToExpectedType(GetColumnByOrdinal(0, dataType), dataType, walkedTypePath) + } + } + + /** + * When we build the `deserializer` for an encoder, we set up a lot of "unresolved" stuff + * and lost the required data type, which may lead to runtime error if the real type doesn't + * match the encoder's schema. + * For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type + * is [a: int, b: long], then we will hit runtime error and say that we can't construct class + * `Data` with int and long, because we lost the information that `b` should be a string. + * + * This method help us "remember" the required data type by adding a `UpCast`. Note that we + * only need to do this for leaf nodes. + */ + def upCastToExpectedType( + expr: Expression, + expected: DataType, + walkedTypePath: Seq[String]): Expression = expected match { + case _: StructType => expr + case _: ArrayType => expr + // TODO: ideally we should also skip MapType, but nested StructType inside MapType is rare and + // it's not trivial to support by-name resolution for StructType inside MapType. + case _ => UpCast(expr, expected, walkedTypePath) + } + tpe.dealias match { - case t if !dataTypeFor(t).isInstanceOf[ObjectType] => path + case t if !dataTypeFor(t).isInstanceOf[ObjectType] => getPath case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t @@ -213,44 +216,44 @@ object ScalaReflection extends ScalaReflection { case t if t <:< localTypeOf[java.lang.Integer] => val boxedType = classOf[java.lang.Integer] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Long] => val boxedType = classOf[java.lang.Long] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Double] => val boxedType = classOf[java.lang.Double] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Float] => val boxedType = classOf[java.lang.Float] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Short] => val boxedType = classOf[java.lang.Short] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Byte] => val boxedType = classOf[java.lang.Byte] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.Boolean] => val boxedType = classOf[java.lang.Boolean] val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", path :: Nil, returnNullable = false) + StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.sql.Date] => StaticInvoke( DateTimeUtils.getClass, ObjectType(classOf[java.sql.Date]), "toJavaDate", - path :: Nil, + getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.sql.Timestamp] => @@ -258,25 +261,25 @@ object ScalaReflection extends ScalaReflection { DateTimeUtils.getClass, ObjectType(classOf[java.sql.Timestamp]), "toJavaTimestamp", - path :: Nil, + getPath :: Nil, returnNullable = false) case t if t <:< localTypeOf[java.lang.String] => - Invoke(path, "toString", ObjectType(classOf[String]), returnNullable = false) + Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) case t if t <:< localTypeOf[java.math.BigDecimal] => - Invoke(path, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), + Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), returnNullable = false) case t if t <:< localTypeOf[BigDecimal] => - Invoke(path, "toBigDecimal", ObjectType(classOf[BigDecimal]), returnNullable = false) + Invoke(getPath, "toBigDecimal", ObjectType(classOf[BigDecimal]), returnNullable = false) case t if t <:< localTypeOf[java.math.BigInteger] => - Invoke(path, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger]), + Invoke(getPath, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger]), returnNullable = false) case t if t <:< localTypeOf[scala.math.BigInt] => - Invoke(path, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt]), + Invoke(getPath, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt]), returnNullable = false) case t if t <:< localTypeOf[Array[_]] => @@ -288,7 +291,7 @@ object ScalaReflection extends ScalaReflection { val mapFunction: Expression => Expression = element => { // upcast the array element to the data type the encoder expected. val casted = upCastToExpectedType(element, dataType, newTypePath) - val converter = deserializerFor(elementType, casted, newTypePath) + val converter = deserializerFor(elementType, Some(casted), newTypePath) if (elementNullable) { converter } else { @@ -296,7 +299,7 @@ object ScalaReflection extends ScalaReflection { } } - val arrayData = UnresolvedMapObjects(mapFunction, path) + val arrayData = UnresolvedMapObjects(mapFunction, getPath) val arrayCls = arrayClassFor(elementType) if (elementNullable) { @@ -328,7 +331,7 @@ object ScalaReflection extends ScalaReflection { val mapFunction: Expression => Expression = element => { // upcast the array element to the data type the encoder expected. val casted = upCastToExpectedType(element, dataType, newTypePath) - val converter = deserializerFor(elementType, casted, newTypePath) + val converter = deserializerFor(elementType, Some(casted), newTypePath) if (elementNullable) { converter } else { @@ -343,16 +346,16 @@ object ScalaReflection extends ScalaReflection { classOf[scala.collection.Set[_]] case _ => mirror.runtimeClass(t.typeSymbol.asClass) } - UnresolvedMapObjects(mapFunction, path, Some(cls)) + UnresolvedMapObjects(mapFunction, getPath, Some(cls)) case t if t <:< localTypeOf[Map[_, _]] => // TODO: add walked type path for map val TypeRef(_, _, Seq(keyType, valueType)) = t - UnresolvedCatalystToExternalMap( - path, - p => deserializerFor(keyType, p, walkedTypePath), - p => deserializerFor(valueType, p, walkedTypePath), + CatalystToExternalMap( + p => deserializerFor(keyType, Some(p), walkedTypePath), + p => deserializerFor(valueType, Some(p), walkedTypePath), + getPath, mirror.runtimeClass(t.typeSymbol.asClass) ) @@ -362,7 +365,7 @@ object ScalaReflection extends ScalaReflection { udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) case t if UDTRegistration.exists(getClassNameFromType(t)) => val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() @@ -371,7 +374,7 @@ object ScalaReflection extends ScalaReflection { udt.getClass, Nil, dataType = ObjectType(udt.getClass)) - Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) + Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) @@ -386,12 +389,12 @@ object ScalaReflection extends ScalaReflection { val constructor = if (cls.getName startsWith "scala.Tuple") { deserializerFor( fieldType, - addToPathOrdinal(i, dataType, newTypePath), + Some(addToPathOrdinal(i, dataType, newTypePath)), newTypePath) } else { deserializerFor( fieldType, - addToPath(fieldName, dataType, newTypePath), + Some(addToPath(fieldName, dataType, newTypePath)), newTypePath) } @@ -404,17 +407,20 @@ object ScalaReflection extends ScalaReflection { val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) - expressions.If( - IsNull(path), - expressions.Literal.create(null, ObjectType(cls)), + if (path.nonEmpty) { + expressions.If( + IsNull(getPath), + expressions.Literal.create(null, ObjectType(cls)), + newInstance + ) + } else { newInstance - ) + } } } /** - * Returns an expression for serializing an object of type T to Spark SQL representation. The - * input object is located at ordinal 0 of a row, i.e., `BoundReference(0, _)`. + * Returns an expression for serializing an object of type T to an internal row. * * If the given type is not supported, i.e. there is no encoder can be built for this type, * an [[UnsupportedOperationException]] will be thrown with detailed error message to explain @@ -425,21 +431,17 @@ object ScalaReflection extends ScalaReflection { * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` */ - def serializerForType(tpe: `Type`): Expression = ScalaReflection.cleanUpReflectionObjects { + def serializerFor[T : TypeTag](inputObject: Expression): CreateNamedStruct = { + val tpe = localTypeOf[T] val clsName = getClassNameFromType(tpe) val walkedTypePath = s"""- root class: "$clsName"""" :: Nil - - // The input object to `ExpressionEncoder` is located at first column of an row. - val isPrimitive = tpe.typeSymbol.asClass.isPrimitive - val inputObject = BoundReference(0, dataTypeFor(tpe), nullable = !isPrimitive) - - serializerFor(inputObject, tpe, walkedTypePath) + serializerFor(inputObject, tpe, walkedTypePath) match { + case expressions.If(_, _, s: CreateNamedStruct) if definedByConstructorParams(tpe) => s + case other => CreateNamedStruct(expressions.Literal("value") :: other :: Nil) + } } - /** - * Returns an expression for serializing the value of an input expression into Spark SQL - * internal representation. - */ + /** Helper for extracting internal fields from a case class. */ private def serializerFor( inputObject: Expression, tpe: `Type`, @@ -877,7 +879,7 @@ object ScalaReflection extends ScalaReflection { * Support for generating catalyst schemas for scala objects. Note that unlike its companion * object, this trait able to work in both the runtime and the compile time (macro) universe. */ -trait ScalaReflection extends Logging { +trait ScalaReflection { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe @@ -930,23 +932,6 @@ trait ScalaReflection extends Logging { tpe.dealias.erasure.typeSymbol.asClass.fullName } - /** - * Returns the nullability of the input parameter types of the scala function object. - * - * Note that this only works with Scala 2.11, and the information returned may be inaccurate if - * used with a different Scala version. - */ - def getParameterTypeNullability(func: AnyRef): Seq[Boolean] = { - if (!Properties.versionString.contains("2.11")) { - logWarning(s"Scala ${Properties.versionString} cannot get type nullability correctly via " + - "reflection, thus Spark cannot add proper input null check for UDF. To avoid this " + - "problem, use the typed UDF interfaces instead.") - } - val methods = func.getClass.getMethods.filter(m => m.getName == "apply" && !m.isBridge) - assert(methods.length == 1) - methods.head.getParameterTypes.map(!_.isPrimitive) - } - /** * Returns the parameter names and types for the primary constructor of this type. * 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 c2d22c5e7ce60..d72e512e0df56 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 @@ -2137,27 +2137,34 @@ class Analyzer( case p => p transformExpressionsUp { - case udf @ ScalaUDF(_, _, inputs, inputsNullSafe, _, _, _, _) - if inputsNullSafe.contains(false) => - // Otherwise, add special handling of null for fields that can't accept null. - // The result of operations like this, when passed null, is generally to return null. - assert(inputsNullSafe.length == inputs.length) - - // TODO: skip null handling for not-nullable primitive inputs after we can completely - // trust the `nullable` information. - val inputsNullCheck = inputsNullSafe.zip(inputs) - .filter { case (nullSafe, _) => !nullSafe } - .map { case (_, expr) => IsNull(expr) } - .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) - // Once we add an `If` check above the udf, it is safe to mark those checked inputs - // as null-safe (i.e., set `inputsNullSafe` all `true`), because the null-returning - // branch of `If` will be called if any of these checked inputs is null. Thus we can - // prevent this rule from being applied repeatedly. - val newInputsNullSafe = inputsNullSafe.map(_ => true) - inputsNullCheck - .map(If(_, Literal.create(null, udf.dataType), - udf.copy(inputsNullSafe = newInputsNullSafe))) - .getOrElse(udf) + case udf@ScalaUDF(func, _, inputs, _, _, _, _, nullableTypes) => + if (nullableTypes.isEmpty) { + // If no nullability info is available, do nothing. No fields will be specially + // checked for null in the plan. If nullability info is incorrect, the results + // of the UDF could be wrong. + udf + } else { + // Otherwise, add special handling of null for fields that can't accept null. + // The result of operations like this, when passed null, is generally to return null. + assert(nullableTypes.length == inputs.length) + + // TODO: skip null handling for not-nullable primitive inputs after we can completely + // trust the `nullable` information. + val inputsNullCheck = nullableTypes.zip(inputs) + .filter { case (nullable, _) => !nullable } + .map { case (_, expr) => IsNull(expr) } + .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) + // Once we add an `If` check above the udf, it is safe to mark those checked inputs + // as not nullable (i.e., wrap them with `KnownNotNull`), because the null-returning + // branch of `If` will be called if any of these checked inputs is null. Thus we can + // prevent this rule from being applied repeatedly. + val newInputs = nullableTypes.zip(inputs).map { case (nullable, expr) => + if (nullable) expr else KnownNotNull(expr) + } + inputsNullCheck + .map(If(_, Literal.create(null, udf.dataType), udf.copy(children = newInputs))) + .getOrElse(udf) + } } } } @@ -2384,23 +2391,14 @@ class Analyzer( case UnresolvedMapObjects(func, inputData, cls) if inputData.resolved => inputData.dataType match { case ArrayType(et, cn) => - MapObjects(func, inputData, et, cn, cls) transformUp { + val expr = MapObjects(func, inputData, et, cn, cls) transformUp { case UnresolvedExtractValue(child, fieldName) if child.resolved => ExtractValue(child, fieldName, resolver) } + expr case other => throw new AnalysisException("need an array field but got " + other.catalogString) } - case u: UnresolvedCatalystToExternalMap if u.child.resolved => - u.child.dataType match { - case _: MapType => - CatalystToExternalMap(u) transformUp { - case UnresolvedExtractValue(child, fieldName) if child.resolved => - ExtractValue(child, fieldName, resolver) - } - case other => - throw new AnalysisException("need a map field but got " + other.catalogString) - } } validateNestedTupleFields(result) result 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 cf8fb7eea9580..7dafebff79874 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 @@ -300,9 +300,6 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), - expression[EveryAgg]("every"), - expression[AnyAgg]("any"), - expression[SomeAgg]("some"), // string functions expression[Ascii]("ascii"), @@ -523,11 +520,7 @@ object FunctionRegistry { castAlias("date", DateType), castAlias("timestamp", TimestampType), castAlias("binary", BinaryType), - castAlias("string", StringType), - - // csv - expression[CsvToStructs]("from_csv"), - expression[SchemaOfCsv]("schema_of_csv") + castAlias("string", StringType) ) val builtin: SimpleFunctionRegistry = { 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 857cf382b8f2c..c1ec736c32ed4 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 @@ -407,10 +407,7 @@ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Une * can be key of Map, index of Array, field name of Struct. */ case class UnresolvedExtractValue(child: Expression, extraction: Expression) - extends BinaryExpression with Unevaluable { - - override def left: Expression = child - override def right: Expression = extraction + extends UnaryExpression with Unevaluable { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index 6134d54531a19..af74693000c44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.internal.SQLConf */ case class AliasViewChild(conf: SQLConf) extends Rule[LogicalPlan] with CastSupport { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case v @ View(desc, output, child) if child.resolved && !v.sameOutput(child) => + case v @ View(desc, output, child) if child.resolved && output != child.output => val resolver = conf.resolver val queryColumnNames = desc.viewQueryColumnNames val queryOutput = if (queryColumnNames.nonEmpty) { @@ -70,7 +70,7 @@ case class AliasViewChild(conf: SQLConf) extends Rule[LogicalPlan] with CastSupp } // Map the attributes in the query output to the attributes in the view output by index. val newOutput = output.zip(queryOutput).map { - case (attr, originAttr) if !attr.semanticEquals(originAttr) => + case (attr, originAttr) if attr != originAttr => // The dataType of the output attributes may be not the same with that of the view // output, so we should cast the attribute to the dataType of the view output attribute. // Will throw an AnalysisException if the cast can't perform or might truncate. @@ -112,8 +112,8 @@ object EliminateView extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // The child should have the same output attributes with the View operator, so we simply // remove the View operator. - case v @ View(_, output, child) => - assert(v.sameOutput(child), + case View(_, output, child) => + assert(output == child.output, s"The output of the child ${child.output.mkString("[", ",", "]")} is different from the " + s"view output ${output.mkString("[", ",", "]")}") child diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala deleted file mode 100644 index bbe27831f01df..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.csv - -object CSVExprUtils { - /** - * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). - * This is currently being used in CSV reading path and CSV schema inference. - */ - def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = { - iter.filter { line => - line.trim.nonEmpty && !line.startsWith(options.comment.toString) - } - } - - def skipComments(iter: Iterator[String], options: CSVOptions): Iterator[String] = { - if (options.isCommentSet) { - val commentPrefix = options.comment.toString - iter.dropWhile { line => - line.trim.isEmpty || line.trim.startsWith(commentPrefix) - } - } else { - iter.dropWhile(_.trim.isEmpty) - } - } - - /** - * Extracts header and moves iterator forward so that only data remains in it - */ - def extractHeader(iter: Iterator[String], options: CSVOptions): Option[String] = { - val nonEmptyLines = skipComments(iter, options) - if (nonEmptyLines.hasNext) { - Some(nonEmptyLines.next()) - } else { - None - } - } - - /** - * Helper method that converts string representation of a character to actual character. - * It handles some Java escaped strings and throws exception if given string is longer than one - * character. - */ - @throws[IllegalArgumentException] - def toChar(str: String): Char = { - (str: Seq[Char]) match { - case Seq() => throw new IllegalArgumentException("Delimiter cannot be empty string") - case Seq('\\') => throw new IllegalArgumentException("Single backslash is prohibited." + - " It has special meaning as beginning of an escape sequence." + - " To get the backslash character, pass a string with two backslashes as the delimiter.") - case Seq(c) => c - case Seq('\\', 't') => '\t' - case Seq('\\', 'r') => '\r' - case Seq('\\', 'b') => '\b' - case Seq('\\', 'f') => '\f' - // In case user changes quote char and uses \" as delimiter in options - case Seq('\\', '\"') => '\"' - case Seq('\\', '\'') => '\'' - case Seq('\\', '\\') => '\\' - case _ if str == """\u0000""" => '\u0000' - case Seq('\\', _) => - throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") - case _ => - throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") - } - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala deleted file mode 100644 index c39f77e891ae1..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.csv - -import com.univocity.parsers.csv.CsvParser - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType - -/** - * Checks that column names in a CSV header and field names in the schema are the same - * by taking into account case sensitivity. - * - * @param schema provided (or inferred) schema to which CSV must conform. - * @param options parsed CSV options. - * @param source name of CSV source that are currently checked. It is used in error messages. - * @param isStartOfFile indicates if the currently processing partition is the start of the file. - * if unknown or not applicable (for instance when the input is a dataset), - * can be omitted. - */ -class CSVHeaderChecker( - schema: StructType, - options: CSVOptions, - source: String, - isStartOfFile: Boolean = false) extends Logging { - - // Indicates if it is set to `false`, comparison of column names and schema field - // names is not case sensitive. - private val caseSensitive = SQLConf.get.caseSensitiveAnalysis - - // Indicates if it is `true`, column names are ignored otherwise the CSV column - // names are checked for conformance to the schema. In the case if - // the column name don't conform to the schema, an exception is thrown. - private val enforceSchema = options.enforceSchema - - /** - * Checks that column names in a CSV header and field names in the schema are the same - * by taking into account case sensitivity. - * - * @param columnNames names of CSV columns that must be checked against to the schema. - */ - private def checkHeaderColumnNames(columnNames: Array[String]): Unit = { - if (columnNames != null) { - val fieldNames = schema.map(_.name).toIndexedSeq - val (headerLen, schemaSize) = (columnNames.size, fieldNames.length) - var errorMessage: Option[String] = None - - if (headerLen == schemaSize) { - var i = 0 - while (errorMessage.isEmpty && i < headerLen) { - var (nameInSchema, nameInHeader) = (fieldNames(i), columnNames(i)) - if (!caseSensitive) { - // scalastyle:off caselocale - nameInSchema = nameInSchema.toLowerCase - nameInHeader = nameInHeader.toLowerCase - // scalastyle:on caselocale - } - if (nameInHeader != nameInSchema) { - errorMessage = Some( - s"""|CSV header does not conform to the schema. - | Header: ${columnNames.mkString(", ")} - | Schema: ${fieldNames.mkString(", ")} - |Expected: ${fieldNames(i)} but found: ${columnNames(i)} - |$source""".stripMargin) - } - i += 1 - } - } else { - errorMessage = Some( - s"""|Number of column in CSV header is not equal to number of fields in the schema: - | Header length: $headerLen, schema size: $schemaSize - |$source""".stripMargin) - } - - errorMessage.foreach { msg => - if (enforceSchema) { - logWarning(msg) - } else { - throw new IllegalArgumentException(msg) - } - } - } - } - - // This is currently only used to parse CSV from Dataset[String]. - def checkHeaderColumnNames(line: String): Unit = { - if (options.headerFlag) { - val parser = new CsvParser(options.asParserSettings) - checkHeaderColumnNames(parser.parseLine(line)) - } - } - - // This is currently only used to parse CSV with multiLine mode. - private[csv] def checkHeaderColumnNames(tokenizer: CsvParser): Unit = { - assert(options.multiLine, "This method should be executed with multiLine.") - if (options.headerFlag) { - val firstRecord = tokenizer.parseNext() - checkHeaderColumnNames(firstRecord) - } - } - - // This is currently only used to parse CSV with non-multiLine mode. - private[csv] def checkHeaderColumnNames(lines: Iterator[String], tokenizer: CsvParser): Unit = { - assert(!options.multiLine, "This method should not be executed with multiline.") - // Checking that column names in the header are matched to field names of the schema. - // The header will be removed from lines. - // Note: if there are only comments in the first block, the header would probably - // be not extracted. - if (options.headerFlag && isStartOfFile) { - CSVExprUtils.extractHeader(lines, options).foreach { header => - checkHeaderColumnNames(tokenizer.parseLine(header)) - } - } - } -} 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 592520c59a761..cbea3c017a265 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 @@ -25,11 +25,10 @@ import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaRefle import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} -import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, InitializeJavaBean, Invoke, NewInstance} +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, NewInstance} import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} -import org.apache.spark.sql.types.{ObjectType, StringType, StructField, StructType} -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types.{BooleanType, ObjectType, StructField, StructType} import org.apache.spark.util.Utils /** @@ -44,8 +43,8 @@ import org.apache.spark.util.Utils * to the name `value`. */ object ExpressionEncoder { - def apply[T : TypeTag](): ExpressionEncoder[T] = { + // We convert the not-serializable TypeTag into StructType and ClassTag. val mirror = ScalaReflection.mirror val tpe = typeTag[T].in(mirror).tpe @@ -59,11 +58,25 @@ object ExpressionEncoder { } val cls = mirror.runtimeClass(tpe) - val serializer = ScalaReflection.serializerForType(tpe) - val deserializer = ScalaReflection.deserializerForType(tpe) + val flat = !ScalaReflection.definedByConstructorParams(tpe) + + val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], nullable = !cls.isPrimitive) + val nullSafeInput = if (flat) { + inputObject + } else { + // For input object of Product type, we can't encode it to row if it's null, as Spark SQL + // doesn't allow top-level row to be null, only its columns can be null. + AssertNotNull(inputObject, Seq("top level Product input object")) + } + val serializer = ScalaReflection.serializerFor[T](nullSafeInput) + val deserializer = ScalaReflection.deserializerFor[T] + + val schema = serializer.dataType new ExpressionEncoder[T]( - serializer, + schema, + flat, + serializer.flatten, deserializer, ClassTag[T](cls)) } @@ -73,12 +86,14 @@ object ExpressionEncoder { val schema = JavaTypeInference.inferDataType(beanClass)._1 assert(schema.isInstanceOf[StructType]) - val objSerializer = JavaTypeInference.serializerFor(beanClass) - val objDeserializer = JavaTypeInference.deserializerFor(beanClass) + val serializer = JavaTypeInference.serializerFor(beanClass) + val deserializer = JavaTypeInference.deserializerFor(beanClass) new ExpressionEncoder[T]( - objSerializer, - objDeserializer, + schema.asInstanceOf[StructType], + flat = false, + serializer.flatten, + deserializer, ClassTag[T](beanClass)) } @@ -88,52 +103,77 @@ object ExpressionEncoder { * name/positional binding is preserved. */ def tuple(encoders: Seq[ExpressionEncoder[_]]): ExpressionEncoder[_] = { - // TODO: check if encoders length is more than 22 and throw exception for it. encoders.foreach(_.assertUnresolved()) - val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}") + val schema = StructType(encoders.zipWithIndex.map { + case (e, i) => + val (dataType, nullable) = if (e.flat) { + e.schema.head.dataType -> e.schema.head.nullable + } else { + e.schema -> true + } + StructField(s"_${i + 1}", dataType, nullable) + }) - val newSerializerInput = BoundReference(0, ObjectType(cls), nullable = true) - val serializers = encoders.zipWithIndex.map { case (enc, index) => - val boundRefs = enc.objSerializer.collect { case b: BoundReference => b }.distinct - assert(boundRefs.size == 1, "object serializer should have only one bound reference but " + - s"there are ${boundRefs.size}") + val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}") - val originalInputObject = boundRefs.head + val serializer = encoders.zipWithIndex.map { case (enc, index) => + val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head val newInputObject = Invoke( - newSerializerInput, + BoundReference(0, ObjectType(cls), nullable = true), s"_${index + 1}", - originalInputObject.dataType, - returnNullable = originalInputObject.nullable) - - val newSerializer = enc.objSerializer.transformUp { - case BoundReference(0, _, _) => newInputObject + originalInputObject.dataType) + + val newSerializer = enc.serializer.map(_.transformUp { + case b: BoundReference if b == originalInputObject => newInputObject + }) + + val serializerExpr = if (enc.flat) { + newSerializer.head + } else { + // For non-flat encoder, the input object is not top level anymore after being combined to + // a tuple encoder, thus it can be null and we should wrap the `CreateStruct` with `If` and + // null check to handle null case correctly. + // e.g. for Encoder[(Int, String)], the serializer expressions will create 2 columns, and is + // not able to handle the case when the input tuple is null. This is not a problem as there + // is a check to make sure the input object won't be null. However, if this encoder is used + // to create a bigger tuple encoder, the original input object becomes a filed of the new + // input tuple and can be null. So instead of creating a struct directly here, we should add + // a null/None check and return a null struct if the null/None check fails. + val struct = CreateStruct(newSerializer) + val nullCheck = Or( + IsNull(newInputObject), + Invoke(Literal.fromObject(None), "equals", BooleanType, newInputObject :: Nil)) + If(nullCheck, Literal.create(null, struct.dataType), struct) } - - Alias(newSerializer, s"_${index + 1}")() + Alias(serializerExpr, s"_${index + 1}")() } - val newSerializer = CreateStruct(serializers) - val newDeserializerInput = GetColumnByOrdinal(0, newSerializer.dataType) - val deserializers = encoders.zipWithIndex.map { case (enc, index) => - val getColExprs = enc.objDeserializer.collect { case c: GetColumnByOrdinal => c }.distinct - assert(getColExprs.size == 1, "object deserializer should have only one " + - s"`GetColumnByOrdinal`, but there are ${getColExprs.size}") - - val input = GetStructField(newDeserializerInput, index) - enc.objDeserializer.transformUp { - case GetColumnByOrdinal(0, _) => input + val childrenDeserializers = encoders.zipWithIndex.map { case (enc, index) => + if (enc.flat) { + enc.deserializer.transform { + case g: GetColumnByOrdinal => g.copy(ordinal = index) + } + } else { + val input = GetColumnByOrdinal(index, enc.schema) + val deserialized = enc.deserializer.transformUp { + case UnresolvedAttribute(nameParts) => + assert(nameParts.length == 1) + UnresolvedExtractValue(input, Literal(nameParts.head)) + case GetColumnByOrdinal(ordinal, _) => GetStructField(input, ordinal) + } + If(IsNull(input), Literal.create(null, deserialized.dataType), deserialized) } } - val newDeserializer = NewInstance(cls, deserializers, ObjectType(cls), propagateNull = false) - def nullSafe(input: Expression, result: Expression): Expression = { - If(IsNull(input), Literal.create(null, result.dataType), result) - } + val deserializer = + NewInstance(cls, childrenDeserializers, ObjectType(cls), propagateNull = false) new ExpressionEncoder[Any]( - nullSafe(newSerializerInput, newSerializer), - nullSafe(newDeserializerInput, newDeserializer), + schema, + flat = false, + serializer, + deserializer, ClassTag(cls)) } @@ -172,90 +212,21 @@ object ExpressionEncoder { * A generic encoder for JVM objects that uses Catalyst Expressions for a `serializer` * and a `deserializer`. * - * @param objSerializer An expression that can be used to encode a raw object to corresponding - * Spark SQL representation that can be a primitive column, array, map or a - * struct. This represents how Spark SQL generally serializes an object of - * type `T`. - * @param objDeserializer An expression that will construct an object given a Spark SQL - * representation. This represents how Spark SQL generally deserializes - * a serialized value in Spark SQL representation back to an object of - * type `T`. + * @param schema The schema after converting `T` to a Spark SQL row. + * @param serializer A set of expressions, one for each top-level field that can be used to + * extract the values from a raw object into an [[InternalRow]]. + * @param deserializer An expression that will construct an object given an [[InternalRow]]. * @param clsTag A classtag for `T`. */ case class ExpressionEncoder[T]( - objSerializer: Expression, - objDeserializer: Expression, + schema: StructType, + flat: Boolean, + serializer: Seq[Expression], + deserializer: Expression, clsTag: ClassTag[T]) extends Encoder[T] { - /** - * A sequence of expressions, one for each top-level field that can be used to - * extract the values from a raw object into an [[InternalRow]]: - * 1. If `serializer` encodes a raw object to a struct, strip the outer If-IsNull and get - * the `CreateNamedStruct`. - * 2. For other cases, wrap the single serializer with `CreateNamedStruct`. - */ - val serializer: Seq[NamedExpression] = { - val clsName = Utils.getSimpleName(clsTag.runtimeClass) - - if (isSerializedAsStruct) { - val nullSafeSerializer = objSerializer.transformUp { - case r: BoundReference => - // For input object of Product type, we can't encode it to row if it's null, as Spark SQL - // doesn't allow top-level row to be null, only its columns can be null. - AssertNotNull(r, Seq("top level Product or row object")) - } - nullSafeSerializer match { - case If(_: IsNull, _, s: CreateNamedStruct) => s - case _ => - throw new RuntimeException(s"class $clsName has unexpected serializer: $objSerializer") - } - } else { - // For other input objects like primitive, array, map, etc., we construct a struct to wrap - // the serializer which is a column of an row. - CreateNamedStruct(Literal("value") :: objSerializer :: Nil) - } - }.flatten - - /** - * Returns an expression that can be used to deserialize an input row to an object of type `T` - * with a compatible schema. Fields of the row will be extracted using `UnresolvedAttribute`. - * of the same name as the constructor arguments. - * - * For complex objects that are encoded to structs, Fields of the struct will be extracted using - * `GetColumnByOrdinal` with corresponding ordinal. - */ - val deserializer: Expression = { - if (isSerializedAsStruct) { - // We serialized this kind of objects to root-level row. The input of general deserializer - // is a `GetColumnByOrdinal(0)` expression to extract first column of a row. We need to - // transform attributes accessors. - objDeserializer.transform { - case UnresolvedExtractValue(GetColumnByOrdinal(0, _), - Literal(part: UTF8String, StringType)) => - UnresolvedAttribute.quoted(part.toString) - case GetStructField(GetColumnByOrdinal(0, dt), ordinal, _) => - GetColumnByOrdinal(ordinal, dt) - case If(IsNull(GetColumnByOrdinal(0, _)), _, n: NewInstance) => n - case If(IsNull(GetColumnByOrdinal(0, _)), _, i: InitializeJavaBean) => i - } - } else { - // For other input objects like primitive, array, map, etc., we deserialize the first column - // of a row to the object. - objDeserializer - } - } - - // The schema after converting `T` to a Spark SQL row. This schema is dependent on the given - // serialier. - val schema: StructType = StructType(serializer.map { s => - StructField(s.name, s.dataType, s.nullable) - }) - - /** - * Returns true if the type `T` is serialized as a struct. - */ - def isSerializedAsStruct: Boolean = objSerializer.dataType.isInstanceOf[StructType] + if (flat) require(serializer.size == 1) // serializer expressions are used to encode an object to a row, while the object is usually an // intermediate value produced inside an operator, not from the output of the child operator. This @@ -287,7 +258,7 @@ case class ExpressionEncoder[T]( analyzer.checkAnalysis(analyzedPlan) val resolved = SimplifyCasts(analyzedPlan).asInstanceOf[DeserializeToObject].deserializer val bound = BindReferences.bindReference(resolved, attrs) - copy(objDeserializer = bound) + copy(deserializer = bound) } @transient diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index d905f8f9858e8..3340789398f9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -58,10 +58,12 @@ object RowEncoder { def apply(schema: StructType): ExpressionEncoder[Row] = { val cls = classOf[Row] val inputObject = BoundReference(0, ObjectType(cls), nullable = true) - val serializer = serializerFor(inputObject, schema) - val deserializer = deserializerFor(GetColumnByOrdinal(0, serializer.dataType), schema) + val serializer = serializerFor(AssertNotNull(inputObject, Seq("top level row object")), schema) + val deserializer = deserializerFor(schema) new ExpressionEncoder[Row]( - serializer, + schema, + flat = false, + serializer.asInstanceOf[CreateNamedStruct].flatten, deserializer, ClassTag(cls)) } @@ -169,7 +171,7 @@ object RowEncoder { if (inputObject.nullable) { If(IsNull(inputObject), - Literal.create(null, nonNullOutput.dataType), + Literal.create(null, inputType), nonNullOutput) } else { nonNullOutput @@ -185,9 +187,7 @@ object RowEncoder { val convertedField = if (field.nullable) { If( Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil), - // Because we strip UDTs, `field.dataType` can be different from `fieldValue.dataType`. - // We should use `fieldValue.dataType` here. - Literal.create(null, fieldValue.dataType), + Literal.create(null, field.dataType), fieldValue ) } else { @@ -198,7 +198,7 @@ object RowEncoder { if (inputObject.nullable) { If(IsNull(inputObject), - Literal.create(null, nonNullOutput.dataType), + Literal.create(null, inputType), nonNullOutput) } else { nonNullOutput @@ -235,9 +235,13 @@ object RowEncoder { case udt: UserDefinedType[_] => ObjectType(udt.userClass) } - private def deserializerFor(input: Expression, schema: StructType): Expression = { + private def deserializerFor(schema: StructType): Expression = { val fields = schema.zipWithIndex.map { case (f, i) => - deserializerFor(GetStructField(input, i)) + val dt = f.dataType match { + case p: PythonUserDefinedType => p.sqlType + case other => other + } + deserializerFor(GetColumnByOrdinal(i, dt)) } CreateExternalRow(fields, schema) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala deleted file mode 100644 index 040b56cc1caea..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData -import org.apache.spark.sql.types.{DataType, MapType, StringType, StructType} -import org.apache.spark.unsafe.types.UTF8String - -object ExprUtils { - - def evalSchemaExpr(exp: Expression): StructType = { - // Use `DataType.fromDDL` since the type string can be struct<...>. - val dataType = exp match { - case Literal(s, StringType) => - DataType.fromDDL(s.toString) - case e @ SchemaOfCsv(_: Literal, _) => - val ddlSchema = e.eval(EmptyRow).asInstanceOf[UTF8String] - DataType.fromDDL(ddlSchema.toString) - case e => throw new AnalysisException( - "Schema should be specified in DDL format as a string literal or output of " + - s"the schema_of_csv function instead of ${e.sql}") - } - - if (!dataType.isInstanceOf[StructType]) { - throw new AnalysisException( - s"Schema should be struct type but got ${dataType.sql}.") - } - dataType.asInstanceOf[StructType] - } - - def evalTypeExpr(exp: Expression): DataType = exp match { - case Literal(s, StringType) => DataType.fromDDL(s.toString) - case e @ SchemaOfJson(_: Literal, _) => - val ddlSchema = e.eval(EmptyRow).asInstanceOf[UTF8String] - DataType.fromDDL(ddlSchema.toString) - case e => throw new AnalysisException( - "Schema should be specified in DDL format as a string literal or output of " + - s"the schema_of_json function instead of ${e.sql}") - } - - def convertToMapData(exp: Expression): Map[String, String] = exp match { - case m: CreateMap - if m.dataType.acceptsType(MapType(StringType, StringType, valueContainsNull = false)) => - val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] - ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => - key.toString -> value.toString - } - case m: CreateMap => - throw new AnalysisException( - s"A type of keys and values in map() must be string, but got ${m.dataType.catalogString}") - case _ => - throw new AnalysisException("Must use a map() function for options") - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ccc5b9043a0aa..c215735ab1c98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -21,7 +21,6 @@ import java.util.Locale import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion} -import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.TreeNode @@ -283,31 +282,6 @@ trait RuntimeReplaceable extends UnaryExpression with Unevaluable { override lazy val canonicalized: Expression = child.canonicalized } -/** - * An aggregate expression that gets rewritten (currently by the optimizer) into a - * different aggregate expression for evaluation. This is mainly used to provide compatibility - * with other databases. For example, we use this to support every, any/some aggregates by rewriting - * them with Min and Max respectively. - */ -trait UnevaluableAggregate extends DeclarativeAggregate { - - override def nullable: Boolean = true - - override lazy val aggBufferAttributes = - throw new UnsupportedOperationException(s"Cannot evaluate aggBufferAttributes: $this") - - override lazy val initialValues: Seq[Expression] = - throw new UnsupportedOperationException(s"Cannot evaluate initialValues: $this") - - override lazy val updateExpressions: Seq[Expression] = - throw new UnsupportedOperationException(s"Cannot evaluate updateExpressions: $this") - - override lazy val mergeExpressions: Seq[Expression] = - throw new UnsupportedOperationException(s"Cannot evaluate mergeExpressions: $this") - - override lazy val evaluateExpression: Expression = - throw new UnsupportedOperationException(s"Cannot evaluate evaluateExpression: $this") -} /** * Expressions that don't have SQL representation should extend this trait. Examples are diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index fae90caebf96c..8954fe8a58e6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types.DataType @@ -31,9 +31,6 @@ import org.apache.spark.sql.types.DataType * null. Use boxed type or [[Option]] if you wanna do the null-handling yourself. * @param dataType Return type of function. * @param children The input expressions of this UDF. - * @param inputsNullSafe Whether the inputs are of non-primitive types or not nullable. Null values - * of Scala primitive types will be converted to the type's default value and - * lead to wrong results, thus need special handling before calling the UDF. * @param inputTypes The expected input types of this UDF, used to perform type coercion. If we do * not want to perform coercion, simply use "Nil". Note that it would've been * better to use Option of Seq[DataType] so we can use "None" as the case for no @@ -42,16 +39,17 @@ import org.apache.spark.sql.types.DataType * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result * each time it is invoked with a particular input. + * @param nullableTypes which of the inputTypes are nullable (i.e. not primitive) */ case class ScalaUDF( function: AnyRef, dataType: DataType, children: Seq[Expression], - inputsNullSafe: Seq[Boolean], inputTypes: Seq[DataType] = Nil, udfName: Option[String] = None, nullable: Boolean = true, - udfDeterministic: Boolean = true) + udfDeterministic: Boolean = true, + nullableTypes: Seq[Boolean] = Nil) extends Expression with ImplicitCastInputTypes with NonSQLExpression with UserDefinedExpression { // The constructor for SPARK 2.1 and 2.2 @@ -62,8 +60,8 @@ case class ScalaUDF( inputTypes: Seq[DataType], udfName: Option[String]) = { this( - function, dataType, children, ScalaReflection.getParameterTypeNullability(function), - inputTypes, udfName, nullable = true, udfDeterministic = true) + function, dataType, children, inputTypes, udfName, nullable = true, + udfDeterministic = true, nullableTypes = Nil) } override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index 8dd80dc06ab2a..5ecb77be5965e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -23,21 +23,9 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns the mean calculated from values of a group.") -case class Average(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes { - - override def prettyName: String = "avg" - - override def children: Seq[Expression] = child :: Nil - - override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) - - override def checkInputDataTypes(): TypeCheckResult = - TypeUtils.checkForNumericExpr(child.dataType, "function average") +abstract class AverageLike(child: Expression) extends DeclarativeAggregate { override def nullable: Boolean = true - // Return data type. override def dataType: DataType = resultType @@ -75,11 +63,28 @@ case class Average(child: Expression) extends DeclarativeAggregate with Implicit sum.cast(resultType) / count.cast(resultType) } - override lazy val updateExpressions: Seq[Expression] = Seq( + protected def updateExpressionsDef: Seq[Expression] = Seq( /* sum = */ Add( sum, coalesce(child.cast(sumDataType), Literal(0).cast(sumDataType))), /* count = */ If(child.isNull, count, count + 1L) ) + + override lazy val updateExpressions = updateExpressionsDef +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the mean calculated from values of a group.") +case class Average(child: Expression) + extends AverageLike(child) with ImplicitCastInputTypes { + + override def prettyName: String = "avg" + + override def children: Seq[Expression] = child :: Nil + + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) + + override def checkInputDataTypes(): TypeCheckResult = + TypeUtils.checkForNumericExpr(child.dataType, "function average") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala index d402f2d592b44..40582d0abd762 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala @@ -21,17 +21,10 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = """ - _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. - - _FUNC_(expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are all non-null. - - _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. - """) -// scalastyle:on line.size.limit -case class Count(children: Seq[Expression]) extends DeclarativeAggregate { +/** + * Base class for all counting aggregators. + */ +abstract class CountLike extends DeclarativeAggregate { override def nullable: Boolean = false // Return data type. @@ -52,6 +45,19 @@ case class Count(children: Seq[Expression]) extends DeclarativeAggregate { override lazy val evaluateExpression = count override def defaultResult: Option[Literal] = Option(Literal(0L)) +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. + + _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-null. + + _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. + """) +// scalastyle:on line.size.limit +case class Count(children: Seq[Expression]) extends CountLike { override lazy val updateExpressions = { val nullableChildren = children.filter(_.nullable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala deleted file mode 100644 index fc33ef919498b..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions.aggregate - -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -abstract class UnevaluableBooleanAggBase(arg: Expression) - extends UnevaluableAggregate with ImplicitCastInputTypes { - - override def children: Seq[Expression] = arg :: Nil - - override def dataType: DataType = BooleanType - - override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType) - - override def checkInputDataTypes(): TypeCheckResult = { - arg.dataType match { - case dt if dt != BooleanType => - TypeCheckResult.TypeCheckFailure(s"Input to function '$prettyName' should have been " + - s"${BooleanType.simpleString}, but it's [${arg.dataType.catalogString}].") - case _ => TypeCheckResult.TypeCheckSuccess - } - } -} - -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns true if all values of `expr` are true.", - since = "3.0.0") -case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Every" -} - -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.", - since = "3.0.0") -case class AnyAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Any" -} - -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.", - since = "3.0.0") -case class SomeAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { - override def nodeName: String = "Some" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala deleted file mode 100644 index e70296fe31292..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import com.univocity.parsers.csv.CsvParser - -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult -import org.apache.spark.sql.catalyst.csv._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -/** - * Converts a CSV input string to a [[StructType]] with the specified schema. - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(csvStr, schema[, options]) - Returns a struct value with the given `csvStr` and `schema`.", - examples = """ - Examples: - > SELECT _FUNC_('1, 0.8', 'a INT, b DOUBLE'); - {"a":1, "b":0.8} - > SELECT _FUNC_('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) - {"time":2015-08-26 00:00:00.0} - """, - since = "3.0.0") -// scalastyle:on line.size.limit -case class CsvToStructs( - schema: StructType, - options: Map[String, String], - child: Expression, - timeZoneId: Option[String] = None) - extends UnaryExpression - with TimeZoneAwareExpression - with CodegenFallback - with ExpectsInputTypes - with NullIntolerant { - - override def nullable: Boolean = child.nullable - - // The CSV input data might be missing certain fields. We force the nullability - // of the user-provided schema to avoid data corruptions. - val nullableSchema: StructType = schema.asNullable - - // Used in `FunctionRegistry` - def this(child: Expression, schema: Expression, options: Map[String, String]) = - this( - schema = ExprUtils.evalSchemaExpr(schema), - options = options, - child = child, - timeZoneId = None) - - def this(child: Expression, schema: Expression) = this(child, schema, Map.empty[String, String]) - - def this(child: Expression, schema: Expression, options: Expression) = - this( - schema = ExprUtils.evalSchemaExpr(schema), - options = ExprUtils.convertToMapData(options), - child = child, - timeZoneId = None) - - // This converts parsed rows to the desired output by the given schema. - @transient - lazy val converter = (rows: Iterator[InternalRow]) => { - if (rows.hasNext) { - val result = rows.next() - // CSV's parser produces one record only. - assert(!rows.hasNext) - result - } else { - throw new IllegalArgumentException("Expected one row from CSV parser.") - } - } - - @transient lazy val parser = { - val parsedOptions = new CSVOptions(options, columnPruning = true, timeZoneId.get) - val mode = parsedOptions.parseMode - if (mode != PermissiveMode && mode != FailFastMode) { - throw new AnalysisException(s"from_csv() doesn't support the ${mode.name} mode. " + - s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}.") - } - val actualSchema = - StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val rawParser = new UnivocityParser(actualSchema, actualSchema, parsedOptions) - new FailureSafeParser[String]( - input => Seq(rawParser.parse(input)), - mode, - nullableSchema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) - } - - override def dataType: DataType = nullableSchema - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { - copy(timeZoneId = Option(timeZoneId)) - } - - override def nullSafeEval(input: Any): Any = { - val csv = input.asInstanceOf[UTF8String].toString - converter(parser.parse(csv)) - } - - override def inputTypes: Seq[AbstractDataType] = StringType :: Nil - - override def prettyName: String = "from_csv" -} - -/** - * A function infers schema of CSV string. - */ -@ExpressionDescription( - usage = "_FUNC_(csv[, options]) - Returns schema in the DDL format of CSV string.", - examples = """ - Examples: - > SELECT _FUNC_('1,abc'); - struct<_c0:int,_c1:string> - """, - since = "3.0.0") -case class SchemaOfCsv( - child: Expression, - options: Map[String, String]) - extends UnaryExpression with CodegenFallback { - - def this(child: Expression) = this(child, Map.empty[String, String]) - - def this(child: Expression, options: Expression) = this( - child = child, - options = ExprUtils.convertToMapData(options)) - - override def dataType: DataType = StringType - - override def nullable: Boolean = false - - @transient - private lazy val csv = child.eval().asInstanceOf[UTF8String] - - override def checkInputDataTypes(): TypeCheckResult = child match { - case Literal(s, StringType) if s != null => super.checkInputDataTypes() - case _ => TypeCheckResult.TypeCheckFailure( - s"The input csv should be a string literal and not null; however, got ${child.sql}.") - } - - override def eval(v: InternalRow): Any = { - val parsedOptions = new CSVOptions(options, true, "UTC") - val parser = new CsvParser(parsedOptions.asParserSettings) - val row = parser.parseLine(csv.toString) - assert(row != null, "Parsed CSV record should not be null.") - - val header = row.zipWithIndex.map { case (_, index) => s"_c$index" } - val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) - val fieldTypes = CSVInferSchema.inferRowType(parsedOptions)(startType, row) - val st = StructType(CSVInferSchema.toStructFields(fieldTypes, header, parsedOptions)) - UTF8String.fromString(st.catalogString) - } - - override def prettyName: String = "schema_of_csv" -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index eafcb6161036e..f5297dde10ed6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -529,7 +529,7 @@ case class JsonToStructs( // Used in `FunctionRegistry` def this(child: Expression, schema: Expression, options: Map[String, String]) = this( - schema = ExprUtils.evalTypeExpr(schema), + schema = JsonExprUtils.evalSchemaExpr(schema), options = options, child = child, timeZoneId = None) @@ -538,8 +538,8 @@ case class JsonToStructs( def this(child: Expression, schema: Expression, options: Expression) = this( - schema = ExprUtils.evalTypeExpr(schema), - options = ExprUtils.convertToMapData(options), + schema = JsonExprUtils.evalSchemaExpr(schema), + options = JsonExprUtils.convertToMapData(options), child = child, timeZoneId = None) @@ -554,36 +554,18 @@ case class JsonToStructs( @transient lazy val converter = nullableSchema match { case _: StructType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null + (rows: Seq[InternalRow]) => if (rows.length == 1) rows.head else null case _: ArrayType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null + (rows: Seq[InternalRow]) => rows.head.getArray(0) case _: MapType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null + (rows: Seq[InternalRow]) => rows.head.getMap(0) } - val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - @transient lazy val parser = { - val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord) - val mode = parsedOptions.parseMode - if (mode != PermissiveMode && mode != FailFastMode) { - throw new IllegalArgumentException(s"from_json() doesn't support the ${mode.name} mode. " + - s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}.") - } - val rawParser = new JacksonParser(nullableSchema, parsedOptions, allowArrayAsStructs = false) - val createParser = CreateJacksonParser.utf8String _ - - val parserSchema = nullableSchema match { - case s: StructType => s - case other => StructType(StructField("value", other) :: Nil) - } - - new FailureSafeParser[UTF8String]( - input => rawParser.parse(input, createParser, identity[UTF8String]), - mode, - parserSchema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) - } + @transient + lazy val parser = + new JacksonParser( + nullableSchema, + new JSONOptions(options + ("mode" -> FailFastMode.name), timeZoneId.get)) override def dataType: DataType = nullableSchema @@ -591,7 +573,35 @@ case class JsonToStructs( copy(timeZoneId = Option(timeZoneId)) override def nullSafeEval(json: Any): Any = { - converter(parser.parse(json.asInstanceOf[UTF8String])) + // When input is, + // - `null`: `null`. + // - invalid json: `null`. + // - empty string: `null`. + // + // When the schema is array, + // - json array: `Array(Row(...), ...)` + // - json object: `Array(Row(...))` + // - empty json array: `Array()`. + // - empty json object: `Array(Row(null))`. + // + // When the schema is a struct, + // - json object/array with single element: `Row(...)` + // - json array with multiple elements: `null` + // - empty json array: `null`. + // - empty json object: `Row(null)`. + + // We need `null` if the input string is an empty string. `JacksonParser` can + // deal with this but produces `Nil`. + if (json.toString.trim.isEmpty) return null + + try { + converter(parser.parse( + json.asInstanceOf[UTF8String], + CreateJacksonParser.utf8String, + identity[UTF8String])) + } catch { + case _: BadRecordException => null + } } override def inputTypes: Seq[AbstractDataType] = StringType :: Nil @@ -600,8 +610,6 @@ case class JsonToStructs( case _: MapType => "entries" case _ => super.sql } - - override def prettyName: String = "from_json" } /** @@ -642,7 +650,7 @@ case class StructsToJson( def this(child: Expression) = this(Map.empty, child, None) def this(child: Expression, options: Expression) = this( - options = ExprUtils.convertToMapData(options), + options = JsonExprUtils.convertToMapData(options), child = child, timeZoneId = None) @@ -722,8 +730,6 @@ case class StructsToJson( override def nullSafeEval(value: Any): Any = converter(value) override def inputTypes: Seq[AbstractDataType] = TypeCollection(ArrayType, StructType) :: Nil - - override def prettyName: String = "to_json" } /** @@ -742,17 +748,13 @@ case class StructsToJson( case class SchemaOfJson( child: Expression, options: Map[String, String]) - extends UnaryExpression with CodegenFallback { + extends UnaryExpression with String2StringExpression with CodegenFallback { def this(child: Expression) = this(child, Map.empty[String, String]) def this(child: Expression, options: Expression) = this( child = child, - options = ExprUtils.convertToMapData(options)) - - override def dataType: DataType = StringType - - override def nullable: Boolean = false + options = JsonExprUtils.convertToMapData(options)) @transient private lazy val jsonOptions = new JSONOptions(options, "UTC") @@ -764,23 +766,39 @@ case class SchemaOfJson( factory } - @transient - private lazy val json = child.eval().asInstanceOf[UTF8String] - - override def checkInputDataTypes(): TypeCheckResult = child match { - case Literal(s, StringType) if s != null => super.checkInputDataTypes() - case _ => TypeCheckResult.TypeCheckFailure( - s"The input json should be a string literal and not null; however, got ${child.sql}.") - } - - override def eval(v: InternalRow): Any = { - val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => + override def convert(v: UTF8String): UTF8String = { + val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser => parser.nextToken() inferField(parser, jsonOptions) } UTF8String.fromString(dt.catalogString) } +} + +object JsonExprUtils { + + def evalSchemaExpr(exp: Expression): DataType = exp match { + case Literal(s, StringType) => DataType.fromDDL(s.toString) + case e @ SchemaOfJson(_: Literal, _) => + val ddlSchema = e.eval().asInstanceOf[UTF8String] + DataType.fromDDL(ddlSchema.toString) + case e => throw new AnalysisException( + "Schema should be specified in DDL format as a string literal" + + s" or output of the schema_of_json function instead of ${e.sql}") + } - override def prettyName: String = "schema_of_json" + def convertToMapData(exp: Expression): Map[String, String] = exp match { + case m: CreateMap + if m.dataType.acceptsType(MapType(StringType, StringType, valueContainsNull = false)) => + val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] + ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => + key.toString -> value.toString + } + case m: CreateMap => + throw new AnalysisException( + s"A type of keys and values in map() must be string, but got ${m.dataType.catalogString}") + case _ => + throw new AnalysisException("Must use a map() function for options") + } } 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 34d252886ffb0..2bcbb92f1a469 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 @@ -40,10 +40,9 @@ import org.json4s.JsonAST._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData} +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ -import org.apache.spark.util.Utils object Literal { val TrueLiteral: Literal = Literal(true, BooleanType) @@ -197,47 +196,6 @@ object Literal { case other => throw new RuntimeException(s"no default for type $dataType") } - - private[expressions] def validateLiteralValue(value: Any, dataType: DataType): Unit = { - def doValidate(v: Any, dataType: DataType): Boolean = dataType match { - case _ if v == null => true - case BooleanType => v.isInstanceOf[Boolean] - case ByteType => v.isInstanceOf[Byte] - case ShortType => v.isInstanceOf[Short] - case IntegerType | DateType => v.isInstanceOf[Int] - case LongType | TimestampType => v.isInstanceOf[Long] - case FloatType => v.isInstanceOf[Float] - case DoubleType => v.isInstanceOf[Double] - case _: DecimalType => v.isInstanceOf[Decimal] - case CalendarIntervalType => v.isInstanceOf[CalendarInterval] - case BinaryType => v.isInstanceOf[Array[Byte]] - case StringType => v.isInstanceOf[UTF8String] - case st: StructType => - v.isInstanceOf[InternalRow] && { - val row = v.asInstanceOf[InternalRow] - st.fields.map(_.dataType).zipWithIndex.forall { - case (dt, i) => doValidate(row.get(i, dt), dt) - } - } - case at: ArrayType => - v.isInstanceOf[ArrayData] && { - val ar = v.asInstanceOf[ArrayData] - ar.numElements() == 0 || doValidate(ar.get(0, at.elementType), at.elementType) - } - case mt: MapType => - v.isInstanceOf[MapData] && { - val map = v.asInstanceOf[MapData] - doValidate(map.keyArray(), ArrayType(mt.keyType)) && - doValidate(map.valueArray(), ArrayType(mt.valueType)) - } - case ObjectType(cls) => cls.isInstance(v) - case udt: UserDefinedType[_] => doValidate(v, udt.sqlType) - case _ => false - } - require(doValidate(value, dataType), - s"Literal must have a corresponding value to ${dataType.catalogString}, " + - s"but class ${Utils.getSimpleName(value.getClass)} found.") - } } /** @@ -282,8 +240,6 @@ object DecimalLiteral { */ case class Literal (value: Any, dataType: DataType) extends LeafExpression { - Literal.validateLiteralValue(value, dataType) - override def foldable: Boolean = true override def nullable: Boolean = value == null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 4fd36a47cef52..3189e6841a525 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -30,13 +30,13 @@ import org.apache.spark.serializer._ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.ScalaReflection.universe.TermName -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedException} import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils /** @@ -962,32 +962,25 @@ case class MapObjects private( } } -/** - * Similar to [[UnresolvedMapObjects]], this is a placeholder of [[CatalystToExternalMap]]. - * - * @param child An expression that when evaluated returns a map object. - * @param keyFunction The function applied on the key collection elements. - * @param valueFunction The function applied on the value collection elements. - * @param collClass The type of the resulting collection. - */ -case class UnresolvedCatalystToExternalMap( - child: Expression, - @transient keyFunction: Expression => Expression, - @transient valueFunction: Expression => Expression, - collClass: Class[_]) extends UnaryExpression with Unevaluable { - - override lazy val resolved = false - - override def dataType: DataType = ObjectType(collClass) -} - object CatalystToExternalMap { private val curId = new java.util.concurrent.atomic.AtomicInteger() - def apply(u: UnresolvedCatalystToExternalMap): CatalystToExternalMap = { + /** + * Construct an instance of CatalystToExternalMap case class. + * + * @param keyFunction The function applied on the key collection elements. + * @param valueFunction The function applied on the value collection elements. + * @param inputData An expression that when evaluated returns a map object. + * @param collClass The type of the resulting collection. + */ + def apply( + keyFunction: Expression => Expression, + valueFunction: Expression => Expression, + inputData: Expression, + collClass: Class[_]): CatalystToExternalMap = { val id = curId.getAndIncrement() val keyLoopValue = s"CatalystToExternalMap_keyLoopValue$id" - val mapType = u.child.dataType.asInstanceOf[MapType] + val mapType = inputData.dataType.asInstanceOf[MapType] val keyLoopVar = LambdaVariable(keyLoopValue, "", mapType.keyType, nullable = false) val valueLoopValue = s"CatalystToExternalMap_valueLoopValue$id" val valueLoopIsNull = if (mapType.valueContainsNull) { @@ -997,9 +990,9 @@ object CatalystToExternalMap { } val valueLoopVar = LambdaVariable(valueLoopValue, valueLoopIsNull, mapType.valueType) CatalystToExternalMap( - keyLoopValue, u.keyFunction(keyLoopVar), - valueLoopValue, valueLoopIsNull, u.valueFunction(valueLoopVar), - u.child, u.collClass) + keyLoopValue, keyFunction(keyLoopVar), + valueLoopValue, valueLoopIsNull, valueFunction(valueLoopVar), + inputData, collClass) } } @@ -1096,9 +1089,15 @@ case class CatalystToExternalMap private( val tupleLoopValue = ctx.freshName("tupleLoopValue") val builderValue = ctx.freshName("builderValue") + val getLength = s"${genInputData.value}.numElements()" + val keyArray = ctx.freshName("keyArray") val valueArray = ctx.freshName("valueArray") + val getKeyArray = + s"${classOf[ArrayData].getName} $keyArray = ${genInputData.value}.keyArray();" val getKeyLoopVar = CodeGenerator.getValue(keyArray, inputDataType(mapType.keyType), loopIndex) + val getValueArray = + s"${classOf[ArrayData].getName} $valueArray = ${genInputData.value}.valueArray();" val getValueLoopVar = CodeGenerator.getValue( valueArray, inputDataType(mapType.valueType), loopIndex) @@ -1147,10 +1146,10 @@ case class CatalystToExternalMap private( ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${genInputData.isNull}) { - int $dataLength = ${genInputData.value}.numElements(); + int $dataLength = $getLength; $constructBuilder - ArrayData $keyArray = ${genInputData.value}.keyArray(); - ArrayData $valueArray = ${genInputData.value}.valueArray(); + $getKeyArray + $getValueArray int $loopIndex = 0; while ($loopIndex < $dataLength) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 16e0bc3aaf35b..2125340f38ee8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -129,13 +129,6 @@ case class Not(child: Expression) override def inputTypes: Seq[DataType] = Seq(BooleanType) - // +---------+-----------+ - // | CHILD | NOT CHILD | - // +---------+-----------+ - // | TRUE | FALSE | - // | FALSE | TRUE | - // | UNKNOWN | UNKNOWN | - // +---------+-----------+ protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -411,13 +404,6 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with override def sqlOperator: String = "AND" - // +---------+---------+---------+---------+ - // | AND | TRUE | FALSE | UNKNOWN | - // +---------+---------+---------+---------+ - // | TRUE | TRUE | FALSE | UNKNOWN | - // | FALSE | FALSE | FALSE | FALSE | - // | UNKNOWN | UNKNOWN | FALSE | UNKNOWN | - // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == false) { @@ -481,13 +467,6 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P override def sqlOperator: String = "OR" - // +---------+---------+---------+---------+ - // | OR | TRUE | FALSE | UNKNOWN | - // +---------+---------+---------+---------+ - // | TRUE | TRUE | TRUE | TRUE | - // | FALSE | TRUE | FALSE | UNKNOWN | - // | UNKNOWN | TRUE | UNKNOWN | UNKNOWN | - // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == true) { @@ -611,13 +590,6 @@ case class EqualTo(left: Expression, right: Expression) override def symbol: String = "=" - // +---------+---------+---------+---------+ - // | = | TRUE | FALSE | UNKNOWN | - // +---------+---------+---------+---------+ - // | TRUE | TRUE | FALSE | UNKNOWN | - // | FALSE | FALSE | TRUE | UNKNOWN | - // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | - // +---------+---------+---------+---------+ protected override def nullSafeEval(left: Any, right: Any): Any = ordering.equiv(left, right) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -655,13 +627,6 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp override def nullable: Boolean = false - // +---------+---------+---------+---------+ - // | <=> | TRUE | FALSE | UNKNOWN | - // +---------+---------+---------+---------+ - // | TRUE | TRUE | FALSE | UNKNOWN | - // | FALSE | FALSE | TRUE | UNKNOWN | - // | UNKNOWN | UNKNOWN | UNKNOWN | TRUE | - // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) val input2 = right.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 4f5ea1e95f833..bf0c35fe61018 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -157,7 +157,7 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi arguments = """ Arguments: * str - a string expression - * regexp - a string expression. The regex string should be a Java regular expression. + * regexp - a string expression. The pattern string should be a Java regular expression. Since Spark 2.0, string literals (including regex patterns) are unescaped in our SQL parser. For example, to match "\abc", a regular expression for `regexp` can be @@ -229,53 +229,33 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress /** - * Splits str around matches of the given regex. + * Splits str around pat (pattern is a regular expression). */ @ExpressionDescription( - usage = "_FUNC_(str, regex, limit) - Splits `str` around occurrences that match `regex`" + - " and returns an array with a length of at most `limit`", - arguments = """ - Arguments: - * str - a string expression to split. - * regex - a string representing a regular expression. The regex string should be a - Java regular expression. - * limit - an integer expression which controls the number of times the regex is applied. - * limit > 0: The resulting array's length will not be more than `limit`, - and the resulting array's last entry will contain all input - beyond the last matched regex. - * limit <= 0: `regex` will be applied as many times as possible, and - the resulting array can be of any size. - """, + usage = "_FUNC_(str, regex) - Splits `str` around occurrences that match `regex`.", examples = """ Examples: > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]'); ["one","two","three",""] - > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', -1); - ["one","two","three",""] - > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', 2); - ["one","twoBthreeC"] """) -case class StringSplit(str: Expression, regex: Expression, limit: Expression) - extends TernaryExpression with ImplicitCastInputTypes { +case class StringSplit(str: Expression, pattern: Expression) + extends BinaryExpression with ImplicitCastInputTypes { + override def left: Expression = str + override def right: Expression = pattern override def dataType: DataType = ArrayType(StringType) - override def inputTypes: Seq[DataType] = Seq(StringType, StringType, IntegerType) - override def children: Seq[Expression] = str :: regex :: limit :: Nil - - def this(exp: Expression, regex: Expression) = this(exp, regex, Literal(-1)); + override def inputTypes: Seq[DataType] = Seq(StringType, StringType) - override def nullSafeEval(string: Any, regex: Any, limit: Any): Any = { - val strings = string.asInstanceOf[UTF8String].split( - regex.asInstanceOf[UTF8String], limit.asInstanceOf[Int]) + override def nullSafeEval(string: Any, regex: Any): Any = { + val strings = string.asInstanceOf[UTF8String].split(regex.asInstanceOf[UTF8String], -1) new GenericArrayData(strings.asInstanceOf[Array[Any]]) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val arrayClass = classOf[GenericArrayData].getName - nullSafeCodeGen(ctx, ev, (str, regex, limit) => { + nullSafeCodeGen(ctx, ev, (str, pattern) => // Array in java is covariant, so we don't need to cast UTF8String[] to Object[]. - s"""${ev.value} = new $arrayClass($str.split($regex,$limit));""".stripMargin - }) + s"""${ev.value} = new $arrayClass($str.split($pattern, -1));""") } override def prettyName: String = "split" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index 0b674d025d1ac..707f312499734 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -206,7 +206,7 @@ case class SpecifiedWindowFrame( // Check combination (of expressions). (lower, upper) match { case (l: Expression, u: Expression) if !isValidFrameBoundary(l, u) => - TypeCheckFailure(s"Window frame upper bound '$upper' does not follow the lower bound " + + TypeCheckFailure(s"Window frame upper bound '$upper' does not followes the lower bound " + s"'$lower'.") case (l: SpecialFrameBoundary, _) => TypeCheckSuccess case (_, u: SpecialFrameBoundary) => TypeCheckSuccess @@ -242,12 +242,8 @@ case class SpecifiedWindowFrame( case e: Expression => e.sql + " FOLLOWING" } - // Check whether the left boundary value is greater than the right boundary value. It's required - // that the both expressions have the same data type. - // Since CalendarIntervalType is not comparable, we only compare expressions that are AtomicType. - private def isGreaterThan(l: Expression, r: Expression): Boolean = l.dataType match { - case _: AtomicType => GreaterThan(l, r).eval().asInstanceOf[Boolean] - case _ => false + private def isGreaterThan(l: Expression, r: Expression): Boolean = { + GreaterThan(l, r).eval().asInstanceOf[Boolean] } private def checkBoundary(b: Expression, location: String): TypeCheckResult = b match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index 57c7f2faf3107..984979ac5e9b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -38,8 +38,7 @@ import org.apache.spark.util.Utils */ class JacksonParser( schema: DataType, - val options: JSONOptions, - allowArrayAsStructs: Boolean) extends Logging { + val options: JSONOptions) extends Logging { import JacksonUtils._ import com.fasterxml.jackson.core.JsonToken._ @@ -85,7 +84,7 @@ class JacksonParser( // List([str_a_1,null]) // List([str_a_2,null], [null,str_b_3]) // - case START_ARRAY if allowArrayAsStructs => + case START_ARRAY => val array = convertArray(parser, elementConverter) // Here, as we support reading top level JSON arrays and take every element // in such an array as a row, this case is possible. @@ -94,8 +93,6 @@ class JacksonParser( } else { array.toArray[InternalRow](schema).toSeq } - case START_ARRAY => - throw new RuntimeException("Parsing JSON arrays as structs is forbidden.") } } @@ -171,7 +168,7 @@ class JacksonParser( case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => parser.getFloatValue - case VALUE_STRING if parser.getTextLength >= 1 => + case VALUE_STRING => // Special case handling for NaN and Infinity. parser.getText match { case "NaN" => Float.NaN @@ -187,7 +184,7 @@ class JacksonParser( case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => parser.getDoubleValue - case VALUE_STRING if parser.getTextLength >= 1 => + case VALUE_STRING => // Special case handling for NaN and Infinity. parser.getText match { case "NaN" => Double.NaN @@ -214,7 +211,7 @@ class JacksonParser( case TimestampType => (parser: JsonParser) => parseJsonToken[java.lang.Long](parser, dataType) { - case VALUE_STRING if parser.getTextLength >= 1 => + case VALUE_STRING => val stringValue = parser.getText // This one will lose microseconds parts. // See https://issues.apache.org/jira/browse/SPARK-10681. @@ -233,7 +230,7 @@ class JacksonParser( case DateType => (parser: JsonParser) => parseJsonToken[java.lang.Integer](parser, dataType) { - case VALUE_STRING if parser.getTextLength >= 1 => + case VALUE_STRING => val stringValue = parser.getText // This one will lose microseconds parts. // See https://issues.apache.org/jira/browse/SPARK-10681.x @@ -313,17 +310,16 @@ class JacksonParser( } /** - * This function throws an exception for failed conversion. For empty string on data types - * except for string and binary types, this also throws an exception. + * This function throws an exception for failed conversion, but returns null for empty string, + * to guard the non string types. */ private def failedConversion[R >: Null]( parser: JsonParser, dataType: DataType): PartialFunction[JsonToken, R] = { - - // SPARK-25040: Disallow empty strings for data types except for string and binary types. case VALUE_STRING if parser.getTextLength < 1 => - throw new RuntimeException( - s"Failed to parse an empty string for data type ${dataType.catalogString}") + // If conversion is failed, this produces `null` rather than throwing exception. + // This will protect the mismatch of types. + null case token => // We cannot parse this token based on the given data type. So, we throw a 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 a330a84a3a24f..da8009d50b5ec 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 @@ -84,7 +84,6 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) SimplifyConditionals, RemoveDispensableExpressions, SimplifyBinaryComparison, - ReplaceNullWithFalse, PruneFilters, EliminateSorts, SimplifyCasts, @@ -531,6 +530,9 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper * p2 is usually inserted by this rule and useless, p1 could prune the columns anyway. */ object ColumnPruning extends Rule[LogicalPlan] { + private def sameOutput(output1: Seq[Attribute], output2: Seq[Attribute]): Boolean = + output1.size == output2.size && + output1.zip(output2).forall(pair => pair._1.semanticEquals(pair._2)) def apply(plan: LogicalPlan): LogicalPlan = removeProjectBeforeFilter(plan transform { // Prunes the unused columns from project list of Project/Aggregate/Expand @@ -605,7 +607,7 @@ object ColumnPruning extends Rule[LogicalPlan] { case w: Window if w.windowExpressions.isEmpty => w.child // Eliminate no-op Projects - case p @ Project(_, child) if child.sameOutput(p) => child + case p @ Project(_, child) if sameOutput(child.output, p.output) => child // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index b9468007cac61..4448ace7105a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -95,7 +95,7 @@ import org.apache.spark.sql.types.IntegerType * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and - * exchange operators. Keeping the number of distinct groups as low as possible should be priority, + * exchange operators. Keeping the number of distinct groups as low a possible should be priority, * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ @@ -241,7 +241,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), a.child) - // Construct the first aggregate operator. This de-duplicates all the children of + // Construct the first aggregate operator. This de-duplicates the all the children of // distinct operators, and applies the regular aggregate operators. val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 2b29b49d00ab9..f8037588fa71e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,37 +276,15 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - // The following optimizations are applicable only when the operands are not nullable, - // since the three-value logic of AND and OR are different in NULL handling. - // See the chart: - // +---------+---------+---------+---------+ - // | operand | operand | OR | AND | - // +---------+---------+---------+---------+ - // | TRUE | TRUE | TRUE | TRUE | - // | TRUE | FALSE | TRUE | FALSE | - // | FALSE | FALSE | FALSE | FALSE | - // | UNKNOWN | TRUE | TRUE | UNKNOWN | - // | UNKNOWN | FALSE | UNKNOWN | FALSE | - // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | - // +---------+---------+---------+---------+ - - // (NULL And (NULL Or FALSE)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. - case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) - // (NULL And (FALSE Or NULL)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. - case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) - // ((NULL Or FALSE) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. - case (a Or b) And c if !c.nullable && a.semanticEquals(Not(c)) => And(b, c) - // ((FALSE Or NULL) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. - case (a Or b) And c if !c.nullable && b.semanticEquals(Not(c)) => And(a, c) - - // (NULL Or (NULL And TRUE)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. - case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) - // (NULL Or (TRUE And NULL)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. - case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) - // ((NULL And TRUE) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. - case (a And b) Or c if !c.nullable && a.semanticEquals(Not(c)) => Or(b, c) - // ((TRUE And NULL) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. - case (a And b) Or c if !c.nullable && b.semanticEquals(Not(c)) => Or(a, c) + 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) => @@ -736,60 +714,3 @@ object CombineConcats extends Rule[LogicalPlan] { flattenConcats(concat) } } - -/** - * A rule that replaces `Literal(null, _)` with `FalseLiteral` for further optimizations. - * - * This rule applies to conditions in [[Filter]] and [[Join]]. Moreover, it transforms predicates - * in all [[If]] expressions as well as branch conditions in all [[CaseWhen]] expressions. - * - * For example, `Filter(Literal(null, _))` is equal to `Filter(FalseLiteral)`. - * - * Another example containing branches is `Filter(If(cond, FalseLiteral, Literal(null, _)))`; - * this can be optimized to `Filter(If(cond, FalseLiteral, FalseLiteral))`, and eventually - * `Filter(FalseLiteral)`. - * - * As this rule is not limited to conditions in [[Filter]] and [[Join]], arbitrary plans can - * benefit from it. For example, `Project(If(And(cond, Literal(null)), Literal(1), Literal(2)))` - * can be simplified into `Project(Literal(2))`. - * - * As a result, many unnecessary computations can be removed in the query optimization phase. - */ -object ReplaceNullWithFalse extends Rule[LogicalPlan] { - - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(cond, _) => f.copy(condition = replaceNullWithFalse(cond)) - case j @ Join(_, _, _, Some(cond)) => j.copy(condition = Some(replaceNullWithFalse(cond))) - case p: LogicalPlan => p transformExpressions { - case i @ If(pred, _, _) => i.copy(predicate = replaceNullWithFalse(pred)) - case cw @ CaseWhen(branches, _) => - val newBranches = branches.map { case (cond, value) => - replaceNullWithFalse(cond) -> value - } - cw.copy(branches = newBranches) - } - } - - /** - * Recursively replaces `Literal(null, _)` with `FalseLiteral`. - * - * Note that `transformExpressionsDown` can not be used here as we must stop as soon as we hit - * an expression that is not [[CaseWhen]], [[If]], [[And]], [[Or]] or `Literal(null, _)`. - */ - private def replaceNullWithFalse(e: Expression): Expression = e match { - case cw: CaseWhen if cw.dataType == BooleanType => - val newBranches = cw.branches.map { case (cond, value) => - replaceNullWithFalse(cond) -> replaceNullWithFalse(value) - } - val newElseValue = cw.elseValue.map(replaceNullWithFalse) - CaseWhen(newBranches, newElseValue) - case i @ If(pred, trueVal, falseVal) if i.dataType == BooleanType => - If(replaceNullWithFalse(pred), replaceNullWithFalse(trueVal), replaceNullWithFalse(falseVal)) - case And(left, right) => - And(replaceNullWithFalse(left), replaceNullWithFalse(right)) - case Or(left, right) => - Or(replaceNullWithFalse(left), replaceNullWithFalse(right)) - case Literal(null, _) => FalseLiteral - case _ => e - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index fe196ec7c9d54..af0837e36e8ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -21,7 +21,6 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -29,24 +28,13 @@ import org.apache.spark.sql.types._ /** - * Finds all the expressions that are unevaluable and replace/rewrite them with semantically - * equivalent expressions that can be evaluated. Currently we replace two kinds of expressions: - * 1) [[RuntimeReplaceable]] expressions - * 2) [[UnevaluableAggregate]] expressions such as Every, Some, Any - * This is mainly used to provide compatibility with other databases. - * Few examples are: - * we use this to support "nvl" by replacing it with "coalesce". - * we use this to replace Every and Any with Min and Max respectively. - * - * TODO: In future, explore an option to replace aggregate functions similar to - * how RruntimeReplaceable does. + * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can + * be evaluated. This is mainly used to provide compatibility with other databases. + * For example, we use this to support "nvl" by replacing it with "coalesce". */ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child - case SomeAgg(arg) => Max(arg) - case AnyAgg(arg) => Max(arg) - case EveryAgg(arg) => Min(arg) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 672bffcfc0cad..da12a6519bd28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -394,17 +394,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Filter(expression(ctx), plan) } - def withHaving(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = { - // Note that we add a cast to non-predicate expressions. If the expression itself is - // already boolean, the optimizer will get rid of the unnecessary cast. - val predicate = expression(ctx) match { - case p: Predicate => p - case e => Cast(e, BooleanType) - } - Filter(predicate, plan) - } - - // Expressions. val expressions = Option(namedExpressionSeq).toSeq .flatMap(_.namedExpression.asScala) @@ -457,34 +446,30 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case e: NamedExpression => e case e: Expression => UnresolvedAlias(e) } - - def createProject() = if (namedExpressions.nonEmpty) { + val withProject = if (aggregation != null) { + withAggregation(aggregation, namedExpressions, withFilter) + } else if (namedExpressions.nonEmpty) { Project(namedExpressions, withFilter) } else { withFilter } - val withProject = if (aggregation == null && having != null) { - if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { - // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. - withHaving(having, createProject()) - } else { - // According to SQL standard, HAVING without GROUP BY means global aggregate. - withHaving(having, Aggregate(Nil, namedExpressions, withFilter)) + // Having + val withHaving = withProject.optional(having) { + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(having) match { + case p: Predicate => p + case e => Cast(e, BooleanType) } - } else if (aggregation != null) { - val aggregate = withAggregation(aggregation, namedExpressions, withFilter) - aggregate.optionalMap(having)(withHaving) - } else { - // When hitting this branch, `having` must be null. - createProject() + Filter(predicate, withProject) } // Distinct val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) { - Distinct(withProject) + Distinct(withHaving) } else { - withProject + withHaving } // Window @@ -716,7 +701,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging // Resolve the join type and join condition val (joinType, condition) = Option(join.joinCriteria) match { case Some(c) if c.USING != null => - (UsingJoin(baseJoinType, visitIdentifierList(c.identifierList)), None) + (UsingJoin(baseJoinType, c.identifier.asScala.map(_.getText)), None) case Some(c) if c.booleanExpression != null => (baseJoinType, Option(expression(c.booleanExpression))) case None if join.NATURAL != null => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index a520eba001af1..5f136629eb15b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -130,20 +130,6 @@ abstract class LogicalPlan * Returns the output ordering that this plan generates. */ def outputOrdering: Seq[SortOrder] = Nil - - /** - * Returns true iff `other`'s output is semantically the same, ie.: - * - it contains the same number of `Attribute`s; - * - references are the same; - * - the order is equal too. - */ - def sameOutput(other: LogicalPlan): Boolean = { - val thisOutput = this.output - val otherOutput = other.output - thisOutput.length == otherOutput.length && thisOutput.zip(otherOutput).forall { - case (a1, a2) => a1.semanticEquals(a2) - } - } } /** @@ -166,10 +152,10 @@ abstract class UnaryNode extends LogicalPlan { override final def children: Seq[LogicalPlan] = child :: Nil /** - * Generates all valid constraints including an set of aliased constraints by replacing the - * original constraint expressions with the corresponding alias + * Generates an additional set of aliased constraints by replacing the original constraint + * expressions with the corresponding alias */ - protected def getAllValidConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { + protected def getAliasedConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { var allConstraints = child.constraints.asInstanceOf[Set[Expression]] projectList.foreach { case a @ Alias(l: Literal, _) => @@ -184,7 +170,7 @@ abstract class UnaryNode extends LogicalPlan { case _ => // Don't change. } - allConstraints + allConstraints -- child.constraints } override protected def validConstraints: Set[Expression] = child.constraints diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index f09c5ceefed13..7ff83a9be3622 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -64,7 +64,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) } override def validConstraints: Set[Expression] = - getAllValidConstraints(projectList) + child.constraints.union(getAliasedConstraints(projectList)) } /** @@ -595,7 +595,7 @@ case class Aggregate( override def validConstraints: Set[Expression] = { val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) - getAllValidConstraints(nonAgg) + child.constraints.union(getAliasedConstraints(nonAgg)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8fc2b3236ff20..e03d66438af27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -414,8 +414,7 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_DATE_ENABLED = buildConf("spark.sql.parquet.filterPushdown.date") .doc("If true, enables Parquet filter push-down optimization for Date. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + - "enabled.") + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") .internal() .booleanConf .createWithDefault(true) @@ -423,7 +422,7 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED = buildConf("spark.sql.parquet.filterPushdown.timestamp") .doc("If true, enables Parquet filter push-down optimization for Timestamp. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is " + "enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.") .internal() .booleanConf @@ -432,8 +431,7 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED = buildConf("spark.sql.parquet.filterPushdown.decimal") .doc("If true, enables Parquet filter push-down optimization for Decimal. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + - "enabled.") + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") .internal() .booleanConf .createWithDefault(true) @@ -441,8 +439,7 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED = buildConf("spark.sql.parquet.filterPushdown.string.startsWith") .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + - "enabled.") + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") .internal() .booleanConf .createWithDefault(true) @@ -453,8 +450,7 @@ object SQLConf { "Large threshold won't necessarily provide much better performance. " + "The experiment argued that 300 is the limit threshold. " + "By setting this value to 0 this feature can be disabled. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + - "enabled.") + "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") .internal() .intConf .checkValue(threshold => threshold >= 0, "The threshold must not be negative.") @@ -469,6 +465,13 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") + .doc("If true, enables Parquet's native record-level filtering using the pushed down " + + "filters. This configuration only has an effect when 'spark.sql.parquet.filterPushdown' " + + "is enabled.") + .booleanConf + .createWithDefault(false) + val PARQUET_OUTPUT_COMMITTER_CLASS = buildConf("spark.sql.parquet.output.committer.class") .doc("The output committer class used by Parquet. The specified class needs to be a " + "subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, it's also a subclass " + @@ -484,15 +487,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") - .doc("If true, enables Parquet's native record-level filtering using the pushed down " + - "filters. " + - s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' " + - "is enabled and the vectorized reader is not used. You can ensure the vectorized reader " + - s"is not used by setting '${PARQUET_VECTORIZED_READER_ENABLED.key}' to false.") - .booleanConf - .createWithDefault(false) - val PARQUET_PARTITION_PRUNING_ENABLED = buildConf("spark.sql.parquet.enablePartitionPruning") .doc("Enables driver-side partition pruning for Parquet.") .booleanConf @@ -658,7 +652,7 @@ object SQLConf { .internal() .doc("When true, a table created by a Hive CTAS statement (no USING clause) " + "without specifying any storage property will be converted to a data source table, " + - s"using the data source set by ${DEFAULT_DATA_SOURCE_NAME.key}.") + "using the data source set by spark.sql.sources.default.") .booleanConf .createWithDefault(false) @@ -1124,7 +1118,7 @@ object SQLConf { val DEFAULT_SIZE_IN_BYTES = buildConf("spark.sql.defaultSizeInBytes") .internal() .doc("The default table size used in query planning. By default, it is set to Long.MaxValue " + - s"which is larger than `${AUTO_BROADCASTJOIN_THRESHOLD.key}` to be more conservative. " + + "which is larger than `spark.sql.autoBroadcastJoinThreshold` to be more conservative. " + "That is to say by default the optimizer will not choose to broadcast a table unless it " + "knows for sure its size is small enough.") .longConf @@ -1295,7 +1289,7 @@ object SQLConf { val ARROW_FALLBACK_ENABLED = buildConf("spark.sql.execution.arrow.fallback.enabled") - .doc(s"When true, optimizations enabled by '${ARROW_EXECUTION_ENABLED.key}' will " + + .doc("When true, optimizations enabled by 'spark.sql.execution.arrow.enabled' will " + "fallback automatically to non-optimized implementations if an error occurs.") .booleanConf .createWithDefault(true) @@ -1499,16 +1493,15 @@ object SQLConf { val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled") .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " + "displayed if and only if the REPL supports the eager evaluation. Currently, the " + - "eager evaluation is supported in PySpark and SparkR. In PySpark, for the notebooks like " + - "Jupyter, the HTML table (generated by _repr_html_) will be returned. For plain Python " + - "REPL, the returned outputs are formatted like dataframe.show(). In SparkR, the returned " + - "outputs are showed similar to R data.frame would.") + "eager evaluation is only supported in PySpark. For the notebooks like Jupyter, " + + "the HTML table (generated by _repr_html_) will be returned. For plain Python REPL, " + + "the returned outputs are formatted like dataframe.show().") .booleanConf .createWithDefault(false) val REPL_EAGER_EVAL_MAX_NUM_ROWS = buildConf("spark.sql.repl.eagerEval.maxNumRows") .doc("The max number of rows that are returned by eager evaluation. This only takes " + - s"effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true. The valid range of this " + + "effect when spark.sql.repl.eagerEval.enabled is set to true. The valid range of this " + "config is from 0 to (Int.MaxValue - 1), so the invalid config like negative and " + "greater than (Int.MaxValue - 1) will be normalized to 0 and (Int.MaxValue - 1).") .intConf @@ -1516,7 +1509,7 @@ object SQLConf { val REPL_EAGER_EVAL_TRUNCATE = buildConf("spark.sql.repl.eagerEval.truncate") .doc("The max number of characters for each cell that is returned by eager evaluation. " + - s"This only takes effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true.") + "This only takes effect when spark.sql.repl.eagerEval.enabled is set to true.") .intConf .createWithDefault(20) @@ -1585,14 +1578,6 @@ object SQLConf { .internal() .booleanConf .createWithDefault(false) - - val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE = - buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere") - .internal() - .doc("If it is set to true, the parser will treat HAVING without GROUP BY as a normal " + - "WHERE, which does not follow SQL standard.") - .booleanConf - .createWithDefault(false) } /** diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java index 8da778800bb9f..2da87113c6229 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java @@ -123,8 +123,9 @@ public void tearDown() { @Test public void emptyBatch() throws Exception { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { Assert.assertEquals(0, batch.numRows()); try { batch.getKeyRow(-1); @@ -151,24 +152,31 @@ public void emptyBatch() throws Exception { // Expected exception; do nothing. } Assert.assertFalse(batch.rowIterator().next()); + } finally { + batch.close(); } } @Test - public void batchType() { - try (RowBasedKeyValueBatch batch1 = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY); - RowBasedKeyValueBatch batch2 = RowBasedKeyValueBatch.allocate(fixedKeySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + public void batchType() throws Exception { + RowBasedKeyValueBatch batch1 = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + RowBasedKeyValueBatch batch2 = RowBasedKeyValueBatch.allocate(fixedKeySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { Assert.assertEquals(batch1.getClass(), VariableLengthRowBasedKeyValueBatch.class); Assert.assertEquals(batch2.getClass(), FixedLengthRowBasedKeyValueBatch.class); + } finally { + batch1.close(); + batch2.close(); } } @Test public void setAndRetrieve() { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { UnsafeRow ret1 = appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); Assert.assertTrue(checkValue(ret1, 1, 1)); UnsafeRow ret2 = appendRow(batch, makeKeyRow(2, "B"), makeValueRow(2, 2)); @@ -196,27 +204,33 @@ public void setAndRetrieve() { } catch (AssertionError e) { // Expected exception; do nothing. } + } finally { + batch.close(); } } @Test public void setUpdateAndRetrieve() { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); Assert.assertEquals(1, batch.numRows()); UnsafeRow retrievedValue = batch.getValueRow(0); updateValueRow(retrievedValue, 2, 2); UnsafeRow retrievedValue2 = batch.getValueRow(0); Assert.assertTrue(checkValue(retrievedValue2, 2, 2)); + } finally { + batch.close(); } } @Test public void iteratorTest() throws Exception { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { appendRow(batch, makeKeyRow(1, "A"), makeValueRow(1, 1)); appendRow(batch, makeKeyRow(2, "B"), makeValueRow(2, 2)); appendRow(batch, makeKeyRow(3, "C"), makeValueRow(3, 3)); @@ -239,13 +253,16 @@ public void iteratorTest() throws Exception { Assert.assertTrue(checkKey(key3, 3, "C")); Assert.assertTrue(checkValue(value3, 3, 3)); Assert.assertFalse(iterator.next()); + } finally { + batch.close(); } } @Test public void fixedLengthTest() throws Exception { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(fixedKeySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(fixedKeySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { appendRow(batch, makeKeyRow(11, 11), makeValueRow(1, 1)); appendRow(batch, makeKeyRow(22, 22), makeValueRow(2, 2)); appendRow(batch, makeKeyRow(33, 33), makeValueRow(3, 3)); @@ -276,13 +293,16 @@ public void fixedLengthTest() throws Exception { Assert.assertTrue(checkKey(key3, 33, 33)); Assert.assertTrue(checkValue(value3, 3, 3)); Assert.assertFalse(iterator.next()); + } finally { + batch.close(); } } @Test public void appendRowUntilExceedingCapacity() throws Exception { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, 10)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, 10); + try { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(1, 1); for (int i = 0; i < 10; i++) { @@ -301,6 +321,8 @@ public void appendRowUntilExceedingCapacity() throws Exception { Assert.assertTrue(checkValue(value1, 1, 1)); } Assert.assertFalse(iterator.next()); + } finally { + batch.close(); } } @@ -308,8 +330,9 @@ public void appendRowUntilExceedingCapacity() throws Exception { public void appendRowUntilExceedingPageSize() throws Exception { // Use default size or spark.buffer.pageSize if specified int pageSizeToUse = (int) memoryManager.pageSizeBytes(); - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, pageSizeToUse)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, pageSizeToUse); //enough capacity + try { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(1, 1); int recordLength = 8 + key.getSizeInBytes() + value.getSizeInBytes() + 8; @@ -333,44 +356,49 @@ public void appendRowUntilExceedingPageSize() throws Exception { Assert.assertTrue(checkValue(value1, 1, 1)); } Assert.assertFalse(iterator.next()); + } finally { + batch.close(); } } @Test public void failureToAllocateFirstPage() throws Exception { memoryManager.limit(1024); - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + try { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(11, 11); UnsafeRow ret = appendRow(batch, key, value); Assert.assertNull(ret); Assert.assertFalse(batch.rowIterator().next()); + } finally { + batch.close(); } } @Test public void randomizedTest() { - try (RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, DEFAULT_CAPACITY)) { - int numEntry = 100; - long[] expectedK1 = new long[numEntry]; - String[] expectedK2 = new String[numEntry]; - long[] expectedV1 = new long[numEntry]; - long[] expectedV2 = new long[numEntry]; - - for (int i = 0; i < numEntry; i++) { - long k1 = rand.nextLong(); - String k2 = getRandomString(rand.nextInt(256)); - long v1 = rand.nextLong(); - long v2 = rand.nextLong(); - appendRow(batch, makeKeyRow(k1, k2), makeValueRow(v1, v2)); - expectedK1[i] = k1; - expectedK2[i] = k2; - expectedV1[i] = v1; - expectedV2[i] = v2; - } - + RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, + valueSchema, taskMemoryManager, DEFAULT_CAPACITY); + int numEntry = 100; + long[] expectedK1 = new long[numEntry]; + String[] expectedK2 = new String[numEntry]; + long[] expectedV1 = new long[numEntry]; + long[] expectedV2 = new long[numEntry]; + + for (int i = 0; i < numEntry; i++) { + long k1 = rand.nextLong(); + String k2 = getRandomString(rand.nextInt(256)); + long v1 = rand.nextLong(); + long v2 = rand.nextLong(); + appendRow(batch, makeKeyRow(k1, k2), makeValueRow(v1, v2)); + expectedK1[i] = k1; + expectedK2[i] = k2; + expectedV1[i] = v1; + expectedV2[i] = v2; + } + try { for (int j = 0; j < 10000; j++) { int rowId = rand.nextInt(numEntry); if (rand.nextBoolean()) { @@ -382,6 +410,8 @@ public void randomizedTest() { Assert.assertTrue(checkValue(value, expectedV1[rowId], expectedV2[rowId])); } } + } finally { + batch.close(); } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala index 3b4b80daf0843..7a2a66c9b1d33 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection @@ -26,87 +26,94 @@ import org.apache.spark.sql.types._ /** * Benchmark for the previous interpreted hash function(InternalRow.hashCode) vs codegened * hash expressions (Murmur3Hash/xxHash64). - * To run this benchmark: - * {{{ - * 1. without sbt: - * bin/spark-submit --class --jars - * 2. build/sbt "catalyst/test:runMain " - * 3. generate result: - * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain " - * Results will be written to "benchmarks/HashBenchmark-results.txt". - * }}} */ -object HashBenchmark extends BenchmarkBase { +object HashBenchmark { def test(name: String, schema: StructType, numRows: Int, iters: Int): Unit = { - runBenchmark(name) { - val generator = RandomDataGenerator.forType(schema, nullable = false).get - val encoder = RowEncoder(schema) - val attrs = schema.toAttributes - val safeProjection = GenerateSafeProjection.generate(attrs, attrs) + val generator = RandomDataGenerator.forType(schema, nullable = false).get + val encoder = RowEncoder(schema) + val attrs = schema.toAttributes + val safeProjection = GenerateSafeProjection.generate(attrs, attrs) - val rows = (1 to numRows).map(_ => - // The output of encoder is UnsafeRow, use safeProjection to turn in into safe format. - safeProjection(encoder.toRow(generator().asInstanceOf[Row])).copy() - ).toArray + val rows = (1 to numRows).map(_ => + // The output of encoder is UnsafeRow, use safeProjection to turn in into safe format. + safeProjection(encoder.toRow(generator().asInstanceOf[Row])).copy() + ).toArray - val benchmark = new Benchmark("Hash For " + name, iters * numRows.toLong, output = output) - benchmark.addCase("interpreted version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += rows(i).hashCode() - i += 1 - } + val benchmark = new Benchmark("Hash For " + name, iters * numRows.toLong) + benchmark.addCase("interpreted version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += rows(i).hashCode() + i += 1 } } + } - val getHashCode = UnsafeProjection.create(new Murmur3Hash(attrs) :: Nil, attrs) - benchmark.addCase("codegen version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHashCode(rows(i)).getInt(0) - i += 1 - } + val getHashCode = UnsafeProjection.create(new Murmur3Hash(attrs) :: Nil, attrs) + benchmark.addCase("codegen version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHashCode(rows(i)).getInt(0) + i += 1 } } + } - val getHashCode64b = UnsafeProjection.create(new XxHash64(attrs) :: Nil, attrs) - benchmark.addCase("codegen version 64-bit") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHashCode64b(rows(i)).getInt(0) - i += 1 - } + val getHashCode64b = UnsafeProjection.create(new XxHash64(attrs) :: Nil, attrs) + benchmark.addCase("codegen version 64-bit") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHashCode64b(rows(i)).getInt(0) + i += 1 } } + } - val getHiveHashCode = UnsafeProjection.create(new HiveHash(attrs) :: Nil, attrs) - benchmark.addCase("codegen HiveHash version") { _: Int => - var sum = 0 - for (_ <- 0L until iters) { - var i = 0 - while (i < numRows) { - sum += getHiveHashCode(rows(i)).getInt(0) - i += 1 - } + val getHiveHashCode = UnsafeProjection.create(new HiveHash(attrs) :: Nil, attrs) + benchmark.addCase("codegen HiveHash version") { _: Int => + var sum = 0 + for (_ <- 0L until iters) { + var i = 0 + while (i < numRows) { + sum += getHiveHashCode(rows(i)).getInt(0) + i += 1 } } - - benchmark.run() } + + benchmark.run() } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + def main(args: Array[String]): Unit = { val singleInt = new StructType().add("i", IntegerType) + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash For single ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + interpreted version 3262 / 3267 164.6 6.1 1.0X + codegen version 6448 / 6718 83.3 12.0 0.5X + codegen version 64-bit 6088 / 6154 88.2 11.3 0.5X + codegen HiveHash version 4732 / 4745 113.5 8.8 0.7X + */ test("single ints", singleInt, 1 << 15, 1 << 14) val singleLong = new StructType().add("i", LongType) + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash For single longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + interpreted version 3716 / 3726 144.5 6.9 1.0X + codegen version 7706 / 7732 69.7 14.4 0.5X + codegen version 64-bit 6370 / 6399 84.3 11.9 0.6X + codegen HiveHash version 4924 / 5026 109.0 9.2 0.8X + */ test("single longs", singleLong, 1 << 15, 1 << 14) val normal = new StructType() @@ -124,18 +131,45 @@ object HashBenchmark extends BenchmarkBase { .add("binary", BinaryType) .add("date", DateType) .add("timestamp", TimestampType) + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash For normal: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + interpreted version 2985 / 3013 0.7 1423.4 1.0X + codegen version 2422 / 2434 0.9 1155.1 1.2X + codegen version 64-bit 856 / 920 2.5 408.0 3.5X + codegen HiveHash version 4501 / 4979 0.5 2146.4 0.7X + */ test("normal", normal, 1 << 10, 1 << 11) val arrayOfInt = ArrayType(IntegerType) val array = new StructType() .add("array", arrayOfInt) .add("arrayOfArray", ArrayType(arrayOfInt)) + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash For array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + interpreted version 3100 / 3555 0.0 23651.8 1.0X + codegen version 5779 / 5865 0.0 44088.4 0.5X + codegen version 64-bit 4738 / 4821 0.0 36151.7 0.7X + codegen HiveHash version 2200 / 2246 0.1 16785.9 1.4X + */ test("array", array, 1 << 8, 1 << 9) val mapOfInt = MapType(IntegerType, IntegerType) val map = new StructType() .add("map", mapOfInt) .add("mapOfMap", MapType(IntegerType, mapOfInt)) + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash For map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + interpreted version 0 / 0 48.1 20.8 1.0X + codegen version 257 / 275 0.0 62768.7 0.0X + codegen version 64-bit 226 / 240 0.0 55224.5 0.0X + codegen HiveHash version 89 / 96 0.0 21708.8 0.0X + */ test("map", map, 1 << 6, 1 << 6) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala index dbfa7bb18aa65..a60eb20d9edef 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala @@ -19,24 +19,15 @@ package org.apache.spark.sql import java.util.Random -import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.catalyst.expressions.{HiveHasher, XXH64} import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 /** * Synthetic benchmark for MurMurHash 3 and xxHash64. - * To run this benchmark: - * {{{ - * 1. without sbt: - * bin/spark-submit --class --jars - * 2. build/sbt "catalyst/test:runMain " - * 3. generate result: - * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "catalyst/test:runMain " - * Results will be written to "benchmarks/HashByteArrayBenchmark-results.txt". - * }}} */ -object HashByteArrayBenchmark extends BenchmarkBase { +object HashByteArrayBenchmark { def test(length: Int, seed: Long, numArrays: Int, iters: Int): Unit = { val random = new Random(seed) val arrays = Array.fill[Array[Byte]](numArrays) { @@ -45,8 +36,8 @@ object HashByteArrayBenchmark extends BenchmarkBase { bytes } - val benchmark = new Benchmark( - "Hash byte arrays with length " + length, iters * numArrays.toLong, output = output) + val benchmark = + new Benchmark("Hash byte arrays with length " + length, iters * numArrays.toLong) benchmark.addCase("Murmur3_x86_32") { _: Int => var sum = 0L for (_ <- 0L until iters) { @@ -83,17 +74,96 @@ object HashByteArrayBenchmark extends BenchmarkBase { benchmark.run() } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runBenchmark("Benchmark for MurMurHash 3 and xxHash64") { - test(8, 42L, 1 << 10, 1 << 11) - test(16, 42L, 1 << 10, 1 << 11) - test(24, 42L, 1 << 10, 1 << 11) - test(31, 42L, 1 << 10, 1 << 11) - test(64 + 31, 42L, 1 << 10, 1 << 11) - test(256 + 31, 42L, 1 << 10, 1 << 11) - test(1024 + 31, 42L, 1 << 10, 1 << 11) - test(2048 + 31, 42L, 1 << 10, 1 << 11) - test(8192 + 31, 42L, 1 << 10, 1 << 11) - } + def main(args: Array[String]): Unit = { + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 8: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 12 / 16 174.3 5.7 1.0X + xxHash 64-bit 17 / 22 120.0 8.3 0.7X + HiveHasher 13 / 15 162.1 6.2 0.9X + */ + test(8, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 16: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 19 / 22 107.6 9.3 1.0X + xxHash 64-bit 20 / 24 104.6 9.6 1.0X + HiveHasher 24 / 28 87.0 11.5 0.8X + */ + test(16, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 24: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 28 / 32 74.8 13.4 1.0X + xxHash 64-bit 24 / 29 87.3 11.5 1.2X + HiveHasher 36 / 41 57.7 17.3 0.8X + */ + test(24, 42L, 1 << 10, 1 << 11) + + // Add 31 to all arrays to create worse case alignment for xxHash. + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 31: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 41 / 45 51.1 19.6 1.0X + xxHash 64-bit 36 / 44 58.8 17.0 1.2X + HiveHasher 49 / 54 42.6 23.5 0.8X + */ + test(31, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 95: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 100 / 110 21.0 47.7 1.0X + xxHash 64-bit 74 / 78 28.2 35.5 1.3X + HiveHasher 189 / 196 11.1 90.3 0.5X + */ + test(64 + 31, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 287: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 299 / 311 7.0 142.4 1.0X + xxHash 64-bit 113 / 122 18.5 54.1 2.6X + HiveHasher 620 / 624 3.4 295.5 0.5X + */ + test(256 + 31, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 1055: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 1068 / 1070 2.0 509.1 1.0X + xxHash 64-bit 306 / 315 6.9 145.9 3.5X + HiveHasher 2316 / 2369 0.9 1104.3 0.5X + */ + test(1024 + 31, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 2079: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 2252 / 2274 0.9 1074.1 1.0X + xxHash 64-bit 534 / 580 3.9 254.6 4.2X + HiveHasher 4739 / 4786 0.4 2259.8 0.5X + */ + test(2048 + 31, 42L, 1 << 10, 1 << 11) + + /* + Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz + Hash byte arrays with length 8223: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Murmur3_x86_32 9249 / 9586 0.2 4410.5 1.0X + xxHash 64-bit 2897 / 3241 0.7 1381.6 3.2X + HiveHasher 19392 / 20211 0.1 9246.6 0.5X + */ + test(8192 + 31, 42L, 1 << 10, 1 << 11) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala index 42a4cfc91f826..e7a99485cdf04 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala @@ -41,7 +41,7 @@ object UnsafeProjectionBenchmark extends BenchmarkBase { (1 to numRows).map(_ => encoder.toRow(generator().asInstanceOf[Row]).copy()).toArray } - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + override def runBenchmarkSuite(): Unit = { runBenchmark("unsafe projection") { val iters = 1024 * 16 val numRows = 1024 * 16 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index d98589db323cc..f9ee948b97e0a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -19,13 +19,12 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} -import scala.reflect.runtime.universe.TypeTag - import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression, If, SpecificInternalRow, UpCast} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, Literal, SpecificInternalRow, UpCast} import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, NewInstance} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String case class PrimitiveData( intField: Int, @@ -113,14 +112,6 @@ object TestingUDT { class ScalaReflectionSuite extends SparkFunSuite { import org.apache.spark.sql.catalyst.ScalaReflection._ - // A helper method used to test `ScalaReflection.serializerForType`. - private def serializerFor[T: TypeTag]: Expression = - serializerForType(ScalaReflection.localTypeOf[T]) - - // A helper method used to test `ScalaReflection.deserializerForType`. - private def deserializerFor[T: TypeTag]: Expression = - deserializerForType(ScalaReflection.localTypeOf[T]) - test("SQLUserDefinedType annotation on Scala structure") { val schema = schemaFor[TestingUDT.NestedStruct] assert(schema === Schema( @@ -272,9 +263,13 @@ class ScalaReflectionSuite extends SparkFunSuite { test("SPARK-15062: Get correct serializer for List[_]") { val list = List(1, 2, 3) - val serializer = serializerFor[List[Int]] - assert(serializer.isInstanceOf[NewInstance]) - assert(serializer.asInstanceOf[NewInstance] + val serializer = serializerFor[List[Int]](BoundReference( + 0, ObjectType(list.getClass), nullable = false)) + assert(serializer.children.size == 2) + assert(serializer.children.head.isInstanceOf[Literal]) + assert(serializer.children.head.asInstanceOf[Literal].value === UTF8String.fromString("value")) + assert(serializer.children.last.isInstanceOf[NewInstance]) + assert(serializer.children.last.asInstanceOf[NewInstance] .cls.isAssignableFrom(classOf[org.apache.spark.sql.catalyst.util.GenericArrayData])) } @@ -285,58 +280,59 @@ class ScalaReflectionSuite extends SparkFunSuite { test("serialize and deserialize arbitrary sequence types") { import scala.collection.immutable.Queue - val queueSerializer = serializerFor[Queue[Int]] - assert(queueSerializer.dataType == + val queueSerializer = serializerFor[Queue[Int]](BoundReference( + 0, ObjectType(classOf[Queue[Int]]), nullable = false)) + assert(queueSerializer.dataType.head.dataType == ArrayType(IntegerType, containsNull = false)) val queueDeserializer = deserializerFor[Queue[Int]] assert(queueDeserializer.dataType == ObjectType(classOf[Queue[_]])) import scala.collection.mutable.ArrayBuffer - val arrayBufferSerializer = serializerFor[ArrayBuffer[Int]] - assert(arrayBufferSerializer.dataType == + val arrayBufferSerializer = serializerFor[ArrayBuffer[Int]](BoundReference( + 0, ObjectType(classOf[ArrayBuffer[Int]]), nullable = false)) + assert(arrayBufferSerializer.dataType.head.dataType == ArrayType(IntegerType, containsNull = false)) val arrayBufferDeserializer = deserializerFor[ArrayBuffer[Int]] assert(arrayBufferDeserializer.dataType == ObjectType(classOf[ArrayBuffer[_]])) } test("serialize and deserialize arbitrary map types") { - val mapSerializer = serializerFor[Map[Int, Int]] - assert(mapSerializer.dataType == + val mapSerializer = serializerFor[Map[Int, Int]](BoundReference( + 0, ObjectType(classOf[Map[Int, Int]]), nullable = false)) + assert(mapSerializer.dataType.head.dataType == MapType(IntegerType, IntegerType, valueContainsNull = false)) val mapDeserializer = deserializerFor[Map[Int, Int]] assert(mapDeserializer.dataType == ObjectType(classOf[Map[_, _]])) import scala.collection.immutable.HashMap - val hashMapSerializer = serializerFor[HashMap[Int, Int]] - assert(hashMapSerializer.dataType == + val hashMapSerializer = serializerFor[HashMap[Int, Int]](BoundReference( + 0, ObjectType(classOf[HashMap[Int, Int]]), nullable = false)) + assert(hashMapSerializer.dataType.head.dataType == MapType(IntegerType, IntegerType, valueContainsNull = false)) val hashMapDeserializer = deserializerFor[HashMap[Int, Int]] assert(hashMapDeserializer.dataType == ObjectType(classOf[HashMap[_, _]])) import scala.collection.mutable.{LinkedHashMap => LHMap} - val linkedHashMapSerializer = serializerFor[LHMap[Long, String]] - assert(linkedHashMapSerializer.dataType == + val linkedHashMapSerializer = serializerFor[LHMap[Long, String]](BoundReference( + 0, ObjectType(classOf[LHMap[Long, String]]), nullable = false)) + assert(linkedHashMapSerializer.dataType.head.dataType == MapType(LongType, StringType, valueContainsNull = true)) val linkedHashMapDeserializer = deserializerFor[LHMap[Long, String]] assert(linkedHashMapDeserializer.dataType == ObjectType(classOf[LHMap[_, _]])) } test("SPARK-22442: Generate correct field names for special characters") { - val serializer = serializerFor[SpecialCharAsFieldData] - .collect { - case If(_, _, s: CreateNamedStruct) => s - }.head + val serializer = serializerFor[SpecialCharAsFieldData](BoundReference( + 0, ObjectType(classOf[SpecialCharAsFieldData]), nullable = false)) val deserializer = deserializerFor[SpecialCharAsFieldData] assert(serializer.dataType(0).name == "field.1") assert(serializer.dataType(1).name == "field 2") - val newInstance = deserializer.collect { case n: NewInstance => n }.head - - val argumentsFields = newInstance.arguments.flatMap { _.collect { - case UpCast(u: UnresolvedExtractValue, _, _) => u.extraction.toString + val argumentsFields = deserializer.asInstanceOf[NewInstance].arguments.flatMap { _.collect { + case UpCast(u: UnresolvedAttribute, _, _) => u.nameParts }} - assert(argumentsFields(0) == "field.1") - assert(argumentsFields(1) == "field 2") + assert(argumentsFields(0) == Seq("field.1")) + assert(argumentsFields(1) == Seq("field 2")) } test("SPARK-22472: add null check for top-level primitive values") { @@ -355,8 +351,8 @@ class ScalaReflectionSuite extends SparkFunSuite { test("SPARK-23835: add null check to non-nullable types in Tuples") { def numberOfCheckedArguments(deserializer: Expression): Int = { - val newInstance = deserializer.collect { case n: NewInstance => n}.head - newInstance.arguments.count(_.isInstanceOf[AssertNotNull]) + assert(deserializer.isInstanceOf[NewInstance]) + deserializer.asInstanceOf[NewInstance].arguments.count(_.isInstanceOf[AssertNotNull]) } assert(numberOfCheckedArguments(deserializerFor[(Double, Double)]) == 2) assert(numberOfCheckedArguments(deserializerFor[(java.lang.Double, Int)]) == 1) 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 da3ae72c3682a..f9facbb71a4e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import java.util.{Locale, TimeZone} +import java.util.TimeZone import scala.reflect.ClassTag @@ -25,7 +25,6 @@ import org.scalatest.Matchers import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -33,7 +32,6 @@ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} -import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -316,24 +314,24 @@ class AnalysisSuite extends AnalysisTest with Matchers { } // non-primitive parameters do not need special null handling - val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, true :: Nil) + val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil) val expected1 = udf1 checkUDF(udf1, expected1) // only primitive parameter needs special null handling val udf2 = ScalaUDF((s: String, d: Double) => "x", StringType, string :: double :: Nil, - true :: false :: Nil) + nullableTypes = true :: false :: Nil) val expected2 = - If(IsNull(double), nullResult, udf2.copy(inputsNullSafe = true :: true :: Nil)) + If(IsNull(double), nullResult, udf2.copy(children = string :: KnownNotNull(double) :: Nil)) checkUDF(udf2, expected2) // special null handling should apply to all primitive parameters val udf3 = ScalaUDF((s: Short, d: Double) => "x", StringType, short :: double :: Nil, - false :: false :: Nil) + nullableTypes = false :: false :: Nil) val expected3 = If( IsNull(short) || IsNull(double), nullResult, - udf3.copy(inputsNullSafe = true :: true :: Nil)) + udf3.copy(children = KnownNotNull(short) :: KnownNotNull(double) :: Nil)) checkUDF(udf3, expected3) // we can skip special null handling for primitive parameters that are not nullable @@ -342,19 +340,19 @@ class AnalysisSuite extends AnalysisTest with Matchers { (s: Short, d: Double) => "x", StringType, short :: double.withNullability(false) :: Nil, - false :: false :: Nil) + nullableTypes = false :: false :: Nil) val expected4 = If( IsNull(short), nullResult, - udf4.copy(inputsNullSafe = true :: true :: Nil)) + udf4.copy(children = KnownNotNull(short) :: double.withNullability(false) :: Nil)) // checkUDF(udf4, expected4) } test("SPARK-24891 Fix HandleNullInputsForUDF rule") { val a = testRelation.output(0) val func = (x: Int, y: Int) => x + y - val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, false :: false :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, false :: false :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } @@ -606,25 +604,4 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis(input, expected) } } - - test("SPARK-25691: AliasViewChild with different nullabilities") { - object ViewAnalyzer extends RuleExecutor[LogicalPlan] { - val batches = Batch("View", Once, AliasViewChild(conf), EliminateView) :: Nil - } - val relation = LocalRelation('a.int.notNull, 'b.string) - val view = View(CatalogTable( - identifier = TableIdentifier("v1"), - tableType = CatalogTableType.VIEW, - storage = CatalogStorageFormat.empty, - schema = StructType(Seq(StructField("a", IntegerType), StructField("b", StringType)))), - output = Seq('a.int, 'b.string), - child = relation) - val tz = Option(conf.sessionLocalTimeZone) - val expected = Project(Seq( - Alias(Cast('a.int.notNull, IntegerType, tz), "a")(), - Alias(Cast('b.string, StringType, tz), "b")()), - relation) - val res = ViewAnalyzer.execute(view) - comparePlans(res, expected) - } } 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 3eb3fe66cebc5..8eec14842c7e7 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 @@ -144,9 +144,6 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Sum('stringField)) assertSuccess(Average('stringField)) assertSuccess(Min('arrayField)) - assertSuccess(new EveryAgg('booleanField)) - assertSuccess(new AnyAgg('booleanField)) - assertSuccess(new SomeAgg('booleanField)) assertError(Min('mapField), "min does not support ordering on type") assertError(Max('mapField), "max does not support ordering on type") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 0b168d060ef6e..0eba1c537d67d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -742,7 +742,7 @@ class TypeCoercionSuite extends AnalysisTest { val nullLit = Literal.create(null, NullType) val floatNullLit = Literal.create(null, FloatType) val floatLit = Literal.create(1.0f, FloatType) - val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) + val timestampLit = Literal.create("2017-04-12", TimestampType) val decimalLit = Literal(new java.math.BigDecimal("1000000000000000000000")) val tsArrayLit = Literal(Array(new Timestamp(System.currentTimeMillis()))) val strArrayLit = Literal(Array("c")) @@ -793,11 +793,11 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) - :: Literal.create(1.0f, FloatType) + :: Literal.create(1.0, FloatType) :: Nil), CreateArray(Literal(1.0) :: Cast(Literal(1), DoubleType) - :: Cast(Literal.create(1.0f, FloatType), DoubleType) + :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, @@ -834,23 +834,23 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") - :: Literal.create(2.0f, FloatType) + :: Literal.create(2.0, FloatType) :: Literal("b") :: Nil), CreateMap(Cast(Literal(1), FloatType) :: Literal("a") - :: Literal.create(2.0f, FloatType) + :: Literal.create(2.0, FloatType) :: Literal("b") :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal.create(null, DecimalType(5, 3)) :: Literal("a") - :: Literal.create(2.0f, FloatType) + :: Literal.create(2.0, FloatType) :: Literal("b") :: Nil), CreateMap(Literal.create(null, DecimalType(5, 3)).cast(DoubleType) :: Literal("a") - :: Literal.create(2.0f, FloatType).cast(DoubleType) + :: Literal.create(2.0, FloatType).cast(DoubleType) :: Literal("b") :: Nil)) // type coercion for map values @@ -895,11 +895,11 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1.0) :: Literal(1) - :: Literal.create(1.0f, FloatType) + :: Literal.create(1.0, FloatType) :: Nil), operator(Literal(1.0) :: Cast(Literal(1), DoubleType) - :: Cast(Literal.create(1.0f, FloatType), DoubleType) + :: Cast(Literal.create(1.0, FloatType), DoubleType) :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1L) @@ -966,7 +966,7 @@ class TypeCoercionSuite extends AnalysisTest { val falseLit = Literal.create(false, BooleanType) val stringLit = Literal.create("c", StringType) val floatLit = Literal.create(1.0f, FloatType) - val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) + val timestampLit = Literal.create("2017-04-12", TimestampType) val decimalLit = Literal(new java.math.BigDecimal("1000000000000000000000")) ruleTest(rule, @@ -1016,16 +1016,14 @@ class TypeCoercionSuite extends AnalysisTest { CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) ruleTest(TypeCoercion.CaseWhenCoercion, + CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Literal(1.2))), - Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2))), - CaseWhen(Seq((Literal(true), Literal(1.2))), - Cast(Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2)), DoubleType)) + Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) ) ruleTest(TypeCoercion.CaseWhenCoercion, - CaseWhen(Seq((Literal(true), Literal(100L))), - Literal.create(BigDecimal.valueOf(1), DecimalType(7, 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(BigDecimal.valueOf(1), DecimalType(7, 2)), DecimalType(22, 2))) + Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) ) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index e9b100b3b30db..f0d61de97ffcd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.{Encoder, Encoders} import org.apache.spark.sql.catalyst.{OptionalData, PrimitiveData} import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.util.ArrayData import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -348,7 +348,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes test("nullable of encoder serializer") { def checkNullable[T: Encoder](nullable: Boolean): Unit = { - assert(encoderFor[T].objSerializer.nullable === nullable) + assert(encoderFor[T].serializer.forall(_.nullable === nullable)) } // test for flat encoders diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index ab819bec72e85..8d89f9c6c41d4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -239,7 +239,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { val encoder = RowEncoder(schema) val e = intercept[RuntimeException](encoder.toRow(null)) assert(e.getMessage.contains("Null value appeared in non-nullable field")) - assert(e.getMessage.contains("top level Product or row object")) + assert(e.getMessage.contains("top level row object")) } test("RowEncoder should validate external type") { @@ -273,14 +273,6 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { assert(e4.getMessage.contains("java.lang.String is not a valid external type")) } - test("SPARK-25791: Datatype of serializers should be accessible") { - val udtSQLType = new StructType().add("a", IntegerType) - val pythonUDT = new PythonUserDefinedType(udtSQLType, "pyUDT", "serializedPyClass") - val schema = new StructType().add("pythonUDT", pythonUDT, true) - val encoder = RowEncoder(schema) - assert(encoder.serializer(0).dataType == pythonUDT.sqlType) - } - for { elementType <- Seq(IntegerType, StringType) containsNull <- Seq(true, false) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 94dee7ea048c3..90c0bf7d8b3d7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -112,7 +112,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast string to timestamp") { - ALL_TIMEZONES.par.foreach { tz => + for (tz <- Random.shuffle(ALL_TIMEZONES).take(50)) { def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala index 7843003a4aac3..5e8113ac8658e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala @@ -113,7 +113,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = UTF8String.fromString("abc") - if (!checkResult(actual.head, expected, expressions.head)) { + if (!checkResult(actual.head, expected, expressions.head.dataType)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -126,7 +126,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = UnsafeArrayData.fromPrimitiveArray(Array.fill(length)(true)) - if (!checkResult(actual.head, expected, expressions.head)) { + if (!checkResult(actual.head, expected, expressions.head.dataType)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -142,7 +142,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = ArrayBasedMapData((0 until length).toArray, Array.fill(length)(true)) - if (!checkResult(actual.head, expected, expressions.head)) { + if (!checkResult(actual.head, expected, expressions.head.dataType)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -154,7 +154,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) val expected = Seq(InternalRow(Seq.fill(length)(true): _*)) - if (!checkResult(actual, expected, expressions.head)) { + if (!checkResult(actual, expected, expressions.head.dataType)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -170,7 +170,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actual.length == 1) val expected = InternalRow(Seq.fill(length)(true): _*) - if (!checkResult(actual.head, expected, expressions.head)) { + if (!checkResult(actual.head, expected, expressions.head.dataType)) { fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") } } @@ -375,7 +375,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actualOr.length == 1) val expectedOr = false - if (!checkResult(actualOr.head, expectedOr, exprOr)) { + if (!checkResult(actualOr.head, expectedOr, exprOr.dataType)) { fail(s"Incorrect Evaluation: expressions: $exprOr, actual: $actualOr, expected: $expectedOr") } @@ -389,7 +389,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(actualAnd.length == 1) val expectedAnd = false - if (!checkResult(actualAnd.head, expectedAnd, exprAnd)) { + if (!checkResult(actualAnd.head, expectedAnd, exprAnd.dataType)) { fail( s"Incorrect Evaluation: expressions: $exprAnd, actual: $actualAnd, expected: $expectedAnd") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala deleted file mode 100644 index 386e0d133dff6..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions - -import java.util.Calendar - -import org.scalatest.exceptions.TestFailedException - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.plans.PlanTestBase -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -class CsvExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with PlanTestBase { - val badCsv = "\u0000\u0000\u0000A\u0001AAA" - - val gmtId = Option(DateTimeUtils.TimeZoneGMT.getID) - - test("from_csv") { - val csvData = "1" - val schema = StructType(StructField("a", IntegerType) :: Nil) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), - InternalRow(1) - ) - } - - test("from_csv - invalid data") { - val csvData = "---" - val schema = StructType(StructField("a", DoubleType) :: Nil) - checkEvaluation( - CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), gmtId), - InternalRow(null)) - - // Default mode is Permissive - checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), InternalRow(null)) - } - - test("from_csv null input column") { - val schema = StructType(StructField("a", IntegerType) :: Nil) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create(null, StringType), gmtId), - null - ) - } - - test("from_csv bad UTF-8") { - val schema = StructType(StructField("a", IntegerType) :: Nil) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(badCsv), gmtId), - InternalRow(null)) - } - - test("from_csv with timestamp") { - val schema = StructType(StructField("t", TimestampType) :: Nil) - - val csvData1 = "2016-01-01T00:00:00.123Z" - var c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) - c.set(2016, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 123) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData1), gmtId), - InternalRow(c.getTimeInMillis * 1000L) - ) - // The result doesn't change because the CSV string includes timezone string ("Z" here), - // which means the string represents the timestamp string in the timezone regardless of - // the timeZoneId parameter. - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal(csvData1), Option("PST")), - InternalRow(c.getTimeInMillis * 1000L) - ) - - val csvData2 = "2016-01-01T00:00:00" - for (tz <- DateTimeTestUtils.outstandingTimezones) { - c = Calendar.getInstance(tz) - c.set(2016, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkEvaluation( - CsvToStructs( - schema, - Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), - Literal(csvData2), - Option(tz.getID)), - InternalRow(c.getTimeInMillis * 1000L) - ) - checkEvaluation( - CsvToStructs( - schema, - Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", - DateTimeUtils.TIMEZONE_OPTION -> tz.getID), - Literal(csvData2), - gmtId), - InternalRow(c.getTimeInMillis * 1000L) - ) - } - } - - test("from_csv empty input column") { - val schema = StructType(StructField("a", IntegerType) :: Nil) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId), - InternalRow(null) - ) - } - - test("forcing schema nullability") { - val input = """1,,"foo"""" - val csvSchema = new StructType() - .add("a", LongType, nullable = false) - .add("b", StringType, nullable = false) - .add("c", StringType, nullable = false) - val output = InternalRow(1L, null, UTF8String.fromString("foo")) - val expr = CsvToStructs(csvSchema, Map.empty, Literal.create(input, StringType), gmtId) - checkEvaluation(expr, output) - val schema = expr.dataType - val schemaToCompare = csvSchema.asNullable - assert(schemaToCompare == schema) - } - - - test("from_csv missing columns") { - val schema = new StructType() - .add("a", IntegerType) - .add("b", IntegerType) - checkEvaluation( - CsvToStructs(schema, Map.empty, Literal.create("1"), gmtId), - InternalRow(1, null) - ) - } - - test("unsupported mode") { - val csvData = "---" - val schema = StructType(StructField("a", DoubleType) :: Nil) - val exception = intercept[TestFailedException] { - checkEvaluation( - CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), gmtId), - InternalRow(null)) - }.getCause - assert(exception.getMessage.contains("from_csv() doesn't support the DROPMALFORMED mode")) - } - - test("infer schema of CSV strings") { - checkEvaluation(new SchemaOfCsv(Literal.create("1,abc")), "struct<_c0:int,_c1:string>") - } - - test("infer schema of CSV strings by using options") { - checkEvaluation( - new SchemaOfCsv(Literal.create("1|abc"), Map("delimiter" -> "|")), - "struct<_c0:int,_c1:string>") - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index da18475276a13..b5986aac65552 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -69,22 +69,11 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa /** * Check the equality between result of expression and expected value, it will handle - * Array[Byte], Spread[Double], MapData and Row. Also check whether nullable in expression is - * true if result is null + * Array[Byte], Spread[Double], MapData and Row. */ - protected def checkResult(result: Any, expected: Any, expression: Expression): Boolean = { - checkResult(result, expected, expression.dataType, expression.nullable) - } - - protected def checkResult( - result: Any, - expected: Any, - exprDataType: DataType, - exprNullable: Boolean): Boolean = { + protected def checkResult(result: Any, expected: Any, exprDataType: DataType): Boolean = { val dataType = UserDefinedType.sqlType(exprDataType) - // The result is null for a non-nullable expression - assert(result != null || exprNullable, "exprNullable should be true if result is null") (result, expected) match { case (result: Array[Byte], expected: Array[Byte]) => java.util.Arrays.equals(result, expected) @@ -94,24 +83,24 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val st = dataType.asInstanceOf[StructType] assert(result.numFields == st.length && expected.numFields == st.length) st.zipWithIndex.forall { case (f, i) => - checkResult( - result.get(i, f.dataType), expected.get(i, f.dataType), f.dataType, f.nullable) + checkResult(result.get(i, f.dataType), expected.get(i, f.dataType), f.dataType) } case (result: ArrayData, expected: ArrayData) => result.numElements == expected.numElements && { - val ArrayType(et, cn) = dataType.asInstanceOf[ArrayType] + val et = dataType.asInstanceOf[ArrayType].elementType var isSame = true var i = 0 while (isSame && i < result.numElements) { - isSame = checkResult(result.get(i, et), expected.get(i, et), et, cn) + isSame = checkResult(result.get(i, et), expected.get(i, et), et) i += 1 } isSame } case (result: MapData, expected: MapData) => - val MapType(kt, vt, vcn) = dataType.asInstanceOf[MapType] - checkResult(result.keyArray, expected.keyArray, ArrayType(kt, false), false) && - checkResult(result.valueArray, expected.valueArray, ArrayType(vt, vcn), false) + val kt = dataType.asInstanceOf[MapType].keyType + val vt = dataType.asInstanceOf[MapType].valueType + checkResult(result.keyArray, expected.keyArray, ArrayType(kt)) && + checkResult(result.valueArray, expected.valueArray, ArrayType(vt)) case (result: Double, expected: Double) => if (expected.isNaN) result.isNaN else expected == result case (result: Float, expected: Float) => @@ -186,7 +175,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val actual = try evaluateWithoutCodegen(expression, inputRow) catch { case e: Exception => fail(s"Exception evaluating $expression", e) } - if (!checkResult(actual, expected, expression)) { + if (!checkResult(actual, expected, expression.dataType)) { val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" fail(s"Incorrect evaluation (codegen off): $expression, " + s"actual: $actual, " + @@ -202,7 +191,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa for (fallbackMode <- modes) { withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { val actual = evaluateWithMutableProjection(expression, inputRow) - if (!checkResult(actual, expected, expression)) { + if (!checkResult(actual, expected, expression.dataType)) { val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" fail(s"Incorrect evaluation (fallback mode = $fallbackMode): $expression, " + s"actual: $actual, expected: $expected$input") @@ -232,12 +221,6 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa val unsafeRow = evaluateWithUnsafeProjection(expression, inputRow) val input = if (inputRow == EmptyRow) "" else s", input: $inputRow" - val dataType = expression.dataType - if (!checkResult(unsafeRow.get(0, dataType), expected, dataType, expression.nullable)) { - fail("Incorrect evaluation in unsafe mode (fallback mode = $fallbackMode): " + - s"$expression, actual: $unsafeRow, expected: $expected, " + - s"dataType: $dataType, nullable: ${expression.nullable}") - } if (expected == null) { if (!unsafeRow.isNullAt(0)) { val expectedRow = InternalRow(expected, expected) @@ -246,7 +229,8 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa } } else { val lit = InternalRow(expected, expected) - val expectedRow = UnsafeProjection.create(Array(dataType, dataType)).apply(lit) + val expectedRow = + UnsafeProjection.create(Array(expression.dataType, expression.dataType)).apply(lit) if (unsafeRow != expectedRow) { fail(s"Incorrect evaluation in unsafe mode (fallback mode = $fallbackMode): " + s"$expression, actual: $unsafeRow, expected: $expectedRow$input") @@ -296,7 +280,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa expression) plan.initialize(0) var actual = plan(inputRow).get(0, expression.dataType) - assert(checkResult(actual, expected, expression)) + assert(checkResult(actual, expected, expression.dataType)) plan = generateProject( GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), @@ -304,7 +288,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa plan.initialize(0) actual = FromUnsafeProjection(expression.dataType :: Nil)( plan(inputRow)).get(0, expression.dataType) - assert(checkResult(actual, expected, expression)) + assert(checkResult(actual, expected, expression.dataType)) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala index 54ef9641bee0d..7c7c4cccee253 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelperSuite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodegenFallback, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData} -import org.apache.spark.sql.types.{DataType, IntegerType, MapType} +import org.apache.spark.sql.types.{DataType, IntegerType} /** * A test suite for testing [[ExpressionEvalHelper]]. @@ -36,13 +35,6 @@ class ExpressionEvalHelperSuite extends SparkFunSuite with ExpressionEvalHelper val e = intercept[RuntimeException] { checkEvaluation(BadCodegenExpression(), 10) } assert(e.getMessage.contains("some_variable")) } - - test("SPARK-25388: checkEvaluation should fail if nullable in DataType is incorrect") { - val e = intercept[RuntimeException] { - checkEvaluation(MapIncorrectDataTypeExpression(), Map(3 -> 7, 6 -> null)) - } - assert(e.getMessage.contains("and exprNullable was")) - } } /** @@ -61,18 +53,3 @@ case class BadCodegenExpression() extends LeafExpression { } override def dataType: DataType = IntegerType } - -/** - * An expression that returns a MapData with incorrect DataType whose valueContainsNull is false - * while its value includes null - */ -case class MapIncorrectDataTypeExpression() extends LeafExpression with CodegenFallback { - override def nullable: Boolean = false - override def eval(input: InternalRow): Any = { - val keys = new GenericArrayData(Array(3, 6)) - val values = new GenericArrayData(Array(7, null)) - new ArrayBasedMapData(keys, values) - } - // since values includes null, valueContainsNull must be true - override def dataType: DataType = MapType(IntegerType, IntegerType, valueContainsNull = false) -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 304642161146b..81ab7d690396a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Calendar -import org.scalatest.exceptions.TestFailedException - -import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.PlanTestBase -import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeTestUtils, DateTimeUtils, GenericArrayData, PermissiveMode} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -411,18 +409,14 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId), - InternalRow(null) + null ) - val exception = intercept[TestFailedException] { - checkEvaluation( - JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), gmtId), - InternalRow(null) - ) - }.getCause - assert(exception.isInstanceOf[SparkException]) - assert(exception.getMessage.contains( - "Malformed records are detected in record parsing. Parse Mode: FAILFAST")) + // Other modes should still return `null`. + checkEvaluation( + JsonToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(jsonData), gmtId), + null + ) } test("from_json - input=array, schema=array, output=array") { @@ -456,23 +450,21 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("from_json - input=array of single object, schema=struct, output=single row") { val input = """[{"a": 1}]""" val schema = StructType(StructField("a", IntegerType) :: Nil) - val output = InternalRow(null) + val output = InternalRow(1) checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) } - test("from_json - input=array, schema=struct, output=single row") { + test("from_json - input=array, schema=struct, output=null") { val input = """[{"a": 1}, {"a": 2}]""" - val corrupted = "corrupted" - val schema = new StructType().add("a", IntegerType).add(corrupted, StringType) - val output = InternalRow(null, UTF8String.fromString(input)) - val options = Map("columnNameOfCorruptRecord" -> corrupted) - checkEvaluation(JsonToStructs(schema, options, Literal(input), gmtId), output) + val schema = StructType(StructField("a", IntegerType) :: Nil) + val output = null + checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) } - test("from_json - input=empty array, schema=struct, output=single row with null") { + test("from_json - input=empty array, schema=struct, output=null") { val input = """[]""" val schema = StructType(StructField("a", IntegerType) :: Nil) - val output = InternalRow(null) + val output = null checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) } @@ -495,7 +487,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( JsonToStructs(schema, Map.empty, Literal(badJson), gmtId), - InternalRow(null)) + null) } test("from_json with timestamp") { @@ -518,7 +510,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with ) val jsonData2 = """{"t": "2016-01-01T00:00:00"}""" - for (tz <- DateTimeTestUtils.outstandingTimezones) { + for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { c = Calendar.getInstance(tz) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) @@ -631,8 +623,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[string, struct] to json") { val schema = MapType(StringType, StructType(StructField("a", IntegerType) :: Nil)) - val input = Literal( - ArrayBasedMapData(Map(UTF8String.fromString("test") -> InternalRow(1))), schema) + val input = Literal.create(ArrayBasedMapData(Map("test" -> InternalRow(1))), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"test":{"a":1}}""" @@ -642,7 +633,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[struct, struct] to json") { val schema = MapType(StructType(StructField("a", IntegerType) :: Nil), StructType(StructField("b", IntegerType) :: Nil)) - val input = Literal(ArrayBasedMapData(Map(InternalRow(1) -> InternalRow(2))), schema) + val input = Literal.create(ArrayBasedMapData(Map(InternalRow(1) -> InternalRow(2))), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"[1]":{"b":2}}""" @@ -651,7 +642,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[string, integer] to json") { val schema = MapType(StringType, IntegerType) - val input = Literal(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)), schema) + val input = Literal.create(ArrayBasedMapData(Map("a" -> 1)), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"a":1}""" @@ -660,18 +651,17 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("to_json - array with maps") { val inputSchema = ArrayType(MapType(StringType, IntegerType)) - val input = new GenericArrayData( - ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: - ArrayBasedMapData(Map(UTF8String.fromString("b") -> 2)) :: Nil) + val input = new GenericArrayData(ArrayBasedMapData( + Map("a" -> 1)) :: ArrayBasedMapData(Map("b" -> 2)) :: Nil) val output = """[{"a":1},{"b":2}]""" checkEvaluation( - StructsToJson(Map.empty, Literal(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), output) } test("to_json - array with single map") { val inputSchema = ArrayType(MapType(StringType, IntegerType)) - val input = new GenericArrayData(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: Nil) + val input = new GenericArrayData(ArrayBasedMapData(Map("a" -> 1)) :: Nil) val output = """[{"a":1}]""" checkEvaluation( StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala index 8818d0135b297..6e07f7a59b730 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext @@ -109,8 +107,8 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val nullLit = Literal.create(null, NullType) val floatNullLit = Literal.create(null, FloatType) val floatLit = Literal.create(1.01f, FloatType) - val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) - val decimalLit = Literal.create(BigDecimal.valueOf(10.2), DecimalType(20, 2)) + val timestampLit = Literal.create("2017-04-12", TimestampType) + val decimalLit = Literal.create(10.2, DecimalType(20, 2)) assert(analyze(new Nvl(decimalLit, stringLit)).dataType == StringType) assert(analyze(new Nvl(doubleLit, decimalLit)).dataType == DoubleType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 06fb73ad83923..d532dc4f77198 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -225,18 +225,11 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row3 = create_row("aa2bb3cc", null) checkEvaluation( - StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+"), -1), Seq("aa", "bb", "cc"), row1) + StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+")), Seq("aa", "bb", "cc"), row1) checkEvaluation( - StringSplit(Literal("aa2bb3cc"), Literal("[1-9]+"), 2), Seq("aa", "bb3cc"), row1) - // limit = 0 should behave just like limit = -1 - checkEvaluation( - StringSplit(Literal("aacbbcddc"), Literal("c"), 0), Seq("aa", "bb", "dd", ""), row1) - checkEvaluation( - StringSplit(Literal("aacbbcddc"), Literal("c"), -1), Seq("aa", "bb", "dd", ""), row1) - checkEvaluation( - StringSplit(s1, s2, -1), Seq("aa", "bb", "cc"), row1) - checkEvaluation(StringSplit(s1, s2, -1), null, row2) - checkEvaluation(StringSplit(s1, s2, -1), null, row3) + StringSplit(s1, s2), Seq("aa", "bb", "cc"), row1) + checkEvaluation(StringSplit(s1, s2), null, row2) + checkEvaluation(StringSplit(s1, s2), null, row3) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index 467cfd5598ff1..e083ae0089244 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.types.{IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("basic") { - val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, true :: Nil) + val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil) checkEvaluation(intUdf, 2) - val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, true :: Nil) + val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil) checkEvaluation(stringUdf, "ax") } @@ -37,8 +37,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { val udf = ScalaUDF( (s: String) => s.toLowerCase(Locale.ROOT), StringType, - Literal.create(null, StringType) :: Nil, - true :: Nil) + Literal.create(null, StringType) :: Nil) val e1 = intercept[SparkException](udf.eval()) assert(e1.getMessage.contains("Failed to execute user defined function")) @@ -51,7 +50,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22695: ScalaUDF should not use global variables") { val ctx = new CodegenContext - ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, true :: Nil).genCode(ctx) + ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala index f2696849d7753..cc2e2a993d629 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.TimeZone import org.apache.spark.SparkFunSuite @@ -32,9 +32,9 @@ class SortOrderExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val b2 = Literal.create(true, BooleanType) val i1 = Literal.create(20132983, IntegerType) val i2 = Literal.create(-20132983, IntegerType) - val l1 = Literal.create(20132983L, LongType) - val l2 = Literal.create(-20132983L, LongType) - val millis = 1524954911000L + val l1 = Literal.create(20132983, LongType) + val l2 = Literal.create(-20132983, LongType) + val millis = 1524954911000L; // Explicitly choose a time zone, since Date objects can create different values depending on // local time zone of the machine on which the test is running val oldDefaultTZ = TimeZone.getDefault @@ -57,7 +57,7 @@ class SortOrderExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val dec1 = Literal(Decimal(20132983L, 10, 2)) val dec2 = Literal(Decimal(20132983L, 19, 2)) val dec3 = Literal(Decimal(20132983L, 21, 2)) - val list1 = Literal.create(Seq(1, 2), ArrayType(IntegerType)) + val list1 = Literal(List(1, 2), ArrayType(IntegerType)) val nullVal = Literal.create(null, IntegerType) checkEvaluation(SortPrefix(SortOrder(b1, Ascending)), 0L) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index d202c2f271d97..d46135c02bc01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -105,9 +105,9 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva } test("parse sql expression for duration in microseconds - long") { - val dur = TimeWindow.invokePrivate(parseExpression(Literal.create(2L << 52, LongType))) + val dur = TimeWindow.invokePrivate(parseExpression(Literal.create(2 << 52, LongType))) assert(dur.isInstanceOf[Long]) - assert(dur === (2L << 52)) + assert(dur === (2 << 52)) } test("parse sql expression for duration in microseconds - invalid interval") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 294fce8e9a10f..2420ba513f287 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -232,14 +232,11 @@ class PercentileSuite extends SparkFunSuite { BooleanType, StringType, DateType, TimestampType, CalendarIntervalType, NullType) invalidDataTypes.foreach { dataType => - val percentage = Literal.default(dataType) + val percentage = Literal(0.5, dataType) val percentile4 = new Percentile(child, percentage) - val checkResult = percentile4.checkInputDataTypes() - assert(checkResult.isFailure) - Seq("argument 2 requires double type, however, ", - s"is of ${dataType.simpleString} type.").foreach { errMsg => - assert(checkResult.asInstanceOf[TypeCheckFailure].message.contains(errMsg)) - } + assertEqual(percentile4.checkInputDataTypes(), + TypeCheckFailure(s"argument 2 requires double type, however, " + + s"'0.5' is of ${dataType.simpleString} type.")) } } 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 a0de5f6930958..6cd1108eef333 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 @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.BooleanType -class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with PredicateHelper { +class BooleanSimplificationSuite extends PlanTest with PredicateHelper { object Optimize extends RuleExecutor[LogicalPlan] { val batches = @@ -71,14 +71,6 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with comparePlans(actual, correctAnswer) } - private def checkConditionInNotNullableRelation( - input: Expression, expected: Expression): Unit = { - val plan = testNotNullableRelationWithData.where(input).analyze - val actual = Optimize.execute(plan) - val correctAnswer = testNotNullableRelationWithData.where(expected).analyze - comparePlans(actual, correctAnswer) - } - private def checkConditionInNotNullableRelation( input: Expression, expected: LogicalPlan): Unit = { val plan = testNotNullableRelationWithData.where(input).analyze @@ -127,55 +119,42 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with 'a === 'b || 'b > 3 && 'a > 3 && 'a < 5) } - test("e && (!e || f) - not nullable") { - checkConditionInNotNullableRelation('e && (!'e || 'f ), 'e && 'f) + test("e && (!e || f)") { + checkCondition('e && (!'e || 'f ), 'e && 'f) - checkConditionInNotNullableRelation('e && ('f || !'e ), 'e && 'f) + checkCondition('e && ('f || !'e ), 'e && 'f) - checkConditionInNotNullableRelation((!'e || 'f ) && 'e, 'f && 'e) + checkCondition((!'e || 'f ) && 'e, 'f && 'e) - checkConditionInNotNullableRelation(('f || !'e ) && 'e, 'f && 'e) + checkCondition(('f || !'e ) && 'e, 'f && 'e) } - test("e && (!e || f) - nullable") { - Seq ('e && (!'e || 'f ), - 'e && ('f || !'e ), - (!'e || 'f ) && 'e, - ('f || !'e ) && 'e, - 'e || (!'e && 'f), - 'e || ('f && !'e), - ('e && 'f) || !'e, - ('f && 'e) || !'e).foreach { expr => - checkCondition(expr, expr) - } - } + test("a < 1 && (!(a < 1) || f)") { + checkCondition('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) + checkCondition('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) - test("a < 1 && (!(a < 1) || f) - not nullable") { - checkConditionInNotNullableRelation('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) - checkConditionInNotNullableRelation('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) + checkCondition('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) + checkCondition('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) + checkCondition('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) + checkCondition('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) - - checkConditionInNotNullableRelation('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) - checkConditionInNotNullableRelation('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) + checkCondition('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) + checkCondition('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) } - test("a < 1 && ((a >= 1) || f) - not nullable") { - checkConditionInNotNullableRelation('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) - checkConditionInNotNullableRelation('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) + test("a < 1 && ((a >= 1) || f)") { + checkCondition('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) + checkCondition('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) + checkCondition('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) + checkCondition('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) + checkCondition('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) + checkCondition('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) - checkConditionInNotNullableRelation('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) - checkConditionInNotNullableRelation('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) + checkCondition('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) + checkCondition('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) } test("DeMorgan's law") { @@ -238,46 +217,4 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with checkCondition('e || !'f, testRelationWithData.where('e || !'f).analyze) checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } - - 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("filter reduction - positive cases") { - val fields = Seq( - 'col1NotNULL.boolean.notNull, - 'col2NotNULL.boolean.notNull - ) - val Seq(col1NotNULL, col2NotNULL) = fields.zipWithIndex.map { case (f, i) => f.at(i) } - - val exprs = Seq( - // actual expressions of the transformations: original -> transformed - (col1NotNULL && (!col1NotNULL || col2NotNULL)) -> (col1NotNULL && col2NotNULL), - (col1NotNULL && (col2NotNULL || !col1NotNULL)) -> (col1NotNULL && col2NotNULL), - ((!col1NotNULL || col2NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), - ((col2NotNULL || !col1NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), - - (col1NotNULL || (!col1NotNULL && col2NotNULL)) -> (col1NotNULL || col2NotNULL), - (col1NotNULL || (col2NotNULL && !col1NotNULL)) -> (col1NotNULL || col2NotNULL), - ((!col1NotNULL && col2NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL), - ((col2NotNULL && !col1NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL) - ) - - // check plans - for ((originalExpr, expectedExpr) <- exprs) { - assertEquivalent(originalExpr, expectedExpr) - } - - // check evaluation - val binaryBooleanValues = Seq(true, false) - for (col1NotNULLVal <- binaryBooleanValues; - col2NotNULLVal <- binaryBooleanValues; - (originalExpr, expectedExpr) <- exprs) { - val inputRow = create_row(col1NotNULLVal, col2NotNULLVal) - val optimizedVal = evaluateWithoutCodegen(expectedExpr, inputRow) - checkEvaluation(originalExpr, optimizedVal, inputRow) - } - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseSuite.scala deleted file mode 100644 index c6b5d0ec96776..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseSuite.scala +++ /dev/null @@ -1,323 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{And, CaseWhen, Expression, GreaterThan, If, Literal, Or} -import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} -import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.types.{BooleanType, IntegerType} - -class ReplaceNullWithFalseSuite extends PlanTest { - - object Optimize extends RuleExecutor[LogicalPlan] { - val batches = - Batch("Replace null literals", FixedPoint(10), - NullPropagation, - ConstantFolding, - BooleanSimplification, - SimplifyConditionals, - ReplaceNullWithFalse) :: Nil - } - - private val testRelation = LocalRelation('i.int, 'b.boolean) - private val anotherTestRelation = LocalRelation('d.int) - - test("replace null inside filter and join conditions") { - testFilter(originalCond = Literal(null), expectedCond = FalseLiteral) - testJoin(originalCond = Literal(null), expectedCond = FalseLiteral) - } - - test("replace null in branches of If") { - val originalCond = If( - UnresolvedAttribute("i") > Literal(10), - FalseLiteral, - Literal(null, BooleanType)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace nulls in nested expressions in branches of If") { - val originalCond = If( - UnresolvedAttribute("i") > Literal(10), - TrueLiteral && Literal(null, BooleanType), - UnresolvedAttribute("b") && Literal(null, BooleanType)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in elseValue of CaseWhen") { - val branches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, - (UnresolvedAttribute("i") > Literal(40)) -> FalseLiteral) - val originalCond = CaseWhen(branches, Literal(null, BooleanType)) - val expectedCond = CaseWhen(branches, FalseLiteral) - testFilter(originalCond, expectedCond) - testJoin(originalCond, expectedCond) - } - - test("replace null in branch values of CaseWhen") { - val branches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> Literal(null, BooleanType), - (UnresolvedAttribute("i") > Literal(40)) -> FalseLiteral) - val originalCond = CaseWhen(branches, Literal(null)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in branches of If inside CaseWhen") { - val originalBranches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> - If(UnresolvedAttribute("i") < Literal(20), Literal(null, BooleanType), FalseLiteral), - (UnresolvedAttribute("i") > Literal(40)) -> TrueLiteral) - val originalCond = CaseWhen(originalBranches) - - val expectedBranches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> FalseLiteral, - (UnresolvedAttribute("i") > Literal(40)) -> TrueLiteral) - val expectedCond = CaseWhen(expectedBranches) - - testFilter(originalCond, expectedCond) - testJoin(originalCond, expectedCond) - } - - test("replace null in complex CaseWhen expressions") { - val originalBranches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, - (Literal(6) <= Literal(1)) -> FalseLiteral, - (Literal(4) === Literal(5)) -> FalseLiteral, - (UnresolvedAttribute("i") > Literal(10)) -> Literal(null, BooleanType), - (Literal(4) === Literal(4)) -> TrueLiteral) - val originalCond = CaseWhen(originalBranches) - - val expectedBranches = Seq( - (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, - (UnresolvedAttribute("i") > Literal(10)) -> FalseLiteral, - TrueLiteral -> TrueLiteral) - val expectedCond = CaseWhen(expectedBranches) - - testFilter(originalCond, expectedCond) - testJoin(originalCond, expectedCond) - } - - test("replace null in Or") { - val originalCond = Or(UnresolvedAttribute("b"), Literal(null)) - val expectedCond = UnresolvedAttribute("b") - testFilter(originalCond, expectedCond) - testJoin(originalCond, expectedCond) - } - - test("replace null in And") { - val originalCond = And(UnresolvedAttribute("b"), Literal(null)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace nulls in nested And/Or expressions") { - val originalCond = And( - And(UnresolvedAttribute("b"), Literal(null)), - Or(Literal(null), And(Literal(null), And(UnresolvedAttribute("b"), Literal(null))))) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in And inside branches of If") { - val originalCond = If( - UnresolvedAttribute("i") > Literal(10), - FalseLiteral, - And(UnresolvedAttribute("b"), Literal(null, BooleanType))) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in branches of If inside And") { - val originalCond = And( - UnresolvedAttribute("b"), - If( - UnresolvedAttribute("i") > Literal(10), - Literal(null), - And(FalseLiteral, UnresolvedAttribute("b")))) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in branches of If inside another If") { - val originalCond = If( - If(UnresolvedAttribute("b"), Literal(null), FalseLiteral), - TrueLiteral, - Literal(null)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in CaseWhen inside another CaseWhen") { - val nestedCaseWhen = CaseWhen(Seq(UnresolvedAttribute("b") -> FalseLiteral), Literal(null)) - val originalCond = CaseWhen(Seq(nestedCaseWhen -> TrueLiteral), Literal(null)) - testFilter(originalCond, expectedCond = FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("inability to replace null in non-boolean branches of If") { - val condition = If( - UnresolvedAttribute("i") > Literal(10), - Literal(5) > If( - UnresolvedAttribute("i") === Literal(15), - Literal(null, IntegerType), - Literal(3)), - FalseLiteral) - testFilter(originalCond = condition, expectedCond = condition) - testJoin(originalCond = condition, expectedCond = condition) - } - - test("inability to replace null in non-boolean values of CaseWhen") { - val nestedCaseWhen = CaseWhen( - Seq((UnresolvedAttribute("i") > Literal(20)) -> Literal(2)), - Literal(null, IntegerType)) - val branchValue = If( - Literal(2) === nestedCaseWhen, - TrueLiteral, - FalseLiteral) - val branches = Seq((UnresolvedAttribute("i") > Literal(10)) -> branchValue) - val condition = CaseWhen(branches) - testFilter(originalCond = condition, expectedCond = condition) - testJoin(originalCond = condition, expectedCond = condition) - } - - test("inability to replace null in non-boolean branches of If inside another If") { - val condition = If( - Literal(5) > If( - UnresolvedAttribute("i") === Literal(15), - Literal(null, IntegerType), - Literal(3)), - TrueLiteral, - FalseLiteral) - testFilter(originalCond = condition, expectedCond = condition) - testJoin(originalCond = condition, expectedCond = condition) - } - - test("replace null in If used as a join condition") { - // this test is only for joins as the condition involves columns from different relations - val originalCond = If( - UnresolvedAttribute("d") > UnresolvedAttribute("i"), - Literal(null), - FalseLiteral) - testJoin(originalCond, expectedCond = FalseLiteral) - } - - test("replace null in CaseWhen used as a join condition") { - // this test is only for joins as the condition involves columns from different relations - val originalBranches = Seq( - (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> Literal(null), - (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) - - val expectedBranches = Seq( - (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> FalseLiteral, - (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) - - testJoin( - originalCond = CaseWhen(originalBranches, FalseLiteral), - expectedCond = CaseWhen(expectedBranches, FalseLiteral)) - } - - test("inability to replace null in CaseWhen inside EqualTo used as a join condition") { - // this test is only for joins as the condition involves columns from different relations - val branches = Seq( - (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> Literal(null, BooleanType), - (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) - val condition = UnresolvedAttribute("b") === CaseWhen(branches, FalseLiteral) - testJoin(originalCond = condition, expectedCond = condition) - } - - test("replace null in predicates of If") { - val predicate = And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)) - testProjection( - originalExpr = If(predicate, Literal(5), Literal(1)).as("out"), - expectedExpr = Literal(1).as("out")) - } - - test("replace null in predicates of If inside another If") { - val predicate = If( - And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)), - TrueLiteral, - FalseLiteral) - testProjection( - originalExpr = If(predicate, Literal(5), Literal(1)).as("out"), - expectedExpr = Literal(1).as("out")) - } - - test("inability to replace null in non-boolean expressions inside If predicates") { - val predicate = GreaterThan( - UnresolvedAttribute("i"), - If(UnresolvedAttribute("b"), Literal(null, IntegerType), Literal(4))) - val column = If(predicate, Literal(5), Literal(1)).as("out") - testProjection(originalExpr = column, expectedExpr = column) - } - - test("replace null in conditions of CaseWhen") { - val branches = Seq( - And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)) -> Literal(5)) - testProjection( - originalExpr = CaseWhen(branches, Literal(2)).as("out"), - expectedExpr = Literal(2).as("out")) - } - - test("replace null in conditions of CaseWhen inside another CaseWhen") { - val nestedCaseWhen = CaseWhen( - Seq(And(UnresolvedAttribute("b"), Literal(null)) -> Literal(5)), - Literal(2)) - val branches = Seq(GreaterThan(Literal(3), nestedCaseWhen) -> Literal(1)) - testProjection( - originalExpr = CaseWhen(branches).as("out"), - expectedExpr = Literal(1).as("out")) - } - - test("inability to replace null in non-boolean exprs inside CaseWhen conditions") { - val condition = GreaterThan( - UnresolvedAttribute("i"), - If(UnresolvedAttribute("b"), Literal(null, IntegerType), Literal(4))) - val column = CaseWhen(Seq(condition -> Literal(5)), Literal(2)).as("out") - testProjection(originalExpr = column, expectedExpr = column) - } - - private def testFilter(originalCond: Expression, expectedCond: Expression): Unit = { - test((rel, exp) => rel.where(exp), originalCond, expectedCond) - } - - private def testJoin(originalCond: Expression, expectedCond: Expression): Unit = { - test((rel, exp) => rel.join(anotherTestRelation, Inner, Some(exp)), originalCond, expectedCond) - } - - private def testProjection(originalExpr: Expression, expectedExpr: Expression): Unit = { - test((rel, exp) => rel.select(exp), originalExpr, expectedExpr) - } - - private def test( - func: (LogicalPlan, Expression) => LogicalPlan, - originalExpr: Expression, - expectedExpr: Expression): Unit = { - - val originalPlan = func(testRelation, originalExpr).analyze - val optimizedPlan = Optimize.execute(originalPlan) - val expectedPlan = func(testRelation, expectedExpr).analyze - comparePlans(optimizedPlan, expectedPlan) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index f5da90f7cf0c6..422bf97e30e7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -108,7 +108,7 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b)) assertEqual( "select a, b from db.c having x < 1", - table("db", "c").groupBy()('a, 'b).where('x < 1)) + table("db", "c").select('a, 'b).where('x < 1)) assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) assertEqual("select from tbl", OneRowRelation().select('from.as("tbl"))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 64aa1ee39046d..b7092f4c42d4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -564,7 +564,7 @@ class TreeNodeSuite extends SparkFunSuite { } test("toJSON should not throws java.lang.StackOverflowError") { - val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr), true :: Nil) + val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr)) // Should not throw java.lang.StackOverflowError udf.toJSON } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala index dfa0fe93a2f9c..0c1feb3aa0882 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -26,16 +26,6 @@ object DateTimeTestUtils { val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) - val outstandingTimezones: Seq[TimeZone] = Seq( - "UTC", - "PST", - "CET", - "Africa/Dakar", - "America/Los_Angeles", - "Antarctica/Vostok", - "Asia/Hong_Kong", - "Europe/Amsterdam").map(TimeZone.getTimeZone) - def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { val originalDefaultTimeZone = TimeZone.getDefault try { diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt deleted file mode 100644 index 9d656fc10dce4..0000000000000 --- a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt +++ /dev/null @@ -1,60 +0,0 @@ -================================================================================================ -Parquet writer benchmark -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Output Single Int Column 2354 / 2438 6.7 149.7 1.0X -Output Single Double Column 2462 / 2485 6.4 156.5 1.0X -Output Int and String Column 8083 / 8100 1.9 513.9 0.3X -Output Partitions 5015 / 5027 3.1 318.8 0.5X -Output Buckets 6883 / 6887 2.3 437.6 0.3X - - -================================================================================================ -ORC writer benchmark -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -ORC writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Output Single Int Column 1769 / 1789 8.9 112.4 1.0X -Output Single Double Column 1989 / 2009 7.9 126.5 0.9X -Output Int and String Column 7323 / 7400 2.1 465.6 0.2X -Output Partitions 4374 / 4381 3.6 278.1 0.4X -Output Buckets 6086 / 6104 2.6 386.9 0.3X - - -================================================================================================ -JSON writer benchmark -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Output Single Int Column 2954 / 4085 5.3 187.8 1.0X -Output Single Double Column 3832 / 3837 4.1 243.6 0.8X -Output Int and String Column 9591 / 10336 1.6 609.8 0.3X -Output Partitions 4956 / 4994 3.2 315.1 0.6X -Output Buckets 6608 / 6676 2.4 420.1 0.4X - - -================================================================================================ -CSV writer benchmark -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -CSV writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Output Single Int Column 4118 / 4125 3.8 261.8 1.0X -Output Single Double Column 4888 / 4891 3.2 310.8 0.8X -Output Int and String Column 9788 / 9872 1.6 622.3 0.4X -Output Partitions 6578 / 6640 2.4 418.2 0.6X -Output Buckets 9125 / 9171 1.7 580.2 0.5X - - diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt deleted file mode 100644 index 865575bec83d8..0000000000000 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ /dev/null @@ -1,27 +0,0 @@ -================================================================================================ -Benchmark to measure CSV read/write performance -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parsing quoted values: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -One quoted string 64733 / 64839 0.0 1294653.1 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Wide rows with 1000 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Select 1000 columns 185609 / 189735 0.0 185608.6 1.0X -Select 100 columns 50195 / 51808 0.0 50194.8 3.7X -Select one column 39266 / 39293 0.0 39265.6 4.7X -count() 10959 / 11000 0.1 10958.5 16.9X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Count a dataset with 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Select 10 columns + count() 24637 / 24768 0.4 2463.7 1.0X -Select 1 column + count() 20026 / 20076 0.5 2002.6 1.2X -count() 3754 / 3877 2.7 375.4 6.6X - diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt deleted file mode 100644 index 2d3bae442cc50..0000000000000 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ /dev/null @@ -1,269 +0,0 @@ -================================================================================================ -SQL Single Numeric Column Scan -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 21508 / 22112 0.7 1367.5 1.0X -SQL Json 8705 / 8825 1.8 553.4 2.5X -SQL Parquet Vectorized 157 / 186 100.0 10.0 136.7X -SQL Parquet MR 1789 / 1794 8.8 113.8 12.0X -SQL ORC Vectorized 156 / 166 100.9 9.9 138.0X -SQL ORC Vectorized with copy 218 / 225 72.1 13.9 98.6X -SQL ORC MR 1448 / 1492 10.9 92.0 14.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 202 / 211 77.7 12.9 1.0X -ParquetReader Vectorized -> Row 118 / 120 133.5 7.5 1.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 23282 / 23312 0.7 1480.2 1.0X -SQL Json 9187 / 9189 1.7 584.1 2.5X -SQL Parquet Vectorized 204 / 218 77.0 13.0 114.0X -SQL Parquet MR 1941 / 1953 8.1 123.4 12.0X -SQL ORC Vectorized 217 / 225 72.6 13.8 107.5X -SQL ORC Vectorized with copy 279 / 289 56.3 17.8 83.4X -SQL ORC MR 1541 / 1549 10.2 98.0 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 288 / 297 54.6 18.3 1.0X -ParquetReader Vectorized -> Row 255 / 257 61.7 16.2 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 24990 / 25012 0.6 1588.8 1.0X -SQL Json 9837 / 9865 1.6 625.4 2.5X -SQL Parquet Vectorized 170 / 180 92.3 10.8 146.6X -SQL Parquet MR 2319 / 2328 6.8 147.4 10.8X -SQL ORC Vectorized 293 / 301 53.7 18.6 85.3X -SQL ORC Vectorized with copy 297 / 309 52.9 18.9 84.0X -SQL ORC MR 1667 / 1674 9.4 106.0 15.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 257 / 274 61.3 16.3 1.0X -ParquetReader Vectorized -> Row 259 / 264 60.8 16.4 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 32537 / 32554 0.5 2068.7 1.0X -SQL Json 12610 / 12668 1.2 801.7 2.6X -SQL Parquet Vectorized 258 / 276 61.0 16.4 126.2X -SQL Parquet MR 2422 / 2435 6.5 154.0 13.4X -SQL ORC Vectorized 378 / 385 41.6 24.0 86.2X -SQL ORC Vectorized with copy 381 / 389 41.3 24.2 85.4X -SQL ORC MR 1797 / 1819 8.8 114.3 18.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 352 / 368 44.7 22.4 1.0X -ParquetReader Vectorized -> Row 351 / 359 44.8 22.3 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 27179 / 27184 0.6 1728.0 1.0X -SQL Json 12578 / 12585 1.3 799.7 2.2X -SQL Parquet Vectorized 161 / 171 97.5 10.3 168.5X -SQL Parquet MR 2361 / 2395 6.7 150.1 11.5X -SQL ORC Vectorized 473 / 480 33.3 30.0 57.5X -SQL ORC Vectorized with copy 478 / 483 32.9 30.4 56.8X -SQL ORC MR 1858 / 1859 8.5 118.2 14.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 251 / 255 62.7 15.9 1.0X -ParquetReader Vectorized -> Row 255 / 259 61.8 16.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 34797 / 34830 0.5 2212.3 1.0X -SQL Json 17806 / 17828 0.9 1132.1 2.0X -SQL Parquet Vectorized 260 / 269 60.6 16.5 134.0X -SQL Parquet MR 2512 / 2534 6.3 159.7 13.9X -SQL ORC Vectorized 582 / 593 27.0 37.0 59.8X -SQL ORC Vectorized with copy 576 / 584 27.3 36.6 60.4X -SQL ORC MR 2309 / 2313 6.8 146.8 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Parquet Reader Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -ParquetReader Vectorized 350 / 363 44.9 22.3 1.0X -ParquetReader Vectorized -> Row 350 / 366 44.9 22.3 1.0X - - -================================================================================================ -Int and String Scan -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 22486 / 22590 0.5 2144.5 1.0X -SQL Json 14124 / 14195 0.7 1347.0 1.6X -SQL Parquet Vectorized 2342 / 2347 4.5 223.4 9.6X -SQL Parquet MR 4660 / 4664 2.2 444.4 4.8X -SQL ORC Vectorized 2378 / 2379 4.4 226.8 9.5X -SQL ORC Vectorized with copy 2548 / 2571 4.1 243.0 8.8X -SQL ORC MR 4206 / 4211 2.5 401.1 5.3X - - -================================================================================================ -Repeated String Scan -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 12150 / 12178 0.9 1158.7 1.0X -SQL Json 7012 / 7014 1.5 668.7 1.7X -SQL Parquet Vectorized 792 / 796 13.2 75.5 15.3X -SQL Parquet MR 1961 / 1975 5.3 187.0 6.2X -SQL ORC Vectorized 482 / 485 21.8 46.0 25.2X -SQL ORC Vectorized with copy 710 / 715 14.8 67.7 17.1X -SQL ORC MR 2081 / 2083 5.0 198.5 5.8X - - -================================================================================================ -Partitioned Table Scan -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Data column - CSV 31789 / 31791 0.5 2021.1 1.0X -Data column - Json 12873 / 12918 1.2 818.4 2.5X -Data column - Parquet Vectorized 267 / 280 58.9 17.0 119.1X -Data column - Parquet MR 3387 / 3402 4.6 215.3 9.4X -Data column - ORC Vectorized 391 / 453 40.2 24.9 81.2X -Data column - ORC Vectorized with copy 392 / 398 40.2 24.9 81.2X -Data column - ORC MR 2508 / 2512 6.3 159.4 12.7X -Partition column - CSV 6965 / 6977 2.3 442.8 4.6X -Partition column - Json 5563 / 5576 2.8 353.7 5.7X -Partition column - Parquet Vectorized 65 / 78 241.1 4.1 487.2X -Partition column - Parquet MR 1811 / 1811 8.7 115.1 17.6X -Partition column - ORC Vectorized 66 / 73 239.0 4.2 483.0X -Partition column - ORC Vectorized with copy 65 / 70 241.1 4.1 487.3X -Partition column - ORC MR 1775 / 1778 8.9 112.8 17.9X -Both columns - CSV 30032 / 30113 0.5 1909.4 1.1X -Both columns - Json 13941 / 13959 1.1 886.3 2.3X -Both columns - Parquet Vectorized 312 / 330 50.3 19.9 101.7X -Both columns - Parquet MR 3858 / 3862 4.1 245.3 8.2X -Both columns - ORC Vectorized 431 / 437 36.5 27.4 73.8X -Both column - ORC Vectorized with copy 523 / 529 30.1 33.3 60.7X -Both columns - ORC MR 2712 / 2805 5.8 172.4 11.7X - - -================================================================================================ -String with Nulls Scan -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13525 / 13823 0.8 1289.9 1.0X -SQL Json 9913 / 9921 1.1 945.3 1.4X -SQL Parquet Vectorized 1517 / 1517 6.9 144.7 8.9X -SQL Parquet MR 3996 / 4008 2.6 381.1 3.4X -ParquetReader Vectorized 1120 / 1128 9.4 106.8 12.1X -SQL ORC Vectorized 1203 / 1224 8.7 114.7 11.2X -SQL ORC Vectorized with copy 1639 / 1646 6.4 156.3 8.3X -SQL ORC MR 3720 / 3780 2.8 354.7 3.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 15860 / 15877 0.7 1512.5 1.0X -SQL Json 7676 / 7688 1.4 732.0 2.1X -SQL Parquet Vectorized 1072 / 1084 9.8 102.2 14.8X -SQL Parquet MR 2890 / 2897 3.6 275.6 5.5X -ParquetReader Vectorized 1052 / 1053 10.0 100.4 15.1X -SQL ORC Vectorized 1248 / 1248 8.4 119.0 12.7X -SQL ORC Vectorized with copy 1627 / 1637 6.4 155.2 9.7X -SQL ORC MR 3365 / 3369 3.1 320.9 4.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13401 / 13561 0.8 1278.1 1.0X -SQL Json 5253 / 5303 2.0 500.9 2.6X -SQL Parquet Vectorized 233 / 242 45.0 22.2 57.6X -SQL Parquet MR 1791 / 1796 5.9 170.8 7.5X -ParquetReader Vectorized 236 / 238 44.4 22.5 56.7X -SQL ORC Vectorized 453 / 473 23.2 43.2 29.6X -SQL ORC Vectorized with copy 573 / 577 18.3 54.7 23.4X -SQL ORC MR 1846 / 1850 5.7 176.0 7.3X - - -================================================================================================ -Single Column Scan From Wide Columns -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 3147 / 3148 0.3 3001.1 1.0X -SQL Json 2666 / 2693 0.4 2542.9 1.2X -SQL Parquet Vectorized 54 / 58 19.5 51.3 58.5X -SQL Parquet MR 220 / 353 4.8 209.9 14.3X -SQL ORC Vectorized 63 / 77 16.8 59.7 50.3X -SQL ORC Vectorized with copy 63 / 66 16.7 59.8 50.2X -SQL ORC MR 317 / 321 3.3 302.2 9.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 7902 / 7921 0.1 7536.2 1.0X -SQL Json 9467 / 9491 0.1 9028.6 0.8X -SQL Parquet Vectorized 73 / 79 14.3 69.8 108.0X -SQL Parquet MR 239 / 247 4.4 228.0 33.1X -SQL ORC Vectorized 78 / 84 13.4 74.6 101.0X -SQL ORC Vectorized with copy 78 / 88 13.4 74.4 101.3X -SQL ORC MR 910 / 918 1.2 867.6 8.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -SQL CSV 13539 / 13543 0.1 12912.0 1.0X -SQL Json 17420 / 17446 0.1 16613.1 0.8X -SQL Parquet Vectorized 103 / 120 10.2 98.1 131.6X -SQL Parquet MR 250 / 258 4.2 238.9 54.1X -SQL ORC Vectorized 99 / 104 10.6 94.6 136.5X -SQL ORC Vectorized with copy 100 / 106 10.5 95.6 135.1X -SQL ORC MR 1653 / 1659 0.6 1576.3 8.2X - - diff --git a/sql/core/benchmarks/JSONBenchmark-results.txt b/sql/core/benchmarks/JSONBenchmark-results.txt deleted file mode 100644 index 99937309a4145..0000000000000 --- a/sql/core/benchmarks/JSONBenchmark-results.txt +++ /dev/null @@ -1,37 +0,0 @@ -================================================================================================ -Benchmark for performance of JSON parsing -================================================================================================ - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON schema inferring: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -No encoding 62946 / 63310 1.6 629.5 1.0X -UTF-8 is set 112814 / 112866 0.9 1128.1 0.6X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -No encoding 16468 / 16553 6.1 164.7 1.0X -UTF-8 is set 16420 / 16441 6.1 164.2 1.0X - -Preparing data for benchmarking ... -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -JSON parsing of wide lines: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -No encoding 39789 / 40053 0.3 3978.9 1.0X -UTF-8 is set 39505 / 39584 0.3 3950.5 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Count a dataset with 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Select 10 columns + count() 15997 / 16015 0.6 1599.7 1.0X -Select 1 column + count() 13280 / 13326 0.8 1328.0 1.2X -count() 3006 / 3021 3.3 300.6 5.3X - - diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt deleted file mode 100644 index 8ceb5e7a7fe94..0000000000000 --- a/sql/core/benchmarks/JoinBenchmark-results.txt +++ /dev/null @@ -1,75 +0,0 @@ -================================================================================================ -Join Benchmark -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Join w long wholestage off 4464 / 4483 4.7 212.9 1.0X -Join w long wholestage on 289 / 339 72.6 13.8 15.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Join w long duplicated wholestage off 5662 / 5678 3.7 270.0 1.0X -Join w long duplicated wholestage on 332 / 345 63.1 15.8 17.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Join w 2 ints wholestage off 173174 / 173183 0.1 8257.6 1.0X -Join w 2 ints wholestage on 166350 / 198362 0.1 7932.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Join w 2 longs wholestage off 7055 / 7214 3.0 336.4 1.0X -Join w 2 longs wholestage on 1869 / 1985 11.2 89.1 3.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Join w 2 longs duplicated wholestage off 19256 / 20283 1.1 918.2 1.0X -Join w 2 longs duplicated wholestage on 2467 / 2544 8.5 117.7 7.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -outer join w long wholestage off 3756 / 3761 5.6 179.1 1.0X -outer join w long wholestage on 218 / 250 96.2 10.4 17.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -semi join w long wholestage off 2393 / 2416 8.8 114.1 1.0X -semi join w long wholestage on 214 / 218 97.9 10.2 11.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort merge join wholestage off 2318 / 2392 0.9 1105.3 1.0X -sort merge join wholestage on 1669 / 1811 1.3 795.9 1.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sort merge join with duplicates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sort merge join with duplicates wholestage off 2966 / 2976 0.7 1414.5 1.0X -sort merge join with duplicates wholestage on 2413 / 2641 0.9 1150.5 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -shuffle hash join wholestage off 1475 / 1479 2.8 351.7 1.0X -shuffle hash join wholestage on 1209 / 1238 3.5 288.3 1.2X - - diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt deleted file mode 100644 index 85acd57893655..0000000000000 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ /dev/null @@ -1,120 +0,0 @@ -================================================================================================ -filter & aggregate without group -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -range/filter/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -range/filter/sum wholestage off 47752 / 48952 43.9 22.8 1.0X -range/filter/sum wholestage on 3123 / 3558 671.5 1.5 15.3X - - -================================================================================================ -range/limit/sum -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -range/limit/sum: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -range/limit/sum wholestage off 229 / 236 2288.9 0.4 1.0X -range/limit/sum wholestage on 257 / 267 2041.0 0.5 0.9X - - -================================================================================================ -sample -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sample with replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sample with replacement wholestage off 12908 / 13076 10.2 98.5 1.0X -sample with replacement wholestage on 7334 / 7346 17.9 56.0 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -sample without replacement: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -sample without replacement wholestage off 3082 / 3095 42.5 23.5 1.0X -sample without replacement wholestage on 1125 / 1211 116.5 8.6 2.7X - - -================================================================================================ -collect -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -collect: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -collect 1 million 291 / 311 3.6 277.3 1.0X -collect 2 millions 552 / 564 1.9 526.6 0.5X -collect 4 millions 1104 / 1108 0.9 1053.0 0.3X - - -================================================================================================ -collect limit -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -collect limit: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -collect limit 1 million 311 / 340 3.4 296.2 1.0X -collect limit 2 millions 581 / 614 1.8 554.4 0.5X - - -================================================================================================ -generate explode -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate explode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate explode array wholestage off 15211 / 15368 1.1 906.6 1.0X -generate explode array wholestage on 10761 / 10776 1.6 641.4 1.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate explode map: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate explode map wholestage off 22128 / 22578 0.8 1318.9 1.0X -generate explode map wholestage on 16421 / 16520 1.0 978.8 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate posexplode array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate posexplode array wholestage off 17108 / 18019 1.0 1019.7 1.0X -generate posexplode array wholestage on 11715 / 11804 1.4 698.3 1.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate inline array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate inline array wholestage off 16358 / 16418 1.0 975.0 1.0X -generate inline array wholestage on 11152 / 11472 1.5 664.7 1.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate big struct array wholestage off 708 / 776 0.1 11803.5 1.0X -generate big struct array wholestage on 535 / 589 0.1 8913.9 1.3X - - -================================================================================================ -generate regular generator -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -generate stack: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -generate stack wholestage off 29082 / 29393 0.6 1733.4 1.0X -generate stack wholestage on 21066 / 21128 0.8 1255.6 1.4X - - diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt deleted file mode 100644 index 21766e0fd8664..0000000000000 --- a/sql/core/benchmarks/RangeBenchmark-results.txt +++ /dev/null @@ -1,16 +0,0 @@ -================================================================================================ -range -================================================================================================ - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_161-b12 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - -range: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -full scan 12674 / 12840 41.4 24.2 1.0X -limit after range 33 / 37 15900.2 0.1 384.4X -filter after range 969 / 985 541.0 1.8 13.1X -count after range 42 / 42 12510.5 0.1 302.4X -count after limit after range 32 / 33 16337.0 0.1 394.9X - - diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt index 6347a6ac6b67c..0b9f791ac85e4 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -1,145 +1,117 @@ -================================================================================================ -parsing large select expressions -================================================================================================ +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 select expressions 6 / 13 0.0 5997373.0 1.0X -100 select expressions 7 / 10 0.0 7204596.0 0.8X -2500 select expressions 103 / 107 0.0 102962705.0 0.1X +1 select expressions 2 / 4 0.0 2050147.0 1.0X +100 select expressions 6 / 7 0.0 6123412.0 0.3X +2500 select expressions 135 / 141 0.0 134623148.0 0.0X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -================================================================================================ -many column field read and write -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 cols x 100000 rows (read in-mem) 40 / 51 2.5 396.5 1.0X -1 cols x 100000 rows (exec in-mem) 41 / 48 2.4 414.4 1.0X -1 cols x 100000 rows (read parquet) 61 / 70 1.6 610.2 0.6X -1 cols x 100000 rows (write parquet) 209 / 233 0.5 2086.1 0.2X -100 cols x 1000 rows (read in-mem) 43 / 49 2.3 433.8 0.9X -100 cols x 1000 rows (exec in-mem) 57 / 66 1.8 568.4 0.7X -100 cols x 1000 rows (read parquet) 60 / 66 1.7 599.0 0.7X -100 cols x 1000 rows (write parquet) 212 / 224 0.5 2120.6 0.2X -2500 cols x 40 rows (read in-mem) 268 / 275 0.4 2676.5 0.1X -2500 cols x 40 rows (exec in-mem) 494 / 504 0.2 4936.9 0.1X -2500 cols x 40 rows (read parquet) 132 / 139 0.8 1319.7 0.3X -2500 cols x 40 rows (write parquet) 371 / 381 0.3 3710.1 0.1X - - -================================================================================================ -wide shallowly nested struct field read and write -================================================================================================ +1 cols x 100000 rows (read in-mem) 16 / 18 6.3 158.6 1.0X +1 cols x 100000 rows (exec in-mem) 17 / 19 6.0 166.7 1.0X +1 cols x 100000 rows (read parquet) 24 / 26 4.3 235.1 0.7X +1 cols x 100000 rows (write parquet) 81 / 85 1.2 811.3 0.2X +100 cols x 1000 rows (read in-mem) 17 / 19 6.0 166.2 1.0X +100 cols x 1000 rows (exec in-mem) 25 / 27 4.0 249.2 0.6X +100 cols x 1000 rows (read parquet) 23 / 25 4.4 226.0 0.7X +100 cols x 1000 rows (write parquet) 83 / 87 1.2 831.0 0.2X +2500 cols x 40 rows (read in-mem) 132 / 137 0.8 1322.9 0.1X +2500 cols x 40 rows (exec in-mem) 326 / 330 0.3 3260.6 0.0X +2500 cols x 40 rows (read parquet) 831 / 839 0.1 8305.8 0.0X +2500 cols x 40 rows (write parquet) 237 / 245 0.4 2372.6 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 37 / 43 2.7 373.6 1.0X -1 wide x 100000 rows (exec in-mem) 47 / 54 2.1 472.7 0.8X -1 wide x 100000 rows (read parquet) 132 / 145 0.8 1316.5 0.3X -1 wide x 100000 rows (write parquet) 205 / 232 0.5 2046.3 0.2X -100 wide x 1000 rows (read in-mem) 68 / 79 1.5 676.3 0.6X -100 wide x 1000 rows (exec in-mem) 88 / 97 1.1 882.2 0.4X -100 wide x 1000 rows (read parquet) 197 / 234 0.5 1971.8 0.2X -100 wide x 1000 rows (write parquet) 236 / 249 0.4 2359.6 0.2X -2500 wide x 40 rows (read in-mem) 77 / 85 1.3 768.0 0.5X -2500 wide x 40 rows (exec in-mem) 386 / 393 0.3 3855.2 0.1X -2500 wide x 40 rows (read parquet) 1741 / 1765 0.1 17408.3 0.0X -2500 wide x 40 rows (write parquet) 243 / 256 0.4 2425.2 0.2X +1 wide x 100000 rows (read in-mem) 15 / 17 6.6 151.0 1.0X +1 wide x 100000 rows (exec in-mem) 20 / 22 5.1 196.6 0.8X +1 wide x 100000 rows (read parquet) 59 / 63 1.7 592.8 0.3X +1 wide x 100000 rows (write parquet) 81 / 87 1.2 814.6 0.2X +100 wide x 1000 rows (read in-mem) 21 / 25 4.8 208.7 0.7X +100 wide x 1000 rows (exec in-mem) 72 / 81 1.4 718.5 0.2X +100 wide x 1000 rows (read parquet) 75 / 85 1.3 752.6 0.2X +100 wide x 1000 rows (write parquet) 88 / 95 1.1 876.7 0.2X +2500 wide x 40 rows (read in-mem) 28 / 34 3.5 282.2 0.5X +2500 wide x 40 rows (exec in-mem) 1269 / 1284 0.1 12688.1 0.0X +2500 wide x 40 rows (read parquet) 549 / 578 0.2 5493.4 0.0X +2500 wide x 40 rows (write parquet) 96 / 104 1.0 959.1 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - -================================================================================================ -deeply nested struct field read and write -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 deep x 100000 rows (read in-mem) 35 / 42 2.9 350.2 1.0X -1 deep x 100000 rows (exec in-mem) 40 / 43 2.5 399.5 0.9X -1 deep x 100000 rows (read parquet) 69 / 73 1.4 691.6 0.5X -1 deep x 100000 rows (write parquet) 203 / 224 0.5 2025.9 0.2X -100 deep x 1000 rows (read in-mem) 70 / 75 1.4 703.7 0.5X -100 deep x 1000 rows (exec in-mem) 654 / 684 0.2 6539.9 0.1X -100 deep x 1000 rows (read parquet) 10503 / 10550 0.0 105030.5 0.0X -100 deep x 1000 rows (write parquet) 235 / 243 0.4 2353.2 0.1X -250 deep x 400 rows (read in-mem) 249 / 259 0.4 2492.6 0.1X -250 deep x 400 rows (exec in-mem) 3842 / 3854 0.0 38424.8 0.0X -250 deep x 400 rows (read parquet) 153080 / 153444 0.0 1530796.1 0.0X -250 deep x 400 rows (write parquet) 434 / 441 0.2 4344.6 0.1X - +1 deep x 100000 rows (read in-mem) 14 / 16 7.0 143.8 1.0X +1 deep x 100000 rows (exec in-mem) 17 / 19 5.9 169.7 0.8X +1 deep x 100000 rows (read parquet) 33 / 35 3.1 327.0 0.4X +1 deep x 100000 rows (write parquet) 79 / 84 1.3 786.9 0.2X +100 deep x 1000 rows (read in-mem) 21 / 24 4.7 211.3 0.7X +100 deep x 1000 rows (exec in-mem) 221 / 235 0.5 2214.5 0.1X +100 deep x 1000 rows (read parquet) 1928 / 1952 0.1 19277.1 0.0X +100 deep x 1000 rows (write parquet) 91 / 96 1.1 909.5 0.2X +250 deep x 400 rows (read in-mem) 57 / 61 1.8 567.1 0.3X +250 deep x 400 rows (exec in-mem) 1329 / 1385 0.1 13291.8 0.0X +250 deep x 400 rows (read parquet) 36563 / 36750 0.0 365630.2 0.0X +250 deep x 400 rows (write parquet) 126 / 130 0.8 1262.0 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -================================================================================================ -bushy struct field read and write -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 x 1 deep x 100000 rows (read in-mem) 37 / 42 2.7 370.2 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 43 / 47 2.4 425.3 0.9X -1 x 1 deep x 100000 rows (read parquet) 48 / 51 2.1 478.7 0.8X -1 x 1 deep x 100000 rows (write parquet) 204 / 215 0.5 2042.0 0.2X -128 x 8 deep x 1000 rows (read in-mem) 32 / 37 3.1 318.6 1.2X -128 x 8 deep x 1000 rows (exec in-mem) 91 / 96 1.1 906.6 0.4X -128 x 8 deep x 1000 rows (read parquet) 351 / 379 0.3 3510.3 0.1X -128 x 8 deep x 1000 rows (write parquet) 199 / 203 0.5 1988.3 0.2X -1024 x 11 deep x 100 rows (read in-mem) 73 / 76 1.4 730.4 0.5X -1024 x 11 deep x 100 rows (exec in-mem) 327 / 334 0.3 3267.2 0.1X -1024 x 11 deep x 100 rows (read parquet) 2063 / 2078 0.0 20629.2 0.0X -1024 x 11 deep x 100 rows (write parquet) 248 / 266 0.4 2475.1 0.1X - - -================================================================================================ -wide array field read and write -================================================================================================ +1 x 1 deep x 100000 rows (read in-mem) 13 / 15 7.8 127.7 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 15 / 17 6.6 151.5 0.8X +1 x 1 deep x 100000 rows (read parquet) 20 / 23 5.0 198.3 0.6X +1 x 1 deep x 100000 rows (write parquet) 77 / 82 1.3 770.4 0.2X +128 x 8 deep x 1000 rows (read in-mem) 12 / 14 8.2 122.5 1.0X +128 x 8 deep x 1000 rows (exec in-mem) 124 / 140 0.8 1241.2 0.1X +128 x 8 deep x 1000 rows (read parquet) 69 / 74 1.4 693.9 0.2X +128 x 8 deep x 1000 rows (write parquet) 78 / 83 1.3 777.7 0.2X +1024 x 11 deep x 100 rows (read in-mem) 25 / 29 4.1 246.1 0.5X +1024 x 11 deep x 100 rows (exec in-mem) 1197 / 1223 0.1 11974.6 0.0X +1024 x 11 deep x 100 rows (read parquet) 426 / 433 0.2 4263.7 0.0X +1024 x 11 deep x 100 rows (write parquet) 91 / 98 1.1 913.5 0.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 33 / 38 3.0 328.4 1.0X -1 wide x 100000 rows (exec in-mem) 40 / 44 2.5 402.7 0.8X -1 wide x 100000 rows (read parquet) 83 / 91 1.2 826.6 0.4X -1 wide x 100000 rows (write parquet) 204 / 218 0.5 2039.1 0.2X -100 wide x 1000 rows (read in-mem) 28 / 31 3.6 277.2 1.2X -100 wide x 1000 rows (exec in-mem) 34 / 37 2.9 343.2 1.0X -100 wide x 1000 rows (read parquet) 56 / 61 1.8 556.4 0.6X -100 wide x 1000 rows (write parquet) 202 / 206 0.5 2017.3 0.2X -2500 wide x 40 rows (read in-mem) 29 / 30 3.5 286.4 1.1X -2500 wide x 40 rows (exec in-mem) 33 / 39 3.0 330.2 1.0X -2500 wide x 40 rows (read parquet) 54 / 66 1.8 544.0 0.6X -2500 wide x 40 rows (write parquet) 196 / 208 0.5 1959.2 0.2X +1 wide x 100000 rows (read in-mem) 14 / 16 7.0 143.2 1.0X +1 wide x 100000 rows (exec in-mem) 17 / 19 5.9 170.9 0.8X +1 wide x 100000 rows (read parquet) 43 / 46 2.3 434.1 0.3X +1 wide x 100000 rows (write parquet) 78 / 83 1.3 777.6 0.2X +100 wide x 1000 rows (read in-mem) 11 / 13 9.0 111.5 1.3X +100 wide x 1000 rows (exec in-mem) 13 / 15 7.8 128.3 1.1X +100 wide x 1000 rows (read parquet) 24 / 27 4.1 245.0 0.6X +100 wide x 1000 rows (write parquet) 74 / 80 1.4 740.5 0.2X +2500 wide x 40 rows (read in-mem) 11 / 13 9.1 109.5 1.3X +2500 wide x 40 rows (exec in-mem) 13 / 15 7.7 129.4 1.1X +2500 wide x 40 rows (read parquet) 24 / 26 4.1 241.3 0.6X +2500 wide x 40 rows (write parquet) 75 / 81 1.3 751.8 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 +Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz - -================================================================================================ -wide map field read and write -================================================================================================ - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 31 / 34 3.3 305.7 1.0X -1 wide x 100000 rows (exec in-mem) 39 / 44 2.6 390.0 0.8X -1 wide x 100000 rows (read parquet) 125 / 132 0.8 1250.5 0.2X -1 wide x 100000 rows (write parquet) 198 / 213 0.5 1979.9 0.2X -100 wide x 1000 rows (read in-mem) 21 / 23 4.7 212.7 1.4X -100 wide x 1000 rows (exec in-mem) 28 / 32 3.5 283.3 1.1X -100 wide x 1000 rows (read parquet) 68 / 73 1.5 683.0 0.4X -100 wide x 1000 rows (write parquet) 188 / 206 0.5 1882.1 0.2X -2500 wide x 40 rows (read in-mem) 25 / 28 4.0 252.2 1.2X -2500 wide x 40 rows (exec in-mem) 32 / 34 3.1 318.5 1.0X -2500 wide x 40 rows (read parquet) 69 / 73 1.4 691.5 0.4X -2500 wide x 40 rows (write parquet) 193 / 202 0.5 1932.8 0.2X - +1 wide x 100000 rows (read in-mem) 16 / 18 6.2 162.6 1.0X +1 wide x 100000 rows (exec in-mem) 21 / 23 4.8 208.2 0.8X +1 wide x 100000 rows (read parquet) 54 / 59 1.8 543.6 0.3X +1 wide x 100000 rows (write parquet) 80 / 86 1.2 804.5 0.2X +100 wide x 1000 rows (read in-mem) 11 / 13 8.7 114.5 1.4X +100 wide x 1000 rows (exec in-mem) 14 / 16 7.0 143.5 1.1X +100 wide x 1000 rows (read parquet) 30 / 32 3.3 300.4 0.5X +100 wide x 1000 rows (write parquet) 75 / 80 1.3 749.9 0.2X +2500 wide x 40 rows (read in-mem) 13 / 15 7.8 128.1 1.3X +2500 wide x 40 rows (exec in-mem) 15 / 18 6.5 153.6 1.1X +2500 wide x 40 rows (read parquet) 30 / 33 3.3 304.4 0.5X +2500 wide x 40 rows (write parquet) 77 / 83 1.3 768.5 0.2X diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java b/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java index 3d0511b7ba838..74c9c05992719 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/BufferedRowIterator.java @@ -73,6 +73,16 @@ public void append(InternalRow row) { currentRows.add(row); } + /** + * Returns whether this iterator should stop fetching next row from [[CodegenSupport#inputRDDs]]. + * + * If it returns true, the caller should exit the loop that [[InputAdapter]] generates. + * This interface is mainly used to limit the number of input rows. + */ + public boolean stopEarly() { + return false; + } + /** * Returns whether `processNext()` should stop processing next row from `input` or not. * 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 a046127c3edb4..ae27690f2e5ba 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 @@ -199,13 +199,13 @@ class Column(val expr: Expression) extends Logging { /** * Extracts a value or values from a complex type. * The following types of extraction are supported: - *

    + * + * - Given an Array, an integer ordinal can be used to retrieve a single value. + * - Given a Map, a key of the correct type can be used to retrieve an individual value. + * - Given a Struct, a string fieldName can be used to extract that field. + * - Given an Array of Structs, a string fieldName can be used to extract filed + * of every struct in that array, and return an Array of fields + * * @group expr_ops * @since 1.4.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 95c97e5c9433c..fe69f252d43e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -22,17 +22,16 @@ import java.util.{Locale, Properties} import scala.collection.JavaConverters._ import com.fasterxml.jackson.databind.ObjectMapper +import com.univocity.parsers.csv.CsvParser import org.apache.spark.Partition import org.apache.spark.annotation.InterfaceStability import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} -import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser} import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource @@ -446,7 +445,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val createParser = CreateJacksonParser.string _ val parsed = jsonDataset.rdd.mapPartitions { iter => - val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val rawParser = new JacksonParser(actualSchema, parsedOptions) val parser = new FailureSafeParser[String]( input => rawParser.parse(input, createParser, UTF8String.fromString), parsedOptions.parseMode, @@ -507,11 +506,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { StructType(schema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val linesWithoutHeader: RDD[String] = maybeFirstLine.map { firstLine => - val headerChecker = new CSVHeaderChecker( + val parser = new CsvParser(parsedOptions.asParserSettings) + val columnNames = parser.parseLine(firstLine) + CSVDataSource.checkHeaderColumnNames( actualSchema, - parsedOptions, - source = s"CSV source: $csvDataset") - headerChecker.checkHeaderColumnNames(firstLine) + columnNames, + csvDataset.getClass.getCanonicalName, + parsedOptions.enforceSchema, + sparkSession.sessionState.conf.caseSensitiveAnalysis) filteredLines.rdd.mapPartitions(CSVUtils.filterHeaderLine(_, firstLine, parsedOptions)) }.getOrElse(filteredLines.rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 5a28870f5d3c2..55e538f49feda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -47,12 +47,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies the behavior when data or table already exists. Options include: - *
      - *
    • `SaveMode.Overwrite`: overwrite the existing data.
    • - *
    • `SaveMode.Append`: append the data.
    • - *
    • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
    • - *
    • `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
    • - *
    + * - `SaveMode.Overwrite`: overwrite the existing data. + * - `SaveMode.Append`: append the data. + * - `SaveMode.Ignore`: ignore the operation (i.e. no-op). + * - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime. * * @since 1.4.0 */ @@ -63,12 +61,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies the behavior when data or table already exists. Options include: - *
      - *
    • `overwrite`: overwrite the existing data.
    • - *
    • `append`: append the data.
    • - *
    • `ignore`: ignore the operation (i.e. no-op).
    • - *
    • `error` or `errorifexists`: default option, throw an exception at runtime.
    • - *
    + * - `overwrite`: overwrite the existing data. + * - `append`: append the data. + * - `ignore`: ignore the operation (i.e. no-op). + * - `error` or `errorifexists`: default option, throw an exception at runtime. * * @since 1.4.0 */ @@ -167,10 +163,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * Partitions the output by the given columns on the file system. If specified, the output is * laid out on the file system similar to Hive's partitioning scheme. As an example, when we * partition a dataset by year and then month, the directory layout would look like: - *
      - *
    • year=2016/month=01/
    • - *
    • year=2016/month=02/
    • - *
    + * + * - year=2016/month=01/ + * - year=2016/month=02/ * * Partitioning is one of the most widely used techniques to optimize physical data layout. * It provides a coarse-grained index for skipping unnecessary data reads when queries have @@ -672,8 +667,17 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ private def runCommand(session: SparkSession, name: String)(command: LogicalPlan): Unit = { val qe = session.sessionState.executePlan(command) - // call `QueryExecution.toRDD` to trigger the execution of commands. - SQLExecution.withNewExecutionId(session, qe, Some(name))(qe.toRdd) + try { + val start = System.nanoTime() + // call `QueryExecution.toRDD` to trigger the execution of commands. + SQLExecution.withNewExecutionId(session, qe)(qe.toRdd) + val end = System.nanoTime() + session.listenerManager.onSuccess(name, qe, end - start) + } catch { + case e: Exception => + session.listenerManager.onFailure(name, qe, e) + throw e + } } /////////////////////////////////////////////////////////////////////////////////////// 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 c91b0d778fab1..fa14aa14ee968 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 @@ -1087,7 +1087,7 @@ class Dataset[T] private[sql]( // Note that we do this before joining them, to enable the join operator to return null for one // side, in cases like outer-join. val left = { - val combined = if (!this.exprEnc.isSerializedAsStruct) { + val combined = if (this.exprEnc.flat) { assert(joined.left.output.length == 1) Alias(joined.left.output.head, "_1")() } else { @@ -1097,7 +1097,7 @@ class Dataset[T] private[sql]( } val right = { - val combined = if (!other.exprEnc.isSerializedAsStruct) { + val combined = if (other.exprEnc.flat) { assert(joined.right.output.length == 1) Alias(joined.right.output.head, "_2")() } else { @@ -1110,14 +1110,14 @@ class Dataset[T] private[sql]( // combine the outputs of each join side. val conditionExpr = joined.condition.get transformUp { case a: Attribute if joined.left.outputSet.contains(a) => - if (!this.exprEnc.isSerializedAsStruct) { + if (this.exprEnc.flat) { left.output.head } else { val index = joined.left.output.indexWhere(_.exprId == a.exprId) GetStructField(left.output.head, index) } case a: Attribute if joined.right.outputSet.contains(a) => - if (!other.exprEnc.isSerializedAsStruct) { + if (other.exprEnc.flat) { right.output.head } else { val index = joined.right.output.indexWhere(_.exprId == a.exprId) @@ -1390,7 +1390,7 @@ class Dataset[T] private[sql]( implicit val encoder = c1.encoder val project = Project(c1.withInputType(exprEnc, logicalPlan.output).named :: Nil, logicalPlan) - if (!encoder.isSerializedAsStruct) { + if (encoder.flat) { new Dataset[U1](sparkSession, project, encoder) } else { // Flattens inner fields of U1 @@ -3356,11 +3356,21 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { - SQLExecution.withNewExecutionId(sparkSession, qe, Some(name)) { + try { qe.executedPlan.foreach { plan => plan.resetMetrics() } - action(qe.executedPlan) + val start = System.nanoTime() + val result = SQLExecution.withNewExecutionId(sparkSession, qe) { + action(qe.executedPlan) + } + val end = System.nanoTime() + sparkSession.listenerManager.onSuccess(name, qe, end - start) + result + } catch { + case e: Exception => + sparkSession.listenerManager.onFailure(name, qe, e) + throw e } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index 52b8c839643e7..b21c50af18433 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -130,11 +130,8 @@ abstract class ForeachWriter[T] extends Serializable { * Called when stopping to process one partition of new data in the executor side. This is * guaranteed to be called either `open` returns `true` or `false`. However, * `close` won't be called in the following cases: - * - *
      - *
    • JVM crashes without throwing a `Throwable`
    • - *
    • `open` throws a `Throwable`.
    • - *
    + * - JVM crashes without throwing a `Throwable` + * - `open` throws a `Throwable`. * * @param errorOrNull the error thrown during processing data or null if there was no error. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 555bcdffb6ee4..6bab21dca0cbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -457,7 +457,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( val encoders = columns.map(_.encoder) val namedColumns = columns.map(_.withInputType(vExprEnc, dataAttributes).named) - val keyColumn = if (!kExprEnc.isSerializedAsStruct) { + val keyColumn = if (kExprEnc.flat) { assert(groupingAttributes.length == 1) groupingAttributes.head } else { 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 1b7e969a7192e..af6018472cb03 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 @@ -755,6 +755,289 @@ class SQLContext private[sql](val sparkSession: SparkSession) sessionState.catalog.listTables(databaseName).map(_.table).toArray } + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + // Deprecated methods + //////////////////////////////////////////////////////////////////////////// + //////////////////////////////////////////////////////////////////////////// + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: RDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + createDataFrame(rowRDD, schema) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: RDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * @deprecated As of 1.3.0, replaced by `createDataFrame()`. + */ + @deprecated("Use createDataFrame instead.", "1.3.0") + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = { + createDataFrame(rdd, beanClass) + } + + /** + * Loads a Parquet file, returning the result as a `DataFrame`. This function returns an empty + * `DataFrame` if no paths are passed in. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().parquet()`. + */ + @deprecated("Use read.parquet() instead.", "1.4.0") + @scala.annotation.varargs + def parquetFile(paths: String*): DataFrame = { + if (paths.isEmpty) { + emptyDataFrame + } else { + read.parquet(paths : _*) + } + } + + /** + * Loads a JSON file (one object per line), returning the result as a `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String): DataFrame = { + read.json(path) + } + + /** + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, schema: StructType): DataFrame = { + read.schema(schema).json(path) + } + + /** + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonFile(path: String, samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(path) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a + * `DataFrame`. + * It goes through the entire dataset once to determine the schema. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String]): DataFrame = read.json(json) + + /** + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an JavaRDD[String] storing JSON objects (one object per record) and applies the given + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { + read.schema(schema).json(json) + } + + /** + * Loads an RDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: RDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Loads a JavaRDD[String] storing JSON objects (one object per record) inferring the + * schema, returning the result as a `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().json()`. + */ + @deprecated("Use read.json() instead.", "1.4.0") + def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { + read.option("samplingRatio", samplingRatio.toString).json(json) + } + + /** + * Returns the dataset stored at path as a DataFrame, + * using the default data source configured by spark.sql.sources.default. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().load(path)`. + */ + @deprecated("Use read.load(path) instead.", "1.4.0") + def load(path: String): DataFrame = { + read.load(path) + } + + /** + * Returns the dataset stored at path as a DataFrame, using the given data source. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).load(path)`. + */ + @deprecated("Use read.format(source).load(path) instead.", "1.4.0") + def load(path: String, source: String): DataFrame = { + read.format(source).load(path) + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: java.util.Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by `read().format(source).options(options).load()`. + */ + @deprecated("Use read.format(source).options(options).load() instead.", "1.4.0") + def load(source: String, options: Map[String, String]): DataFrame = { + read.options(options).format(source).load() + } + + /** + * (Java-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load( + source: String, + schema: StructType, + options: java.util.Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * (Scala-specific) Returns the dataset specified by the given data source and + * a set of options as a DataFrame, using the given schema as the schema of the DataFrame. + * + * @group genericdata + * @deprecated As of 1.4.0, replaced by + * `read().format(source).schema(schema).options(options).load()`. + */ + @deprecated("Use read.format(source).schema(schema).options(options).load() instead.", "1.4.0") + def load(source: String, schema: StructType, options: Map[String, String]): DataFrame = { + read.format(source).schema(schema).options(options).load() + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String): DataFrame = { + read.jdbc(url, table, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. Partitions of the table will be retrieved in parallel based on the parameters + * passed to this function. + * + * @param columnName the name of a column of integral type that will be used for partitioning. + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride + * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split + * evenly into this many partitions + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc( + url: String, + table: String, + columnName: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): DataFrame = { + read.jdbc(url, table, columnName, lowerBound, upperBound, numPartitions, new Properties) + } + + /** + * Construct a `DataFrame` representing the database table accessible via JDBC URL + * url named table. The theParts parameter gives a list expressions + * suitable for inclusion in WHERE clauses; each one defines one partition + * of the `DataFrame`. + * + * @group specificdata + * @deprecated As of 1.4.0, replaced by `read().jdbc()`. + */ + @deprecated("Use read.jdbc() instead.", "1.4.0") + def jdbc(url: String, table: String, theParts: Array[String]): DataFrame = { + read.jdbc(url, table, theParts, new Properties) + } } /** @@ -815,29 +1098,16 @@ object SQLContext { data: Iterator[_], beanClass: Class[_], attrs: Seq[AttributeReference]): Iterator[InternalRow] = { - def createStructConverter(cls: Class[_], fieldTypes: Seq[DataType]): Any => InternalRow = { - val methodConverters = - JavaTypeInference.getJavaBeanReadableProperties(cls).zip(fieldTypes) - .map { case (property, fieldType) => - val method = property.getReadMethod - method -> createConverter(method.getReturnType, fieldType) - } - value => - if (value == null) { - null - } else { - new GenericInternalRow( - methodConverters.map { case (method, converter) => - converter(method.invoke(value)) - }) - } + val extractors = + JavaTypeInference.getJavaBeanReadableProperties(beanClass).map(_.getReadMethod) + val methodsToConverts = extractors.zip(attrs).map { case (e, attr) => + (e, CatalystTypeConverters.createToCatalystConverter(attr.dataType)) } - def createConverter(cls: Class[_], dataType: DataType): Any => Any = dataType match { - case struct: StructType => createStructConverter(cls, struct.map(_.dataType)) - case _ => CatalystTypeConverters.createToCatalystConverter(dataType) + data.map { element => + new GenericInternalRow( + methodsToConverts.map { case (e, convert) => convert(e.invoke(element)) } + ): InternalRow } - val dataConverter = createStructConverter(beanClass, attrs.map(_.dataType)) - data.map(dataConverter) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 3f0b8208612d7..9dcc67e15db4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -84,17 +84,8 @@ class SparkSession private( // The call site where this SparkSession was constructed. private val creationSite: CallSite = Utils.getCallSite() - /** - * Constructor used in Pyspark. Contains explicit application of Spark Session Extensions - * which otherwise only occurs during getOrCreate. We cannot add this to the default constructor - * since that would cause every new session to reinvoke Spark Session Extensions on the currently - * running extensions. - */ private[sql] def this(sc: SparkContext) { - this(sc, None, None, - SparkSession.applyExtensions( - sc.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS), - new SparkSessionExtensions)) + this(sc, None, None, new SparkSessionExtensions) } sparkContext.assertNotStopped() @@ -946,9 +937,23 @@ object SparkSession extends Logging { // Do not update `SparkConf` for existing `SparkContext`, as it's shared by all sessions. } - applyExtensions( - sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS), - extensions) + // Initialize extensions if the user has defined a configurator class. + val extensionConfOption = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) + if (extensionConfOption.isDefined) { + val extensionConfClassName = extensionConfOption.get + try { + val extensionConfClass = Utils.classForName(extensionConfClassName) + val extensionConf = extensionConfClass.newInstance() + .asInstanceOf[SparkSessionExtensions => Unit] + extensionConf(extensions) + } catch { + // Ignore the error if we cannot find the class or when the class has the wrong type. + case e @ (_: ClassCastException | + _: ClassNotFoundException | + _: NoClassDefFoundError) => + logWarning(s"Cannot use $extensionConfClassName to configure session extensions.", e) + } + } session = new SparkSession(sparkContext, None, None, extensions) options.foreach { case (k, v) => session.initialSessionOptions.put(k, v) } @@ -1134,29 +1139,4 @@ object SparkSession extends Logging { SparkSession.clearDefaultSession() } } - - /** - * Initialize extensions for given extension classname. This class will be applied to the - * extensions passed into this function. - */ - private def applyExtensions( - extensionOption: Option[String], - extensions: SparkSessionExtensions): SparkSessionExtensions = { - if (extensionOption.isDefined) { - val extensionConfClassName = extensionOption.get - try { - val extensionConfClass = Utils.classForName(extensionConfClassName) - val extensionConf = extensionConfClass.newInstance() - .asInstanceOf[SparkSessionExtensions => Unit] - extensionConf(extensions) - } catch { - // Ignore the error if we cannot find the class or when the class has the wrong type. - case e@(_: ClassCastException | - _: ClassNotFoundException | - _: NoClassDefFoundError) => - logWarning(s"Cannot use $extensionConfClassName to configure session extensions.", e) - } - } - extensions - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index a4864344b2d25..f99c108161f94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -20,10 +20,6 @@ package org.apache.spark.sql import scala.collection.mutable import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule @@ -34,15 +30,12 @@ import org.apache.spark.sql.catalyst.rules.Rule * regarding binary compatibility and source compatibility of methods here. * * This current provides the following extension points: - * - *
      - *
    • Analyzer Rules.
    • - *
    • Check Analysis Rules.
    • - *
    • Optimizer Rules.
    • - *
    • Planning Strategies.
    • - *
    • Customized Parser.
    • - *
    • (External) Catalog listeners.
    • - *
    + * - Analyzer Rules. + * - Check Analysis Rules + * - Optimizer Rules. + * - Planning Strategies. + * - Customized Parser. + * - (External) Catalog listeners. * * The extensions can be used by calling withExtension on the [[SparkSession.Builder]], for * example: @@ -72,7 +65,6 @@ class SparkSessionExtensions { type CheckRuleBuilder = SparkSession => LogicalPlan => Unit type StrategyBuilder = SparkSession => Strategy type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface - type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder) private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] @@ -176,21 +168,4 @@ class SparkSessionExtensions { def injectParser(builder: ParserBuilder): Unit = { parserBuilders += builder } - - private[this] val injectedFunctions = mutable.Buffer.empty[FunctionDescription] - - private[sql] def registerFunctions(functionRegistry: FunctionRegistry) = { - for ((name, expressionInfo, function) <- injectedFunctions) { - functionRegistry.registerFunction(name, expressionInfo, function) - } - functionRegistry - } - - /** - * Injects a custom function into the [[org.apache.spark.sql.catalyst.analysis.FunctionRegistry]] - * at runtime for all sessions. - */ - def injectFunction(functionDescription: FunctionDescription): Unit = { - injectedFunctions += functionDescription - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index aa3a6c3bf122f..c37ba0c60c3d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -113,7 +113,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends (0 to 22).foreach { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor[A$i]).toOption :: $s"}) + val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor[A$i] :: $s"}) println(s""" |/** | * Registers a deterministic Scala closure of $x arguments as user-defined function (UDF). @@ -122,11 +122,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | */ |def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas: Seq[Option[ScalaReflection.Schema]] = $inputSchemas + | val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try($inputSchemas).toOption | def builder(e: Seq[Expression]) = if (e.length == $x) { - | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - | if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - | Some(name), nullable, udfDeterministic = true) + | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + | udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $x; Found: " + e.length) @@ -152,7 +151,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, e.map(_ => true), udfName = Some(name)) + | ScalaUDF($funcCall, returnType, e, udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -169,11 +168,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -190,11 +188,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -211,11 +208,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -232,11 +228,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -253,11 +248,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -274,11 +268,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -295,11 +288,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -316,11 +308,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -337,11 +328,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -358,11 +348,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -379,11 +368,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -400,11 +388,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -421,11 +408,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -442,11 +428,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -463,11 +448,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -484,11 +468,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -505,11 +488,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -526,11 +508,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -547,11 +528,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -568,11 +548,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -589,11 +568,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -610,11 +588,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -631,11 +608,10 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Try(ScalaReflection.schemaFor[A22]).toOption :: Nil + val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: ScalaReflection.schemaFor[A22] :: Nil).toOption def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), - if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), - Some(name), nullable, udfDeterministic = true) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, + udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) @@ -743,7 +719,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF0[_], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(() => func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -758,7 +734,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -773,7 +749,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -788,7 +764,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -803,7 +779,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -818,7 +794,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -833,7 +809,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -848,7 +824,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -863,7 +839,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -878,7 +854,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -893,7 +869,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -908,7 +884,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -923,7 +899,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -938,7 +914,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -953,7 +929,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -968,7 +944,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -983,7 +959,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -998,7 +974,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -1013,7 +989,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1028,7 +1004,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1043,7 +1019,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1058,7 +1034,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1073,7 +1049,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) + ScalaUDF(func, returnType, e, udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala index 7caff69f23499..48abad9078650 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -34,6 +34,8 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { protected def supportsBatch: Boolean = true + protected def needsUnsafeRowConversion: Boolean = true + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) @@ -134,7 +136,7 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { |if ($batch == null) { | $nextBatchFuncName(); |} - |while ($limitNotReachedCond $batch != null) { + |while ($batch != null) { | int $numRows = $batch.numRows(); | int $localEnd = $numRows - $idx; | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { @@ -157,11 +159,17 @@ private[sql] trait ColumnarBatchScan extends CodegenSupport { ctx.INPUT_ROW = row ctx.currentVars = null + // Always provide `outputVars`, so that the framework can help us build unsafe row if the input + // row is not unsafe row, i.e. `needsUnsafeRowConversion` is true. + val outputVars = output.zipWithIndex.map { case (a, i) => + BoundReference(i, a.dataType, a.nullable).genCode(ctx) + } + val inputRow = if (needsUnsafeRowConversion) null else row s""" - |while ($limitNotReachedCond $input.hasNext()) { + |while ($input.hasNext()) { | InternalRow $row = (InternalRow) $input.next(); | $numOutputRows.add(1); - | ${consume(ctx, null, row).trim} + | ${consume(ctx, outputVars, inputRow).trim} | if (shouldStop()) return; |} """.stripMargin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 9b500c1a040ae..8b23ffed0b632 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -168,11 +168,10 @@ case class FileSourceScanExec( // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsBatch: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) - } + override lazy val supportsBatch: Boolean = relation.fileFormat.supportBatch( + relation.sparkSession, StructType.fromAttributes(output)) - private lazy val needsUnsafeRowConversion: Boolean = { + override lazy val needsUnsafeRowConversion: Boolean = { if (relation.fileFormat.isInstanceOf[ParquetSource]) { SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index dda7cb55f5395..439932b0cc3ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -58,8 +58,7 @@ object SQLExecution { */ def withNewExecutionId[T]( sparkSession: SparkSession, - queryExecution: QueryExecution, - name: Option[String] = None)(body: => T): T = { + queryExecution: QueryExecution)(body: => T): T = { val sc = sparkSession.sparkContext val oldExecutionId = sc.getLocalProperty(EXECUTION_ID_KEY) val executionId = SQLExecution.nextExecutionId @@ -72,35 +71,14 @@ object SQLExecution { val callSite = sc.getCallSite() withSQLConfPropagated(sparkSession) { - var ex: Option[Exception] = None - val startTime = System.nanoTime() + sc.listenerBus.post(SparkListenerSQLExecutionStart( + executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, + SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) try { - sc.listenerBus.post(SparkListenerSQLExecutionStart( - executionId = executionId, - description = callSite.shortForm, - details = callSite.longForm, - physicalPlanDescription = queryExecution.toString, - // `queryExecution.executedPlan` triggers query planning. If it fails, the exception - // will be caught and reported in the `SparkListenerSQLExecutionEnd` - sparkPlanInfo = SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), - time = System.currentTimeMillis())) body - } catch { - case e: Exception => - ex = Some(e) - throw e } finally { - val endTime = System.nanoTime() - val event = SparkListenerSQLExecutionEnd(executionId, System.currentTimeMillis()) - // Currently only `Dataset.withAction` and `DataFrameWriter.runCommand` specify the `name` - // parameter. The `ExecutionListenerManager` only watches SQL executions with name. We - // can specify the execution name in more places in the future, so that - // `QueryExecutionListener` can track more cases. - event.executionName = name - event.duration = endTime - startTime - event.qe = queryExecution - event.executionFailure = ex - sc.listenerBus.post(event) + sc.listenerBus.post(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) } } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index f1470e45f1292..0dc16ba5ce281 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -39,7 +39,7 @@ case class SortExec( global: Boolean, child: SparkPlan, testSpillFrequency: Int = 0) - extends UnaryExecNode with BlockingOperatorWithCodegen { + extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output @@ -124,6 +124,14 @@ case class SortExec( // Name of sorter variable used in codegen. private var sorterVariable: String = _ + // The result rows come from the sort buffer, so this operator doesn't need to copy its result + // even if its child does. + override def needCopyResult: Boolean = false + + // Sort operator always consumes all the input rows before outputting any result, so we don't need + // a stop check before sorting. + override def needStopCheck: Boolean = false + override protected def doProduce(ctx: CodegenContext): String = { val needToSort = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "needToSort", v => s"$v = true;") @@ -164,7 +172,7 @@ case class SortExec( | $needToSort = false; | } | - | while ($limitNotReachedCond $sortedIterator.hasNext()) { + | while ($sortedIterator.hasNext()) { | UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next(); | ${consume(ctx, null, outputRow)} | if (shouldStop()) return; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 364efea52830e..4ed14d3e077f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -282,8 +282,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { throw new ParseException(s"It is not allowed to add database prefix `$database` to " + s"the table name in CACHE TABLE AS SELECT", ctx) } - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - CacheTableCommand(tableIdent, query, ctx.LAZY != null, options) + CacheTableCommand(tableIdent, query, ctx.LAZY != null) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 5f81b6fe743c9..1fc4de9e56015 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -146,10 +146,7 @@ trait CodegenSupport extends SparkPlan { if (outputVars != null) { assert(outputVars.length == output.length) // outputVars will be used to generate the code for UnsafeRow, so we should copy them - outputVars.map(_.copy()) match { - case stream: Stream[ExprCode] => stream.force - case other => other - } + outputVars.map(_.copy()) } else { assert(row != null, "outputVars and row cannot both be null.") ctx.currentVars = null @@ -348,61 +345,6 @@ trait CodegenSupport extends SparkPlan { * don't require shouldStop() in the loop of producing rows. */ def needStopCheck: Boolean = parent.needStopCheck - - /** - * A sequence of checks which evaluate to true if the downstream Limit operators have not received - * enough records and reached the limit. If current node is a data producing node, it can leverage - * this information to stop producing data and complete the data flow earlier. Common data - * producing nodes are leaf nodes like Range and Scan, and blocking nodes like Sort and Aggregate. - * These checks should be put into the loop condition of the data producing loop. - */ - def limitNotReachedChecks: Seq[String] = parent.limitNotReachedChecks - - /** - * A helper method to generate the data producing loop condition according to the - * limit-not-reached checks. - */ - final def limitNotReachedCond: String = { - // InputAdapter is also a leaf node. - val isLeafNode = children.isEmpty || this.isInstanceOf[InputAdapter] - if (!isLeafNode && !this.isInstanceOf[BlockingOperatorWithCodegen]) { - val errMsg = "Only leaf nodes and blocking nodes need to call 'limitNotReachedCond' " + - "in its data producing loop." - if (Utils.isTesting) { - throw new IllegalStateException(errMsg) - } else { - logWarning(s"[BUG] $errMsg Please open a JIRA ticket to report it.") - } - } - if (parent.limitNotReachedChecks.isEmpty) { - "" - } else { - parent.limitNotReachedChecks.mkString("", " && ", " &&") - } - } -} - -/** - * A special kind of operators which support whole stage codegen. Blocking means these operators - * will consume all the inputs first, before producing output. Typical blocking operators are - * sort and aggregate. - */ -trait BlockingOperatorWithCodegen extends CodegenSupport { - - // Blocking operators usually have some kind of buffer to keep the data before producing them, so - // then don't to copy its result even if its child does. - override def needCopyResult: Boolean = false - - // Blocking operators always consume all the input first, so its upstream operators don't need a - // stop check. - override def needStopCheck: Boolean = false - - // Blocking operators need to consume all the inputs before producing any output. This means, - // Limit operator after this blocking operator will never reach its limit during the execution of - // this blocking operator's upstream operators. Here we override this method to return Nil, so - // that upstream operators will not generate useless conditions (which are always evaluated to - // false) for the Limit operators after this blocking operator. - override def limitNotReachedChecks: Seq[String] = Nil } @@ -439,7 +381,7 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp forceInline = true) val row = ctx.freshName("row") s""" - | while ($limitNotReachedCond $input.hasNext()) { + | while ($input.hasNext() && !stopEarly()) { | InternalRow $row = (InternalRow) $input.next(); | ${consume(ctx, null, row).trim} | if (shouldStop()) return; @@ -735,8 +677,6 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def needStopCheck: Boolean = true - override def limitNotReachedChecks: Seq[String] = Nil - override protected def otherCopyArgs: Seq[AnyRef] = Seq(codegenStageId.asInstanceOf[Integer]) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 25d8e7dff3d99..98adba50b2973 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -45,7 +45,7 @@ case class HashAggregateExec( initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode with BlockingOperatorWithCodegen { + extends UnaryExecNode with CodegenSupport { private[this] val aggregateBufferAttributes = { aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes) @@ -151,6 +151,14 @@ case class HashAggregateExec( child.asInstanceOf[CodegenSupport].inputRDDs() } + // The result rows come from the aggregate buffer, or a single row(no grouping keys), so this + // operator doesn't need to copy its result even if its child does. + override def needCopyResult: Boolean = false + + // Aggregate operator always consumes all the input rows before outputting any result, so we + // don't need a stop check before aggregating. + override def needStopCheck: Boolean = false + protected override def doProduce(ctx: CodegenContext): String = { if (groupingExpressions.isEmpty) { doProduceWithoutKeys(ctx) @@ -697,16 +705,13 @@ case class HashAggregateExec( def outputFromRegularHashMap: String = { s""" - |while ($limitNotReachedCond $iterTerm.next()) { + |while ($iterTerm.next()) { | UnsafeRow $keyTerm = (UnsafeRow) $iterTerm.getKey(); | UnsafeRow $bufferTerm = (UnsafeRow) $iterTerm.getValue(); | $outputFunc($keyTerm, $bufferTerm); + | | if (shouldStop()) return; |} - |$iterTerm.close(); - |if ($sorterTerm == null) { - | $hashMapTerm.free(); - |} """.stripMargin } @@ -723,6 +728,11 @@ case class HashAggregateExec( // output the result $outputFromFastHashMap $outputFromRegularHashMap + + $iterTerm.close(); + if ($sorterTerm == null) { + $hashMapTerm.free(); + } """ } @@ -844,47 +854,33 @@ case class HashAggregateExec( val updateRowInHashMap: String = { if (isFastHashMapEnabled) { - if (isVectorizedHashMapEnabled) { - ctx.INPUT_ROW = fastRowBuffer - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) - val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) - val effectiveCodes = subExprs.codes.mkString("\n") - val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { - boundUpdateExpr.map(_.genCode(ctx)) - } - val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) => - val dt = updateExpr(i).dataType - CodeGenerator.updateColumn( - fastRowBuffer, dt, i, ev, updateExpr(i).nullable, isVectorized = true) - } - - // If vectorized fast hash map is on, we first generate code to update row - // in vectorized fast hash map, if the previous loop up hit vectorized fast hash map. - // Otherwise, update row in regular hash map. - s""" - |if ($fastRowBuffer != null) { - | // common sub-expressions - | $effectiveCodes - | // evaluate aggregate function - | ${evaluateVariables(fastRowEvals)} - | // update fast row - | ${updateFastRow.mkString("\n").trim} - |} else { - | $updateRowInRegularHashMap - |} - """.stripMargin - } else { - // If row-based hash map is on and the previous loop up hit fast hash map, - // we reuse regular hash buffer to update row of fast hash map. - // Otherwise, update row in regular hash map. - s""" - |// Updates the proper row buffer - |if ($fastRowBuffer != null) { - | $unsafeRowBuffer = $fastRowBuffer; - |} - |$updateRowInRegularHashMap - """.stripMargin + ctx.INPUT_ROW = fastRowBuffer + val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } + val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) => + val dt = updateExpr(i).dataType + CodeGenerator.updateColumn( + fastRowBuffer, dt, i, ev, updateExpr(i).nullable, isVectorizedHashMapEnabled) } + + // If fast hash map is on, we first generate code to update row in fast hash map, if the + // previous loop up hit fast hash map. Otherwise, update row in regular hash map. + s""" + |if ($fastRowBuffer != null) { + | // common sub-expressions + | $effectiveCodes + | // evaluate aggregate function + | ${evaluateVariables(fastRowEvals)} + | // update fast row + | ${updateFastRow.mkString("\n").trim} + |} else { + | $updateRowInRegularHashMap + |} + """.stripMargin } else { updateRowInRegularHashMap } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 39200ec00e152..6d44890704f49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -38,14 +38,18 @@ object TypedAggregateExpression { val bufferSerializer = bufferEncoder.namedExpressions val outputEncoder = encoderFor[OUT] - val outputType = outputEncoder.objSerializer.dataType + val outputType = if (outputEncoder.flat) { + outputEncoder.schema.head.dataType + } else { + outputEncoder.schema + } // Checks if the buffer object is simple, i.e. the buffer encoder is flat and the serializer // expression is an alias of `BoundReference`, which means the buffer object doesn't need // serialization. val isSimpleBuffer = { bufferSerializer.head match { - case Alias(_: BoundReference, _) if !bufferEncoder.isSerializedAsStruct => true + case Alias(_: BoundReference, _) if bufferEncoder.flat => true case _ => false } } @@ -67,7 +71,7 @@ object TypedAggregateExpression { outputEncoder.serializer, outputEncoder.deserializer.dataType, outputType, - outputEncoder.objSerializer.nullable) + !outputEncoder.flat || outputEncoder.schema.head.nullable) } else { ComplexTypedAggregateExpression( aggregator.asInstanceOf[Aggregator[Any, Any, Any]], @@ -78,7 +82,7 @@ object TypedAggregateExpression { bufferEncoder.resolveAndBind().deserializer, outputEncoder.serializer, outputType, - outputEncoder.objSerializer.nullable) + !outputEncoder.flat || outputEncoder.schema.head.nullable) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 09effe087e195..222a1b8bc7301 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -378,7 +378,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val numOutput = metricTerm(ctx, "numOutputRows") val initTerm = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initRange") - val nextIndex = ctx.addMutableState(CodeGenerator.JAVA_LONG, "nextIndex") + val number = ctx.addMutableState(CodeGenerator.JAVA_LONG, "number") val value = ctx.freshName("value") val ev = ExprCode.forNonNullValue(JavaCode.variable(value, LongType)) @@ -397,7 +397,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) // within a batch, while the code in the outer loop is setting batch parameters and updating // the metrics. - // Once nextIndex == batchEnd, it's time to progress to the next batch. + // Once number == batchEnd, it's time to progress to the next batch. val batchEnd = ctx.addMutableState(CodeGenerator.JAVA_LONG, "batchEnd") // How many values should still be generated by this range operator. @@ -421,13 +421,13 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | | $BigInt st = index.multiply(numElement).divide(numSlice).multiply(step).add(start); | if (st.compareTo($BigInt.valueOf(Long.MAX_VALUE)) > 0) { - | $nextIndex = Long.MAX_VALUE; + | $number = Long.MAX_VALUE; | } else if (st.compareTo($BigInt.valueOf(Long.MIN_VALUE)) < 0) { - | $nextIndex = Long.MIN_VALUE; + | $number = Long.MIN_VALUE; | } else { - | $nextIndex = st.longValue(); + | $number = st.longValue(); | } - | $batchEnd = $nextIndex; + | $batchEnd = $number; | | $BigInt end = index.add($BigInt.ONE).multiply(numElement).divide(numSlice) | .multiply(step).add(start); @@ -440,7 +440,7 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | } | | $BigInt startToEnd = $BigInt.valueOf(partitionEnd).subtract( - | $BigInt.valueOf($nextIndex)); + | $BigInt.valueOf($number)); | $numElementsTodo = startToEnd.divide(step).longValue(); | if ($numElementsTodo < 0) { | $numElementsTodo = 0; @@ -452,49 +452,12 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val localIdx = ctx.freshName("localIdx") val localEnd = ctx.freshName("localEnd") - val stopCheck = if (parent.needStopCheck) { - s""" - |if (shouldStop()) { - | $nextIndex = $value + ${step}L; - | $numOutput.add($localIdx + 1); - | $inputMetrics.incRecordsRead($localIdx + 1); - | return; - |} - """.stripMargin + val range = ctx.freshName("range") + val shouldStop = if (parent.needStopCheck) { + s"if (shouldStop()) { $number = $value + ${step}L; return; }" } else { "// shouldStop check is eliminated" } - val loopCondition = if (limitNotReachedChecks.isEmpty) { - "true" - } else { - limitNotReachedChecks.mkString(" && ") - } - - // An overview of the Range processing. - // - // For each partition, the Range task needs to produce records from partition start(inclusive) - // to end(exclusive). For better performance, we separate the partition range into batches, and - // use 2 loops to produce data. The outer while loop is used to iterate batches, and the inner - // for loop is used to iterate records inside a batch. - // - // `nextIndex` tracks the index of the next record that is going to be consumed, initialized - // with partition start. `batchEnd` tracks the end index of the current batch, initialized - // with `nextIndex`. In the outer loop, we first check if `nextIndex == batchEnd`. If it's true, - // it means the current batch is fully consumed, and we will update `batchEnd` to process the - // next batch. If `batchEnd` reaches partition end, exit the outer loop. Finally we enter the - // inner loop. Note that, when we enter inner loop, `nextIndex` must be different from - // `batchEnd`, otherwise we already exit the outer loop. - // - // The inner loop iterates from 0 to `localEnd`, which is calculated by - // `(batchEnd - nextIndex) / step`. Since `batchEnd` is increased by `nextBatchTodo * step` in - // the outer loop, and initialized with `nextIndex`, so `batchEnd - nextIndex` is always - // divisible by `step`. The `nextIndex` is increased by `step` during each iteration, and ends - // up being equal to `batchEnd` when the inner loop finishes. - // - // The inner loop can be interrupted, if the query has produced at least one result row, so that - // we don't buffer too many result rows and waste memory. It's ok to interrupt the inner loop, - // because `nextIndex` will be updated before interrupting. - s""" | // initialize Range | if (!$initTerm) { @@ -502,30 +465,33 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | $initRangeFuncName(partitionIndex); | } | - | while ($loopCondition) { - | if ($nextIndex == $batchEnd) { - | long $nextBatchTodo; - | if ($numElementsTodo > ${batchSize}L) { - | $nextBatchTodo = ${batchSize}L; - | $numElementsTodo -= ${batchSize}L; - | } else { - | $nextBatchTodo = $numElementsTodo; - | $numElementsTodo = 0; - | if ($nextBatchTodo == 0) break; + | while (true) { + | long $range = $batchEnd - $number; + | if ($range != 0L) { + | int $localEnd = (int)($range / ${step}L); + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | long $value = ((long)$localIdx * ${step}L) + $number; + | ${consume(ctx, Seq(ev))} + | $shouldStop | } - | $batchEnd += $nextBatchTodo * ${step}L; + | $number = $batchEnd; | } | - | int $localEnd = (int)(($batchEnd - $nextIndex) / ${step}L); - | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { - | long $value = ((long)$localIdx * ${step}L) + $nextIndex; - | ${consume(ctx, Seq(ev))} - | $stopCheck - | } - | $nextIndex = $batchEnd; - | $numOutput.add($localEnd); - | $inputMetrics.incRecordsRead($localEnd); | $taskContext.killTaskIfInterrupted(); + | + | long $nextBatchTodo; + | if ($numElementsTodo > ${batchSize}L) { + | $nextBatchTodo = ${batchSize}L; + | $numElementsTodo -= ${batchSize}L; + | } else { + | $nextBatchTodo = $numElementsTodo; + | $numElementsTodo = 0; + | if ($nextBatchTodo == 0) break; + | } + | $numOutput.add($nextBatchTodo); + | $inputMetrics.incRecordsRead($nextBatchTodo); + | + | $batchEnd += $nextBatchTodo * ${step}L; | } """.stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 3b6588587c35a..1a8fbaca53f59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -149,30 +149,30 @@ object InMemoryRelation { tableName: Option[String], logicalPlan: LogicalPlan): InMemoryRelation = { val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName)() - new InMemoryRelation(child.output, cacheBuilder, logicalPlan.outputOrdering)( - statsOfPlanToCache = logicalPlan.stats) + new InMemoryRelation(child.output, cacheBuilder)( + statsOfPlanToCache = logicalPlan.stats, outputOrdering = logicalPlan.outputOrdering) } def apply(cacheBuilder: CachedRDDBuilder, logicalPlan: LogicalPlan): InMemoryRelation = { - new InMemoryRelation(cacheBuilder.cachedPlan.output, cacheBuilder, logicalPlan.outputOrdering)( - statsOfPlanToCache = logicalPlan.stats) + new InMemoryRelation(cacheBuilder.cachedPlan.output, cacheBuilder)( + statsOfPlanToCache = logicalPlan.stats, outputOrdering = logicalPlan.outputOrdering) } } case class InMemoryRelation( output: Seq[Attribute], - @transient cacheBuilder: CachedRDDBuilder, - override val outputOrdering: Seq[SortOrder])( - statsOfPlanToCache: Statistics) + @transient cacheBuilder: CachedRDDBuilder)( + statsOfPlanToCache: Statistics, + override val outputOrdering: Seq[SortOrder]) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[SparkPlan] = Seq(cachedPlan) override def doCanonicalize(): logical.LogicalPlan = copy(output = output.map(QueryPlan.normalizeExprId(_, cachedPlan.output)), - cacheBuilder, - outputOrdering)( - statsOfPlanToCache) + cacheBuilder)( + statsOfPlanToCache, + outputOrdering) override def producedAttributes: AttributeSet = outputSet @@ -195,15 +195,15 @@ case class InMemoryRelation( } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { - InMemoryRelation(newOutput, cacheBuilder, outputOrdering)(statsOfPlanToCache) + InMemoryRelation(newOutput, cacheBuilder)(statsOfPlanToCache, outputOrdering) } override def newInstance(): this.type = { new InMemoryRelation( output.map(_.newInstance()), - cacheBuilder, - outputOrdering)( - statsOfPlanToCache).asInstanceOf[this.type] + cacheBuilder)( + statsOfPlanToCache, + outputOrdering).asInstanceOf[this.type] } override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 8f8d8010e26bd..196d057c2de1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -69,6 +69,8 @@ case class InMemoryTableScanExec( // TODO: revisit this. Shall we always turn off whole stage codegen if the output data are rows? override def supportCodegen: Boolean = supportsBatch + override protected def needsUnsafeRowConversion: Boolean = false + private val columnIndices = attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 04a9442424354..93447a52097ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -210,13 +210,13 @@ case class AnalyzeColumnCommand( def struct(exprs: Expression*): CreateNamedStruct = CreateStruct(exprs.map { expr => expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() } }) - val one = Literal(1L, LongType) + val one = Literal(1, LongType) // the approximate ndv (num distinct value) should never be larger than the number of rows val numNonNulls = if (col.nullable) Count(col) else Count(one) val ndv = Least(Seq(HyperLogLogPlusPlus(col, conf.ndvMaxError), numNonNulls)) val numNulls = Subtract(Count(one), numNonNulls) - val defaultSize = Literal(col.dataType.defaultSize.toLong, LongType) + val defaultSize = Literal(col.dataType.defaultSize, LongType) val nullArray = Literal(null, ArrayType(LongType)) def fixedLenTypeStruct: CreateNamedStruct = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 728604ac5234c..6b00426d2fa91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -17,21 +17,16 @@ package org.apache.spark.sql.execution.command -import java.util.Locale - import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.storage.StorageLevel case class CacheTableCommand( tableIdent: TableIdentifier, plan: Option[LogicalPlan], - isLazy: Boolean, - options: Map[String, String]) extends RunnableCommand { + isLazy: Boolean) extends RunnableCommand { require(plan.isEmpty || tableIdent.database.isEmpty, "Database name is not allowed in CACHE TABLE AS SELECT") @@ -41,21 +36,7 @@ case class CacheTableCommand( plan.foreach { logicalPlan => Dataset.ofRows(sparkSession, logicalPlan).createTempView(tableIdent.quotedString) } - - val storageLevelKey = "storagelevel" - val storageLevelValue = - CaseInsensitiveMap(options).get(storageLevelKey).map(_.toUpperCase(Locale.ROOT)) - val withoutStorageLevel = options.filterKeys(_.toLowerCase(Locale.ROOT) != storageLevelKey) - if (withoutStorageLevel.nonEmpty) { - logWarning(s"Invalid options: ${withoutStorageLevel.mkString(", ")}") - } - - if (storageLevelValue.nonEmpty) { - sparkSession.catalog.cacheTable( - tableIdent.quotedString, StorageLevel.fromString(storageLevelValue.get)) - } else { - sparkSession.catalog.cacheTable(tableIdent.quotedString) - } + sparkSession.catalog.cacheTable(tableIdent.quotedString) if (!isLazy) { // Performs eager caching diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 8fee02a8f6c82..2e859cf1ef253 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -38,7 +38,7 @@ case class AddJarCommand(path: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.resourceLoader.addJar(path) - Seq.empty[Row] + Seq(Row(0)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 823dc0d5ed387..2eca1c40a5b3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -306,8 +306,7 @@ case class LoadDataCommand( val loadPath = { if (isLocal) { val localFS = FileContext.getLocalFSFileContext() - LoadDataCommand.makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), - new Path(path)) + makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), new Path(path)) } else { val loadPath = new Path(path) // Follow Hive's behavior: @@ -324,7 +323,7 @@ case class LoadDataCommand( // by considering the wild card scenario in mind.as per old logic query param is // been considered while creating URI instance and if path contains wild card char '?' // the remaining charecters after '?' will be removed while forming URI instance - LoadDataCommand.makeQualified(defaultFS, uriPath, loadPath) + makeQualified(defaultFS, uriPath, loadPath) } } val fs = loadPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) @@ -364,9 +363,7 @@ case class LoadDataCommand( CommandUtils.updateTableStats(sparkSession, targetTable) Seq.empty[Row] } -} -object LoadDataCommand { /** * Returns a qualified path object. Method ported from org.apache.hadoop.fs.Path class. * @@ -375,9 +372,8 @@ object LoadDataCommand { * @param path Path instance based on the path string specified by the user. * @return qualified path object */ - private[sql] def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { - val newPath = new Path(workingDir, path) - val pathUri = if (path.isAbsolute()) path.toUri() else newPath.toUri() + private def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { + val pathUri = if (path.isAbsolute()) path.toUri() else new Path(workingDir, path).toUri() if (pathUri.getScheme == null || pathUri.getAuthority == null && defaultUri.getAuthority != null) { val scheme = if (pathUri.getScheme == null) defaultUri.getScheme else pathUri.getScheme @@ -387,14 +383,14 @@ object LoadDataCommand { pathUri.getAuthority } try { - val newUri = new URI(scheme, authority, pathUri.getPath, null, pathUri.getFragment) + val newUri = new URI(scheme, authority, pathUri.getPath, pathUri.getFragment) new Path(newUri) } catch { case e: URISyntaxException => throw new IllegalArgumentException(e) } } else { - newPath + path } } } @@ -958,11 +954,9 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman builder ++= metadata.viewText.mkString(" AS\n", "", "\n") } else { showHiveTableHeader(metadata, builder) - showTableComment(metadata, builder) showHiveTableNonDataColumns(metadata, builder) showHiveTableStorageInfo(metadata, builder) - showTableLocation(metadata, builder) - showTableProperties(metadata, builder) + showHiveTableProperties(metadata, builder) } builder.toString() @@ -976,8 +970,14 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman if (columns.nonEmpty) { builder ++= columns.mkString("(", ", ", ")\n") } + + metadata + .comment + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) } + private def showHiveTableNonDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.partitionColumnNames.nonEmpty) { val partCols = metadata.partitionSchema.map(_.toDDL) @@ -1020,24 +1020,15 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman builder ++= s" OUTPUTFORMAT '${escapeSingleQuotedString(format)}'\n" } } - } - private def showTableLocation(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.tableType == EXTERNAL) { - metadata.storage.locationUri.foreach { location => - builder ++= s"LOCATION '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'\n" + storage.locationUri.foreach { uri => + builder ++= s"LOCATION '$uri'\n" } } } - private def showTableComment(metadata: CatalogTable, builder: StringBuilder): Unit = { - metadata - .comment - .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") - .foreach(builder.append) - } - - private def showTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + private def showHiveTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.properties.nonEmpty) { val props = metadata.properties.map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" @@ -1054,9 +1045,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman showDataSourceTableDataColumns(metadata, builder) showDataSourceTableOptions(metadata, builder) showDataSourceTableNonDataColumns(metadata, builder) - showTableComment(metadata, builder) - showTableLocation(metadata, builder) - showTableProperties(metadata, builder) builder.toString() } @@ -1072,6 +1060,14 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman val dataSourceOptions = metadata.storage.properties.map { case (key, value) => s"${quoteIdentifier(key)} '${escapeSingleQuotedString(value)}'" + } ++ metadata.storage.locationUri.flatMap { location => + if (metadata.tableType == MANAGED) { + // If it's a managed table, omit PATH option. Spark SQL always creates external table + // when the table creation DDL contains the PATH option. + None + } else { + Some(s"path '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'") + } } if (dataSourceOptions.nonEmpty) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala similarity index 93% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala index 76745b11c84c9..90e81661bae7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala @@ -15,11 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.util +package org.apache.spark.sql.execution.datasources import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String @@ -73,8 +75,7 @@ class FailureSafeParser[IN]( Iterator.empty case FailFastMode => throw new SparkException("Malformed records are detected in record parsing. " + - s"Parse Mode: ${FailFastMode.name}. To process malformed records as null " + - "result, try setting the option 'mode' as 'PERMISSIVE'.", e) + s"Parse Mode: ${FailFastMode.name}.", e.cause) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 10733810b6416..6499328e89ce7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -179,8 +179,7 @@ class DynamicPartitionDataWriter( val partitionName = ScalaUDF( ExternalCatalogUtils.getPartitionPathString _, StringType, - Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId))), - Seq(true, true)) + Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index ffea33c08ef94..345c9d82ca0e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -85,7 +85,7 @@ class FileScanRDD( // If we do a coalesce, however, we are likely to compute multiple partitions in the same // task and in the same thread, in which case we need to avoid override values written by // previous partitions (SPARK-13071). - private def incTaskInputMetricsBytesRead(): Unit = { + private def updateBytesRead(): Unit = { inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } @@ -104,18 +104,14 @@ class FileScanRDD( val nextElement = currentIterator.next() // TODO: we should have a better separation of row based and batch based scan, so that we // don't need to run this `if` for every record. - val preNumRecordsRead = inputMetrics.recordsRead if (nextElement.isInstanceOf[ColumnarBatch]) { - incTaskInputMetricsBytesRead() inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) } else { - // too costly to update every record - if (inputMetrics.recordsRead % - SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { - incTaskInputMetricsBytesRead() - } inputMetrics.incRecordsRead(1) } + if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + updateBytesRead() + } nextElement } @@ -202,7 +198,7 @@ class FileScanRDD( } override def close(): Unit = { - incTaskInputMetricsBytesRead() + updateBytesRead() InputFileBlockHolder.unset() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index fe418e610da8f..dc5c2ff927e4a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -315,14 +315,7 @@ object InMemoryFileIndex extends Logging { // which is very slow on some file system (RawLocalFileSystem, which is launch a // subprocess and parse the stdout). try { - val locations = fs.getFileBlockLocations(f, 0, f.getLen).map { loc => - // Store BlockLocation objects to consume less memory - if (loc.getClass == classOf[BlockLocation]) { - loc - } else { - new BlockLocation(loc.getNames, loc.getHosts, loc.getOffset, loc.getLength) - } - } + val locations = fs.getFileBlockLocations(f, 0, f.getLen) val lfs = new LocatedFileStatus(f.getLen, f.isDirectory, f.getReplication, f.getBlockSize, f.getModificationTime, 0, null, null, null, null, f.getPath, locations) if (f.isSymlink) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index 4808e8ef042d1..b93f418bcb5be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -34,7 +34,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVInferSchema, CSVOptions, UnivocityParser} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.types.StructType @@ -52,8 +51,11 @@ abstract class CSVDataSource extends Serializable { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - headerChecker: CSVHeaderChecker, - requiredSchema: StructType): Iterator[InternalRow] + requiredSchema: StructType, + // Actual schema of data in the csv file + dataSchema: StructType, + caseSensitive: Boolean, + columnPruning: Boolean): Iterator[InternalRow] /** * Infers the schema from `inputPaths` files. @@ -73,6 +75,48 @@ abstract class CSVDataSource extends Serializable { sparkSession: SparkSession, inputPaths: Seq[FileStatus], parsedOptions: CSVOptions): StructType + + /** + * Generates a header from the given row which is null-safe and duplicate-safe. + */ + protected def makeSafeHeader( + row: Array[String], + caseSensitive: Boolean, + options: CSVOptions): Array[String] = { + if (options.headerFlag) { + val duplicates = { + val headerNames = row.filter(_ != null) + // scalastyle:off caselocale + .map(name => if (caseSensitive) name else name.toLowerCase) + // scalastyle:on caselocale + headerNames.diff(headerNames.distinct).distinct + } + + row.zipWithIndex.map { case (value, index) => + if (value == null || value.isEmpty || value == options.nullValue) { + // When there are empty strings or the values set in `nullValue`, put the + // index as the suffix. + s"_c$index" + // scalastyle:off caselocale + } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { + // scalastyle:on caselocale + // When there are case-insensitive duplicates, put the index as the suffix. + s"$value$index" + } else if (duplicates.contains(value)) { + // When there are duplicates, put the index as the suffix. + s"$value$index" + } else { + value + } + } + } else { + row.zipWithIndex.map { case (_, index) => + // Uses default column names, "_c#" where # is its position of fields + // when header option is disabled. + s"_c$index" + } + } + } } object CSVDataSource extends Logging { @@ -83,6 +127,67 @@ object CSVDataSource extends Logging { TextInputCSVDataSource } } + + /** + * Checks that column names in a CSV header and field names in the schema are the same + * by taking into account case sensitivity. + * + * @param schema - provided (or inferred) schema to which CSV must conform. + * @param columnNames - names of CSV columns that must be checked against to the schema. + * @param fileName - name of CSV file that are currently checked. It is used in error messages. + * @param enforceSchema - if it is `true`, column names are ignored otherwise the CSV column + * names are checked for conformance to the schema. In the case if + * the column name don't conform to the schema, an exception is thrown. + * @param caseSensitive - if it is set to `false`, comparison of column names and schema field + * names is not case sensitive. + */ + def checkHeaderColumnNames( + schema: StructType, + columnNames: Array[String], + fileName: String, + enforceSchema: Boolean, + caseSensitive: Boolean): Unit = { + if (columnNames != null) { + val fieldNames = schema.map(_.name).toIndexedSeq + val (headerLen, schemaSize) = (columnNames.size, fieldNames.length) + var errorMessage: Option[String] = None + + if (headerLen == schemaSize) { + var i = 0 + while (errorMessage.isEmpty && i < headerLen) { + var (nameInSchema, nameInHeader) = (fieldNames(i), columnNames(i)) + if (!caseSensitive) { + // scalastyle:off caselocale + nameInSchema = nameInSchema.toLowerCase + nameInHeader = nameInHeader.toLowerCase + // scalastyle:on caselocale + } + if (nameInHeader != nameInSchema) { + errorMessage = Some( + s"""|CSV header does not conform to the schema. + | Header: ${columnNames.mkString(", ")} + | Schema: ${fieldNames.mkString(", ")} + |Expected: ${fieldNames(i)} but found: ${columnNames(i)} + |CSV file: $fileName""".stripMargin) + } + i += 1 + } + } else { + errorMessage = Some( + s"""|Number of column in CSV header is not equal to number of fields in the schema: + | Header length: $headerLen, schema size: $schemaSize + |CSV file: $fileName""".stripMargin) + } + + errorMessage.foreach { msg => + if (enforceSchema) { + logWarning(msg) + } else { + throw new IllegalArgumentException(msg) + } + } + } + } } object TextInputCSVDataSource extends CSVDataSource { @@ -92,8 +197,10 @@ object TextInputCSVDataSource extends CSVDataSource { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - headerChecker: CSVHeaderChecker, - requiredSchema: StructType): Iterator[InternalRow] = { + requiredSchema: StructType, + dataSchema: StructType, + caseSensitive: Boolean, + columnPruning: Boolean): Iterator[InternalRow] = { val lines = { val linesReader = new HadoopFileLinesReader(file, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => linesReader.close())) @@ -102,7 +209,25 @@ object TextInputCSVDataSource extends CSVDataSource { } } - UnivocityParser.parseIterator(lines, parser, headerChecker, requiredSchema) + val hasHeader = parser.options.headerFlag && file.start == 0 + if (hasHeader) { + // Checking that column names in the header are matched to field names of the schema. + // The header will be removed from lines. + // Note: if there are only comments in the first block, the header would probably + // be not extracted. + CSVUtils.extractHeader(lines, parser.options).foreach { header => + val schema = if (columnPruning) requiredSchema else dataSchema + val columnNames = parser.tokenizer.parseLine(header) + CSVDataSource.checkHeaderColumnNames( + schema, + columnNames, + file.filePath, + parser.options.enforceSchema, + caseSensitive) + } + } + + UnivocityParser.parseIterator(lines, parser, requiredSchema) } override def infer( @@ -126,7 +251,7 @@ object TextInputCSVDataSource extends CSVDataSource { maybeFirstLine.map(csvParser.parseLine(_)) match { case Some(firstRow) if firstRow != null => val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - val header = CSVUtils.makeSafeHeader(firstRow, caseSensitive, parsedOptions) + val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) val sampled: Dataset[String] = CSVUtils.sample(csv, parsedOptions) val tokenRDD = sampled.rdd.mapPartitions { iter => val filteredLines = CSVUtils.filterCommentAndEmpty(iter, parsedOptions) @@ -173,13 +298,26 @@ object MultiLineCSVDataSource extends CSVDataSource { conf: Configuration, file: PartitionedFile, parser: UnivocityParser, - headerChecker: CSVHeaderChecker, - requiredSchema: StructType): Iterator[InternalRow] = { + requiredSchema: StructType, + dataSchema: StructType, + caseSensitive: Boolean, + columnPruning: Boolean): Iterator[InternalRow] = { + def checkHeader(header: Array[String]): Unit = { + val schema = if (columnPruning) requiredSchema else dataSchema + CSVDataSource.checkHeaderColumnNames( + schema, + header, + file.filePath, + parser.options.enforceSchema, + caseSensitive) + } + UnivocityParser.parseStream( CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath))), + parser.options.headerFlag, parser, - headerChecker, - requiredSchema) + requiredSchema, + checkHeader) } override def infer( @@ -196,7 +334,7 @@ object MultiLineCSVDataSource extends CSVDataSource { }.take(1).headOption match { case Some(firstRow) => val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - val header = CSVUtils.makeSafeHeader(firstRow, caseSensitive, parsedOptions) + val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) val tokenRDD = csv.flatMap { lines => UnivocityParser.tokenizeStream( CodecStreams.createInputStreamWithCloseResource( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 954a5a9cdecbb..9aad0bd55e736 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapreduce._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ @@ -131,6 +130,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { "df.filter($\"_corrupt_record\".isNotNull).count()." ) } + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis val columnPruning = sparkSession.sessionState.conf.csvColumnPruning (file: PartitionedFile) => { @@ -139,16 +139,14 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), parsedOptions) - val schema = if (columnPruning) requiredSchema else dataSchema - val isStartOfFile = file.start == 0 - val headerChecker = new CSVHeaderChecker( - schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) CSVDataSource(parsedOptions).readFile( conf, file, parser, - headerChecker, - requiredSchema) + requiredSchema, + dataSchema, + caseSensitive, + columnPruning) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala similarity index 80% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 799e9994451b2..a585cbed2551b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.execution.datasources.csv import java.math.BigDecimal -import scala.util.control.Exception.allCatch +import scala.util.control.Exception._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ -object CSVInferSchema { +private[csv] object CSVInferSchema { /** * Similar to the JSON schema inference @@ -43,7 +43,13 @@ object CSVInferSchema { val rootTypes: Array[DataType] = tokenRDD.aggregate(startType)(inferRowType(options), mergeRowTypes) - toStructFields(rootTypes, header, options) + header.zip(rootTypes).map { case (thisHeader, rootType) => + val dType = rootType match { + case _: NullType => StringType + case other => other + } + StructField(thisHeader, dType, nullable = true) + } } else { // By default fields are assumed to be StringType header.map(fieldName => StructField(fieldName, StringType, nullable = true)) @@ -52,20 +58,7 @@ object CSVInferSchema { StructType(fields) } - def toStructFields( - fieldTypes: Array[DataType], - header: Array[String], - options: CSVOptions): Array[StructField] = { - header.zip(fieldTypes).map { case (thisHeader, rootType) => - val dType = rootType match { - case _: NullType => StringType - case other => other - } - StructField(thisHeader, dType, nullable = true) - } - } - - def inferRowType(options: CSVOptions) + private def inferRowType(options: CSVOptions) (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { var i = 0 while (i < math.min(rowSoFar.length, next.length)) { // May have columns on right missing. @@ -77,7 +70,7 @@ object CSVInferSchema { def mergeRowTypes(first: Array[DataType], second: Array[DataType]): Array[DataType] = { first.zipAll(second, NullType, NullType).map { case (a, b) => - compatibleType(a, b).getOrElse(NullType) + findTightestCommonType(a, b).getOrElse(NullType) } } @@ -95,7 +88,7 @@ object CSVInferSchema { case LongType => tryParseLong(field, options) case _: DecimalType => // DecimalTypes have different precisions and scales, so we try to find the common type. - compatibleType(typeSoFar, tryParseDecimal(field, options)).getOrElse(StringType) + findTightestCommonType(typeSoFar, tryParseDecimal(field, options)).getOrElse(StringType) case DoubleType => tryParseDouble(field, options) case TimestampType => tryParseTimestamp(field, options) case BooleanType => tryParseBoolean(field, options) @@ -179,27 +172,35 @@ object CSVInferSchema { StringType } - /** - * Returns the common data type given two input data types so that the return type - * is compatible with both input data types. - */ - private def compatibleType(t1: DataType, t2: DataType): Option[DataType] = { - TypeCoercion.findTightestCommonType(t1, t2).orElse(findCompatibleTypeForCSV(t1, t2)) - } + private val numericPrecedence: IndexedSeq[DataType] = TypeCoercion.numericPrecedence /** - * The following pattern matching represents additional type promotion rules that - * are CSV specific. + * Copied from internal Spark api + * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]] */ - private val findCompatibleTypeForCSV: (DataType, DataType) => Option[DataType] = { + val findTightestCommonType: (DataType, DataType) => Option[DataType] = { + case (t1, t2) if t1 == t2 => Some(t1) + case (NullType, t1) => Some(t1) + case (t1, NullType) => Some(t1) case (StringType, t2) => Some(StringType) case (t1, StringType) => Some(StringType) + // Promote numeric types to the highest of the two and all numeric types to unlimited decimal + case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => + val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) + Some(numericPrecedence(index)) + + // These two cases below deal with when `DecimalType` is larger than `IntegralType`. + case (t1: IntegralType, t2: DecimalType) if t2.isWiderThan(t1) => + Some(t2) + case (t1: DecimalType, t2: IntegralType) if t1.isWiderThan(t2) => + Some(t1) + // These two cases below deal with when `IntegralType` is larger than `DecimalType`. case (t1: IntegralType, t2: DecimalType) => - compatibleType(DecimalType.forType(t1), t2) + findTightestCommonType(DecimalType.forType(t1), t2) case (t1: DecimalType, t2: IntegralType) => - compatibleType(t1, DecimalType.forType(t2)) + findTightestCommonType(t1, DecimalType.forType(t2)) // Double support larger range than fixed decimal, DecimalType.Maximum should be enough // in most case, also have better precision. @@ -215,6 +216,7 @@ object CSVInferSchema { } else { Some(DecimalType(range + scale, scale)) } + case _ => None } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala similarity index 98% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index cdaaa172e8367..222954a32f1c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.StandardCharsets import java.util.{Locale, TimeZone} @@ -83,7 +83,7 @@ class CSVOptions( } } - val delimiter = CSVExprUtils.toChar( + val delimiter = CSVUtils.toChar( parameters.getOrElse("sep", parameters.getOrElse("delimiter", ","))) val parseMode: ParseMode = parameters.get("mode").map(ParseMode.fromString).getOrElse(PermissiveMode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala index 21fabac472f4b..7ce65fa89b02d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.execution.datasources.csv import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset -import org.apache.spark.sql.catalyst.csv.CSVExprUtils -import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ object CSVUtils { /** @@ -41,6 +40,16 @@ object CSVUtils { } } + /** + * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). + * This is currently being used in CSV reading path and CSV schema inference. + */ + def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = { + iter.filter { line => + line.trim.nonEmpty && !line.startsWith(options.comment.toString) + } + } + /** * Skip the given first line so that only data can remain in a dataset. * This is similar with `dropHeaderLine` below and currently being used in CSV schema inference. @@ -59,45 +68,52 @@ object CSVUtils { } } - /** - * Generates a header from the given row which is null-safe and duplicate-safe. - */ - def makeSafeHeader( - row: Array[String], - caseSensitive: Boolean, - options: CSVOptions): Array[String] = { - if (options.headerFlag) { - val duplicates = { - val headerNames = row.filter(_ != null) - // scalastyle:off caselocale - .map(name => if (caseSensitive) name else name.toLowerCase) - // scalastyle:on caselocale - headerNames.diff(headerNames.distinct).distinct + def skipComments(iter: Iterator[String], options: CSVOptions): Iterator[String] = { + if (options.isCommentSet) { + val commentPrefix = options.comment.toString + iter.dropWhile { line => + line.trim.isEmpty || line.trim.startsWith(commentPrefix) } + } else { + iter.dropWhile(_.trim.isEmpty) + } + } - row.zipWithIndex.map { case (value, index) => - if (value == null || value.isEmpty || value == options.nullValue) { - // When there are empty strings or the values set in `nullValue`, put the - // index as the suffix. - s"_c$index" - // scalastyle:off caselocale - } else if (!caseSensitive && duplicates.contains(value.toLowerCase)) { - // scalastyle:on caselocale - // When there are case-insensitive duplicates, put the index as the suffix. - s"$value$index" - } else if (duplicates.contains(value)) { - // When there are duplicates, put the index as the suffix. - s"$value$index" - } else { - value - } - } + /** + * Extracts header and moves iterator forward so that only data remains in it + */ + def extractHeader(iter: Iterator[String], options: CSVOptions): Option[String] = { + val nonEmptyLines = skipComments(iter, options) + if (nonEmptyLines.hasNext) { + Some(nonEmptyLines.next()) } else { - row.zipWithIndex.map { case (_, index) => - // Uses default column names, "_c#" where # is its position of fields - // when header option is disabled. - s"_c$index" + None + } + } + /** + * Helper method that converts string representation of a character to actual character. + * It handles some Java escaped strings and throws exception if given string is longer than one + * character. + */ + @throws[IllegalArgumentException] + def toChar(str: String): Char = { + if (str.charAt(0) == '\\') { + str.charAt(1) + match { + case 't' => '\t' + case 'r' => '\r' + case 'b' => '\b' + case 'f' => '\f' + case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options + case '\'' => '\'' + case 'u' if str == """\u0000""" => '\u0000' + case _ => + throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") } + } else if (str.length == 1) { + str.charAt(0) + } else { + throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") } } @@ -126,7 +142,4 @@ object CSVUtils { csv.sample(withReplacement = false, options.samplingRatio, 1) } } - - def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = - CSVExprUtils.filterCommentAndEmpty(iter, options) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala index 37d9d9abc8680..4082a0df8ba75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala @@ -22,7 +22,6 @@ import java.io.Writer import com.univocity.parsers.csv.CsvWriter import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala similarity index 92% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala index 46ed58ed92830..9088d43905e28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.csv +package org.apache.spark.sql.execution.datasources.csv import java.io.InputStream import java.math.BigDecimal @@ -28,7 +28,8 @@ import com.univocity.parsers.csv.CsvParser import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.util.{BadRecordException, DateTimeUtils, FailureSafeParser} +import org.apache.spark.sql.catalyst.util.{BadRecordException, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.FailureSafeParser import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -263,7 +264,7 @@ class UnivocityParser( } } -private[sql] object UnivocityParser { +private[csv] object UnivocityParser { /** * Parses a stream that contains CSV strings and turns it into an iterator of tokens. @@ -272,10 +273,7 @@ private[sql] object UnivocityParser { inputStream: InputStream, shouldDropHeader: Boolean, tokenizer: CsvParser): Iterator[Array[String]] = { - val handleHeader: () => Unit = - () => if (shouldDropHeader) tokenizer.parseNext - - convertStream(inputStream, tokenizer, handleHeader)(tokens => tokens) + convertStream(inputStream, shouldDropHeader, tokenizer)(tokens => tokens) } /** @@ -283,9 +281,10 @@ private[sql] object UnivocityParser { */ def parseStream( inputStream: InputStream, + shouldDropHeader: Boolean, parser: UnivocityParser, - headerChecker: CSVHeaderChecker, - schema: StructType): Iterator[InternalRow] = { + schema: StructType, + checkHeader: Array[String] => Unit): Iterator[InternalRow] = { val tokenizer = parser.tokenizer val safeParser = new FailureSafeParser[Array[String]]( input => Seq(parser.convert(input)), @@ -293,26 +292,25 @@ private[sql] object UnivocityParser { schema, parser.options.columnNameOfCorruptRecord, parser.options.multiLine) - - val handleHeader: () => Unit = - () => headerChecker.checkHeaderColumnNames(tokenizer) - - convertStream(inputStream, tokenizer, handleHeader) { tokens => + convertStream(inputStream, shouldDropHeader, tokenizer, checkHeader) { tokens => safeParser.parse(tokens) }.flatten } private def convertStream[T]( inputStream: InputStream, + shouldDropHeader: Boolean, tokenizer: CsvParser, - handleHeader: () => Unit)( + checkHeader: Array[String] => Unit = _ => ())( convert: Array[String] => T) = new Iterator[T] { tokenizer.beginParsing(inputStream) - - // We can handle header here since here the stream is open. - handleHeader() - - private var nextRecord = tokenizer.parseNext() + private var nextRecord = { + if (shouldDropHeader) { + val firstRecord = tokenizer.parseNext() + checkHeader(firstRecord) + } + tokenizer.parseNext() + } override def hasNext: Boolean = nextRecord != null @@ -332,13 +330,10 @@ private[sql] object UnivocityParser { def parseIterator( lines: Iterator[String], parser: UnivocityParser, - headerChecker: CSVHeaderChecker, schema: StructType): Iterator[InternalRow] = { - headerChecker.checkHeaderColumnNames(lines, parser.tokenizer) - val options = parser.options - val filteredLines: Iterator[String] = CSVExprUtils.filterCommentAndEmpty(lines, options) + val filteredLines: Iterator[String] = CSVUtils.filterCommentAndEmpty(lines, options) val safeParser = new FailureSafeParser[String]( input => Seq(parser.parse(input)), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index c7608e2e881ff..76f58371ae264 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -34,7 +34,6 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions} -import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 1f7c9d73f19fe..a9241afba537b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -130,7 +130,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } (file: PartitionedFile) => { - val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val parser = new JacksonParser(actualSchema, parsedOptions) JsonDataSource(parsedOptions).readFile( broadcastedHadoopConf.value.value, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 0a64981b421c6..dbafc468c6c40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -67,16 +67,6 @@ private[sql] object OrcFilters { } } - // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters - // in order to distinguish predicate pushdown for nested columns. - private def quoteAttributeNameIfNeeded(name: String) : String = { - if (!name.contains("`") && name.contains(".")) { - s"`$name`" - } else { - name - } - } - /** * Create ORC filter as a SearchArgument instance. */ @@ -148,23 +138,6 @@ private[sql] object OrcFilters { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) - } - - /** - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the builder so far. - */ - private def createBuilder( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -172,52 +145,32 @@ private[sql] object OrcFilters { expression match { case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. + // At here, it is not safe to just convert one side if we do not understand the + // other side. Here is an example used to explain the reason. // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to // convert b in ('1'). If we only convert a = 2, we will end up with a filter // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => - for { - lhs <- createBuilder(dataTypeMap, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() - - case (Some(_), None) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) - - case (None, Some(_)) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) - - case _ => None - } + // Pushing one side of AND down is only safe to do at the top level. + // You can see ParquetRelation's initializeLocalJobFunc method as an example. + for { + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startAnd()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) + } yield rhs.end() case Or(left, right) => for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts = false) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts = false) - lhs <- createBuilder(dataTypeMap, left, - builder.startOr(), canPartialPushDownConjuncts = false) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts = false) + _ <- buildSearchArgument(dataTypeMap, left, newBuilder) + _ <- buildSearchArgument(dataTypeMap, right, newBuilder) + lhs <- buildSearchArgument(dataTypeMap, left, builder.startOr()) + rhs <- buildSearchArgument(dataTypeMap, right, lhs) } yield rhs.end() case Not(child) => for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) - negate <- createBuilder(dataTypeMap, - child, builder.startNot(), canPartialPushDownConjuncts = false) + _ <- buildSearchArgument(dataTypeMap, child, newBuilder) + negate <- buildSearchArgument(dataTypeMap, child, builder.startNot()) } yield negate.end() // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` @@ -225,47 +178,38 @@ private[sql] object OrcFilters { // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + Some(builder.startAnd().equals(attribute, getType(attribute), castedValue).end()) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + Some(builder.startAnd().nullSafeEquals(attribute, getType(attribute), castedValue).end()) case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + Some(builder.startAnd().lessThan(attribute, getType(attribute), castedValue).end()) case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + Some(builder.startAnd().lessThanEquals(attribute, getType(attribute), castedValue).end()) case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + Some(builder.startNot().lessThanEquals(attribute, getType(attribute), castedValue).end()) case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + Some(builder.startNot().lessThan(attribute, getType(attribute), castedValue).end()) case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + Some(builder.startAnd().isNull(attribute, getType(attribute)).end()) case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + Some(builder.startNot().isNull(attribute, getType(attribute)).end()) case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(quotedName, getType(attribute), + Some(builder.startAnd().in(attribute, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) case _ => None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 2c7a0b15ef7b7..a350b267675a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -424,14 +424,17 @@ class ParquetFileFormat (file: PartitionedFile) => { assert(file.partitionValues.numFields == partitionSchema.size) - val filePath = new Path(new URI(file.filePath)) + val fileSplit = + new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) + val filePath = fileSplit.getPath + val split = new org.apache.parquet.hadoop.ParquetInputSplit( filePath, - file.start, - file.start + file.length, - file.length, - Array.empty, + fileSplit.getStart, + fileSplit.getStart + fileSplit.getLength, + fileSplit.getLength, + fileSplit.getLocations, null) val sharedConf = broadcastedHadoopConf.value.value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 0bc946b208a62..0d36102634547 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -450,22 +450,13 @@ private[parquet] class ParquetFilters( */ def createFilter(schema: MessageType, predicate: sources.Filter): Option[FilterPredicate] = { val nameToParquetField = getFieldMap(schema) - createFilterHelper(nameToParquetField, predicate, canPartialPushDownConjuncts = true) + createFilterHelper(nameToParquetField, predicate, canRemoveOneSideInAnd = true) } - /** - * @param nameToParquetField a map from the field name to its field name and data type. - * This only includes the root fields whose types are primitive types. - * @param predicate the input filter predicates. Not all the predicates can be pushed down. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the Parquet-native filter predicates that are eligible for pushdown. - */ private def createFilterHelper( nameToParquetField: Map[String, ParquetField], predicate: sources.Filter, - canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { + canRemoveOneSideInAnd: Boolean): Option[FilterPredicate] = { // Decimal type must make sure that filter value's scale matched the file. // If doesn't matched, which would cause data corruption. def isDecimalMatched(value: Any, decimalMeta: DecimalMetadata): Boolean = value match { @@ -564,28 +555,24 @@ private[parquet] class ParquetFilters( // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val lhsFilterOption = - createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts) - val rhsFilterOption = - createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts) + val lhsFilterOption = createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd) + val rhsFilterOption = createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd) (lhsFilterOption, rhsFilterOption) match { case (Some(lhsFilter), Some(rhsFilter)) => Some(FilterApi.and(lhsFilter, rhsFilter)) - case (Some(lhsFilter), None) if canPartialPushDownConjuncts => Some(lhsFilter) - case (None, Some(rhsFilter)) if canPartialPushDownConjuncts => Some(rhsFilter) + case (Some(lhsFilter), None) if canRemoveOneSideInAnd => Some(lhsFilter) + case (None, Some(rhsFilter)) if canRemoveOneSideInAnd => Some(rhsFilter) case _ => None } case sources.Or(lhs, rhs) => for { - lhsFilter <- - createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts = false) - rhsFilter <- - createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts = false) + lhsFilter <- createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd = false) + rhsFilter <- createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd = false) } yield FilterApi.or(lhsFilter, rhsFilter) case sources.Not(pred) => - createFilterHelper(nameToParquetField, pred, canPartialPushDownConjuncts = false) + createFilterHelper(nameToParquetField, pred, canRemoveOneSideInAnd = false) .map(FilterApi.not) .map(LogicalInverseRewriter.rewrite) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 25f86a66a8269..04a97735d024d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -106,6 +106,8 @@ case class DataSourceV2ScanExec( override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) + override protected def needsUnsafeRowConversion: Boolean = false + override protected def doExecute(): RDD[InternalRow] = { if (supportsBatch) { WholeStageCodegenExec(this)(codegenStageId = 0).execute() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index d7d3f6d6078b4..f4b9d132122e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -494,7 +494,7 @@ case class SortMergeJoinExec( | $leftRow = null; | } else { | $matches.add((UnsafeRow) $rightRow); - | $rightRow = null; + | $rightRow = null;; | } | } while ($leftRow != null); | } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 256f12b605c92..e3dbc08f7c42e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -49,15 +49,6 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode } } -object BaseLimitExec { - private val curId = new java.util.concurrent.atomic.AtomicInteger() - - def newLimitCountTerm(): String = { - val id = curId.getAndIncrement() - s"_limit_counter_$id" - } -} - /** * Helper trait which defines methods that are shared by both * [[LocalLimitExec]] and [[GlobalLimitExec]]. @@ -78,25 +69,27 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport { // to the parent operator. override def usedInputs: AttributeSet = AttributeSet.empty - private lazy val countTerm = BaseLimitExec.newLimitCountTerm() - - override lazy val limitNotReachedChecks: Seq[String] = { - s"$countTerm < $limit" +: super.limitNotReachedChecks - } - protected override def doProduce(ctx: CodegenContext): String = { child.asInstanceOf[CodegenSupport].produce(ctx, this) } override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`. - // Here we have to inline it to not change its name. This is fine as we won't have many limit - // operators in one query. - ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false) + val stopEarly = + ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "stopEarly") // init as stopEarly = false + + ctx.addNewFunction("stopEarly", s""" + @Override + protected boolean stopEarly() { + return $stopEarly; + } + """, inlineToOuterClass = true) + val countTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "count") // init as count = 0 s""" | if ($countTerm < $limit) { | $countTerm += 1; | ${consume(ctx, input)} + | } else { + | $stopEarly = true; | } """.stripMargin } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 04623b1ab3c2f..18992d7a9f974 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -117,9 +117,9 @@ class ArrowPythonRunner( startTime: Long, env: SparkEnv, worker: Socket, - releasedOrClosed: AtomicBoolean, + released: AtomicBoolean, context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { private val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdin reader for $pythonExec", 0, Long.MaxValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index 752d271c4cc35..cc61faa7e7051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -59,9 +59,9 @@ class PythonUDFRunner( startTime: Long, env: SparkEnv, worker: Socket, - releasedOrClosed: AtomicBoolean, + released: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 1a25cd2a49e36..1b2d8a821b364 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -17,17 +17,16 @@ package org.apache.spark.sql.execution.ui -import java.net.URLEncoder import javax.servlet.http.HttpServletRequest -import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.xml.{Node, NodeSeq, Unparsed} +import scala.xml.{Node, NodeSeq} + +import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus import org.apache.spark.internal.Logging -import org.apache.spark.ui.{PagedDataSource, PagedTable, UIUtils, WebUIPage} -import org.apache.spark.util.Utils +import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with Logging { @@ -56,8 +55,8 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L val _content = mutable.ListBuffer[Node]() if (running.nonEmpty) { - val runningPageTable = - executionsTable(request, "running", running, currentTime, true, true, true) + val runningPageTable = new RunningExecutionTable( + parent, currentTime, running.sortBy(_.submissionTime).reverse).toNodeSeq(request) _content ++= -
    Running Queries: + Running Queries: {running.size} } @@ -130,7 +129,7 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L { if (completed.nonEmpty) {
  • - Completed Queries: + Completed Queries: {completed.size}
  • } @@ -138,232 +137,50 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L { if (failed.nonEmpty) {
  • - Failed Queries: + Failed Queries: {failed.size}
  • } } - UIUtils.headerSparkPage(request, "SQL", summary ++ content, parent, Some(5000)) } - - private def executionsTable( - request: HttpServletRequest, - executionTag: String, - executionData: Seq[SQLExecutionUIData], - currentTime: Long, - showRunningJobs: Boolean, - showSucceededJobs: Boolean, - showFailedJobs: Boolean): Seq[Node] = { - - // stripXSS is called to remove suspicious characters used in XSS attacks - val allParameters = request.getParameterMap.asScala.toMap.map { case (k, v) => - UIUtils.stripXSS(k) -> v.map(UIUtils.stripXSS).toSeq - } - val parameterOtherTable = allParameters.filterNot(_._1.startsWith(executionTag)) - .map(para => para._1 + "=" + para._2(0)) - - val parameterExecutionPage = UIUtils.stripXSS(request.getParameter(s"$executionTag.page")) - val parameterExecutionSortColumn = UIUtils.stripXSS(request - .getParameter(s"$executionTag.sort")) - val parameterExecutionSortDesc = UIUtils.stripXSS(request.getParameter(s"$executionTag.desc")) - val parameterExecutionPageSize = UIUtils.stripXSS(request - .getParameter(s"$executionTag.pageSize")) - val parameterExecutionPrevPageSize = UIUtils.stripXSS(request - .getParameter(s"$executionTag.prevPageSize")) - - val executionPage = Option(parameterExecutionPage).map(_.toInt).getOrElse(1) - val executionSortColumn = Option(parameterExecutionSortColumn).map { sortColumn => - UIUtils.decodeURLParameter(sortColumn) - }.getOrElse("ID") - val executionSortDesc = Option(parameterExecutionSortDesc).map(_.toBoolean).getOrElse( - // New executions should be shown above old executions by default. - executionSortColumn == "ID" - ) - val executionPageSize = Option(parameterExecutionPageSize).map(_.toInt).getOrElse(100) - val executionPrevPageSize = Option(parameterExecutionPrevPageSize).map(_.toInt) - .getOrElse(executionPageSize) - - // If the user has changed to a larger page size, then go to page 1 in order to avoid - // IndexOutOfBoundsException. - val page: Int = if (executionPageSize <= executionPrevPageSize) { - executionPage - } else { - 1 - } - val tableHeaderId = executionTag // "running", "completed" or "failed" - - try { - new ExecutionPagedTable( - request, - parent, - executionData, - tableHeaderId, - executionTag, - UIUtils.prependBaseUri(request, parent.basePath), - "SQL", // subPath - parameterOtherTable, - currentTime, - pageSize = executionPageSize, - sortColumn = executionSortColumn, - desc = executionSortDesc, - showRunningJobs, - showSucceededJobs, - showFailedJobs).table(page) - } catch { - case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => -
    -

    Error while rendering execution table:

    -
    -            {Utils.exceptionString(e)}
    -          
    -
    - } - } } -private[ui] class ExecutionPagedTable( - request: HttpServletRequest, +private[ui] abstract class ExecutionTable( parent: SQLTab, - data: Seq[SQLExecutionUIData], - tableHeaderId: String, - executionTag: String, - basePath: String, - subPath: String, - parameterOtherTable: Iterable[String], + tableId: String, currentTime: Long, - pageSize: Int, - sortColumn: String, - desc: Boolean, + executionUIDatas: Seq[SQLExecutionUIData], showRunningJobs: Boolean, showSucceededJobs: Boolean, - showFailedJobs: Boolean) extends PagedTable[ExecutionTableRowData] { + showFailedJobs: Boolean) { - override val dataSource = new ExecutionDataSource( - request, - parent, - data, - basePath, - currentTime, - pageSize, - sortColumn, - desc, - showRunningJobs, - showSucceededJobs, - showFailedJobs) - - private val parameterPath = s"$basePath/$subPath/?${parameterOtherTable.mkString("&")}" - - override def tableId: String = s"$executionTag-table" - - override def tableCssClass: String = - "table table-bordered table-condensed table-striped " + - "table-head-clickable table-cell-width-limited" - - override def prevPageSizeFormField: String = s"$executionTag.prevPageSize" - - override def pageLink(page: Int): String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - parameterPath + - s"&$pageNumberFormField=$page" + - s"&$executionTag.sort=$encodedSortColumn" + - s"&$executionTag.desc=$desc" + - s"&$pageSizeFormField=$pageSize" + - s"#$tableHeaderId" - } + protected def baseHeader: Seq[String] = Seq( + "ID", + "Description", + "Submitted", + "Duration") - override def pageSizeFormField: String = s"$executionTag.pageSize" + protected def header: Seq[String] - override def pageNumberFormField: String = s"$executionTag.page" - - override def goButtonFormPath: String = { - val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") - s"$parameterPath&$executionTag.sort=$encodedSortColumn&$executionTag.desc=$desc#$tableHeaderId" - } - - override def headers: Seq[Node] = { - // Information for each header: title, sortable - val executionHeadersAndCssClasses: Seq[(String, Boolean)] = - Seq( - ("ID", true), - ("Description", true), - ("Submitted", true), - ("Duration", true)) ++ { - if (showRunningJobs && showSucceededJobs && showFailedJobs) { - Seq( - ("Running Job IDs", true), - ("Succeeded Job IDs", true), - ("Failed Job IDs", true)) - } else if (showSucceededJobs && showFailedJobs) { - Seq( - ("Succeeded Job IDs", true), - ("Failed Job IDs", true)) - } else { - Seq(("Job IDs", true)) - } - } + protected def row( + request: HttpServletRequest, + currentTime: Long, + executionUIData: SQLExecutionUIData): Seq[Node] = { + val submissionTime = executionUIData.submissionTime + val duration = executionUIData.completionTime.map(_.getTime()).getOrElse(currentTime) - + submissionTime - val sortableColumnHeaders = executionHeadersAndCssClasses.filter { - case (_, sortable) => sortable - }.map { case (title, _) => title } - - require(sortableColumnHeaders.contains(sortColumn), s"Unknown column: $sortColumn") - - val headerRow: Seq[Node] = { - executionHeadersAndCssClasses.map { case (header, sortable) => - if (header == sortColumn) { - val headerLink = Unparsed( - parameterPath + - s"&$executionTag.sort=${URLEncoder.encode(header, "UTF-8")}" + - s"&$executionTag.desc=${!desc}" + - s"&$executionTag.pageSize=$pageSize" + - s"#$tableHeaderId") - val arrow = if (desc) "▾" else "▴" // UP or DOWN - - - - {header} -  {Unparsed(arrow)} - - - + def jobLinks(status: JobExecutionStatus): Seq[Node] = { + executionUIData.jobs.flatMap { case (jobId, jobStatus) => + if (jobStatus == status) { + [{jobId.toString}] } else { - if (sortable) { - val headerLink = Unparsed( - parameterPath + - s"&$executionTag.sort=${URLEncoder.encode(header, "UTF-8")}" + - s"&$executionTag.pageSize=$pageSize" + - s"#$tableHeaderId") - - - - {header} - - - } else { - - {header} - - } + None } - } - } - - {headerRow} - - } - - override def row(executionTableRow: ExecutionTableRowData): Seq[Node] = { - val executionUIData = executionTableRow.executionUIData - val submissionTime = executionUIData.submissionTime - val duration = executionTableRow.duration - - def jobLinks(jobData: Seq[Int]): Seq[Node] = { - jobData.map { jobId => - [{jobId.toString}] - } + }.toSeq } @@ -371,7 +188,7 @@ private[ui] class ExecutionPagedTable( {executionUIData.executionId.toString} - {descriptionCell(executionUIData)} + {descriptionCell(request, executionUIData)} {UIUtils.formatDate(submissionTime)} @@ -381,26 +198,27 @@ private[ui] class ExecutionPagedTable( {if (showRunningJobs) { - {jobLinks(executionTableRow.runningJobData)} + {jobLinks(JobExecutionStatus.RUNNING)} }} {if (showSucceededJobs) { - {jobLinks(executionTableRow.completedJobData)} + {jobLinks(JobExecutionStatus.SUCCEEDED)} }} {if (showFailedJobs) { - {jobLinks(executionTableRow.failedJobData)} + {jobLinks(JobExecutionStatus.FAILED)} }} } - private def descriptionCell(execution: SQLExecutionUIData): Seq[Node] = { + private def descriptionCell( + request: HttpServletRequest, + execution: SQLExecutionUIData): Seq[Node] = { val details = if (execution.details != null && execution.details.nonEmpty) { - + +details ++