From 048f07f9427653f9b271cc4d01b1f08d734ea8c5 Mon Sep 17 00:00:00 2001 From: Sungwoo Park Date: Wed, 11 Aug 2021 23:12:36 -0700 Subject: [PATCH] Remove Hive on Spark --- .../common/jsonexplain/JsonParserFactory.java | 1 - .../jsonexplain/spark/SparkJsonParser.java | 35 - .../metrics/common/MetricsConstant.java | 2 - .../org/apache/hadoop/hive/conf/HiveConf.java | 182 +---- .../apache/hadoop/hive/ql/log/PerfLogger.java | 13 - .../apache/hadoop/hive/conf/TestHiveConf.java | 13 - data/conf/perf-reg/spark/hive-site.xml | 284 ------- data/conf/spark/local/hive-site.xml | 274 ------- data/conf/spark/log4j2.properties | 69 -- data/conf/spark/standalone/hive-site.xml | 279 ------- data/conf/spark/yarn-client/hive-site.xml | 319 -------- itests/hive-unit/pom.xml | 55 -- .../ql/exec/spark/TestSparkStatistics.java | 100 --- .../jdbc/TestJdbcWithLocalClusterSpark.java | 318 -------- ...MultiSessionsHS2WithLocalClusterSpark.java | 256 ------- .../apache/hive/jdbc/TestRestrictedList.java | 11 - itests/pom.xml | 63 -- itests/qtest-spark/pom.xml | 474 ------------ .../hive/cli/TestLocalSparkCliDriver.java | 62 -- .../cli/TestMiniSparkOnYarnCliDriver.java | 66 -- .../hadoop/hive/cli/TestSparkCliDriver.java | 66 -- .../hive/cli/TestSparkNegativeCliDriver.java | 63 -- .../hive/cli/TestSparkPerfCliDriver.java | 62 -- .../hadoop/hive/cli/control/CliConfigs.java | 112 --- .../org/apache/hadoop/hive/ql/QTestUtil.java | 30 +- packaging/pom.xml | 3 + packaging/src/main/assembly/src.xml | 3 +- pom.xml | 45 +- ql/pom.xml | 47 +- .../org/apache/hadoop/hive/ql/Driver.java | 3 +- .../org/apache/hadoop/hive/ql/ErrorMsg.java | 35 +- .../hadoop/hive/ql/exec/ExplainTask.java | 31 - .../hadoop/hive/ql/exec/MapJoinOperator.java | 1 - .../hadoop/hive/ql/exec/OperatorFactory.java | 10 - .../hadoop/hive/ql/exec/OperatorUtils.java | 34 - .../hadoop/hive/ql/exec/ScriptOperator.java | 3 - .../hive/ql/exec/SerializationUtilities.java | 4 - .../ql/exec/SparkHashTableSinkOperator.java | 207 ------ .../hadoop/hive/ql/exec/TaskFactory.java | 3 - .../apache/hadoop/hive/ql/exec/Utilities.java | 7 +- .../hadoop/hive/ql/exec/spark/CacheTran.java | 62 -- .../hive/ql/exec/spark/GroupByShuffler.java | 41 -- .../hive/ql/exec/spark/HashTableLoader.java | 210 ------ .../spark/HiveBaseFunctionResultList.java | 123 ---- .../hive/ql/exec/spark/HiveKVResultCache.java | 267 ------- .../hive/ql/exec/spark/HiveMapFunction.java | 65 -- .../exec/spark/HiveMapFunctionResultList.java | 60 -- .../exec/spark/HivePairFlatMapFunction.java | 87 --- .../ql/exec/spark/HiveReduceFunction.java | 55 -- .../spark/HiveReduceFunctionResultList.java | 69 -- .../hive/ql/exec/spark/HiveSparkClient.java | 54 -- .../ql/exec/spark/HiveSparkClientFactory.java | 269 ------- .../hive/ql/exec/spark/HiveVoidFunction.java | 46 -- .../hive/ql/exec/spark/KryoSerializer.java | 97 --- .../ql/exec/spark/LocalHiveSparkClient.java | 245 ------- .../hadoop/hive/ql/exec/spark/MapInput.java | 101 --- .../hadoop/hive/ql/exec/spark/MapTran.java | 45 -- .../hadoop/hive/ql/exec/spark/ReduceTran.java | 45 -- .../ql/exec/spark/RemoteHiveSparkClient.java | 381 ---------- .../hive/ql/exec/spark/ShuffleTran.java | 77 -- .../hive/ql/exec/spark/SmallTableCache.java | 73 -- .../hive/ql/exec/spark/SortByShuffler.java | 67 -- .../spark/SparkDynamicPartitionPruner.java | 277 ------- .../ql/exec/spark/SparkMapRecordHandler.java | 215 ------ .../spark/SparkMergeFileRecordHandler.java | 123 ---- .../hadoop/hive/ql/exec/spark/SparkPlan.java | 192 ----- .../ql/exec/spark/SparkPlanGenerator.java | 378 ---------- .../ql/exec/spark/SparkRecordHandler.java | 128 ---- .../exec/spark/SparkReduceRecordHandler.java | 660 ----------------- .../hive/ql/exec/spark/SparkReporter.java | 83 --- .../hive/ql/exec/spark/SparkShuffler.java | 32 - .../hadoop/hive/ql/exec/spark/SparkTask.java | 520 ------------- .../hadoop/hive/ql/exec/spark/SparkTran.java | 32 - .../hive/ql/exec/spark/SparkUtilities.java | 342 --------- .../exec/spark/Statistic/SparkStatistic.java | 36 - .../spark/Statistic/SparkStatisticGroup.java | 50 -- .../exec/spark/Statistic/SparkStatistics.java | 42 -- .../Statistic/SparkStatisticsBuilder.java | 73 -- .../spark/Statistic/SparkStatisticsNames.java | 45 -- .../ql/exec/spark/session/SparkSession.java | 78 -- .../exec/spark/session/SparkSessionImpl.java | 271 ------- .../spark/session/SparkSessionManager.java | 65 -- .../session/SparkSessionManagerImpl.java | 158 ---- .../spark/status/LocalSparkJobMonitor.java | 142 ---- .../spark/status/RemoteSparkJobMonitor.java | 228 ------ .../ql/exec/spark/status/SparkJobMonitor.java | 376 ---------- .../ql/exec/spark/status/SparkJobRef.java | 29 - .../ql/exec/spark/status/SparkJobState.java | 29 - .../ql/exec/spark/status/SparkJobStatus.java | 53 -- .../exec/spark/status/SparkStageProgress.java | 80 -- .../spark/status/impl/JobMetricsListener.java | 90 --- .../spark/status/impl/LocalSparkJobRef.java | 67 -- .../status/impl/LocalSparkJobStatus.java | 210 ------ .../spark/status/impl/RemoteSparkJobRef.java | 62 -- .../status/impl/RemoteSparkJobStatus.java | 326 --------- .../spark/status/impl/SparkMetricsUtils.java | 64 -- .../VectorSparkHashTableSinkOperator.java | 123 ---- ...ctorSparkPartitionPruningSinkOperator.java | 115 --- .../hadoop/hive/ql/history/HiveHistory.java | 4 +- .../apache/hadoop/hive/ql/hooks/ATSHook.java | 7 +- .../hive/ql/hooks/HiveProtoLoggingHook.java | 4 +- .../hadoop/hive/ql/io/HiveInputFormat.java | 1 - .../hadoop/hive/ql/log/LogDivertAppender.java | 4 +- .../hive/ql/optimizer/GenMapRedUtils.java | 24 - .../ql/optimizer/QueryPlanPostProcessor.java | 8 - .../ql/optimizer/SparkMapJoinProcessor.java | 86 --- .../optimizer/SparkRemoveDynamicPruning.java | 80 -- .../AnnotateRunTimeStatsOptimizer.java | 7 - .../physical/GenSparkSkewJoinProcessor.java | 423 ----------- .../physical/SparkCrossProductCheck.java | 138 ---- .../SparkDynamicPartitionPruningResolver.java | 141 ---- .../physical/SparkMapJoinResolver.java | 394 ---------- .../ql/optimizer/physical/Vectorizer.java | 66 -- .../spark/CombineEquivalentWorkResolver.java | 436 ----------- .../spark/SetSparkReducerParallelism.java | 281 ------- .../spark/SparkJoinHintOptimizer.java | 72 -- .../optimizer/spark/SparkJoinOptimizer.java | 63 -- .../spark/SparkMapJoinOptimizer.java | 554 -------------- .../spark/SparkPartitionPruningSinkDesc.java | 144 ---- .../spark/SparkReduceSinkMapJoinProc.java | 293 -------- .../spark/SparkSMBJoinHintOptimizer.java | 102 --- .../spark/SparkSkewJoinProcFactory.java | 238 ------ .../spark/SparkSkewJoinResolver.java | 127 ---- .../spark/SparkSortMergeJoinFactory.java | 117 --- .../spark/SparkSortMergeJoinOptimizer.java | 112 --- .../spark/SplitSparkWorkResolver.java | 193 ----- .../hive/ql/parse/TaskCompilerFactory.java | 1 - .../ql/parse/spark/GenSparkProcContext.java | 193 ----- .../hive/ql/parse/spark/GenSparkUtils.java | 662 ----------------- .../hive/ql/parse/spark/GenSparkWork.java | 290 -------- .../ql/parse/spark/GenSparkWorkWalker.java | 114 --- .../parse/spark/OptimizeSparkProcContext.java | 80 -- .../hive/ql/parse/spark/SparkCompiler.java | 619 ---------------- .../parse/spark/SparkFileSinkProcessor.java | 57 -- .../SparkPartitionPruningSinkOperator.java | 298 -------- .../parse/spark/SparkProcessAnalyzeTable.java | 159 ---- .../ql/parse/spark/SparkSMBMapJoinInfo.java | 33 - .../ql/parse/spark/SplitOpTreeForDPP.java | 169 ----- .../plan/ConditionalResolverMergeFiles.java | 2 - .../ql/plan/SparkBucketMapJoinContext.java | 39 - .../hive/ql/plan/SparkEdgeProperty.java | 114 --- .../hive/ql/plan/SparkHashTableSinkDesc.java | 73 -- .../apache/hadoop/hive/ql/plan/SparkWork.java | 437 ----------- .../ql/plan/VectorSparkHashTableSinkDesc.java | 35 - .../VectorSparkPartitionPruningSinkDesc.java | 35 - .../hadoop/hive/ql/session/SessionState.java | 14 - .../hadoop/hive/ql/exec/TestExplainTask.java | 90 --- .../hive/ql/exec/TestOperatorNames.java | 13 - .../hadoop/hive/ql/exec/TestUtilities.java | 5 - .../hive/ql/exec/mr/TestMapRedTask.java | 2 - .../hive/ql/exec/mr/TestMapredLocalTask.java | 1 - .../ql/exec/spark/TestHiveKVResultCache.java | 350 --------- .../hive/ql/exec/spark/TestSparkTask.java | 125 ---- .../session/TestSparkSessionManagerImpl.java | 259 ------- .../hadoop/hive/ql/exec/tez/TestTezTask.java | 1 - .../TestExecutionEngineWorkConcurrency.java | 10 +- .../ql/processors/TestResetProcessor.java | 22 - .../hadoop/hive/ql/stats/TestStatsUtils.java | 29 +- .../TestGenericUDAFBinarySetFunctions.java | 6 +- .../clientnegative/spark_job_max_tasks.q | 1 - .../clientnegative/spark_stage_max_tasks.q | 1 - .../clientpositive/autoColumnStats_6.q | 1 - .../disable_merge_for_bucketing.q | 1 - ql/src/test/queries/clientpositive/merge1.q | 1 - ql/src/test/queries/clientpositive/merge2.q | 1 - .../test/queries/clientpositive/merge_empty.q | 1 - ql/src/test/queries/clientpositive/mm_all.q | 3 - .../test/queries/clientpositive/orc_merge1.q | 2 - .../test/queries/clientpositive/orc_merge2.q | 1 - .../test/queries/clientpositive/orc_merge3.q | 1 - .../test/queries/clientpositive/orc_merge4.q | 2 - .../test/queries/clientpositive/orc_merge5.q | 3 - .../test/queries/clientpositive/orc_merge6.q | 3 - .../test/queries/clientpositive/orc_merge7.q | 3 - .../test/queries/clientpositive/orc_merge8.q | 2 - .../test/queries/clientpositive/orc_merge9.q | 1 - .../clientpositive/orc_merge_diff_fs.q | 2 - .../clientpositive/orc_merge_incompat1.q | 1 - .../clientpositive/orc_merge_incompat2.q | 1 - .../clientpositive/skewjoin_union_remove_1.q | 1 - .../clientpositive/skewjoin_union_remove_2.q | 1 - .../spark_combine_equivalent_work_2.q | 1 - .../clientpositive/spark_constprog_dpp.q | 1 - .../spark_dynamic_partition_pruning.q | 14 - .../spark_dynamic_partition_pruning_2.q | 4 - .../spark_dynamic_partition_pruning_3.q | 1 - .../spark_dynamic_partition_pruning_4.q | 10 - .../spark_dynamic_partition_pruning_5.q | 1 - .../spark_dynamic_partition_pruning_6.q | 1 - ...k_dynamic_partition_pruning_mapjoin_only.q | 3 - ...amic_partition_pruning_recursive_mapjoin.q | 1 - .../spark_explain_groupbyshuffle.q | 2 - .../clientpositive/spark_explainuser_1.q | 1 - .../clientpositive/spark_opt_shuffle_serde.q | 3 - .../clientpositive/spark_union_merge.q | 2 - .../clientpositive/spark_use_op_stats.q | 1 - .../spark_use_ts_stats_for_mapjoin.q | 1 - ...ark_vectorized_dynamic_partition_pruning.q | 14 - .../queries/clientpositive/union_remove_1.q | 1 - .../queries/clientpositive/union_remove_10.q | 1 - .../queries/clientpositive/union_remove_11.q | 1 - .../queries/clientpositive/union_remove_12.q | 1 - .../queries/clientpositive/union_remove_13.q | 1 - .../queries/clientpositive/union_remove_14.q | 1 - .../queries/clientpositive/union_remove_15.q | 1 - .../queries/clientpositive/union_remove_16.q | 1 - .../queries/clientpositive/union_remove_17.q | 1 - .../queries/clientpositive/union_remove_18.q | 1 - .../queries/clientpositive/union_remove_19.q | 1 - .../queries/clientpositive/union_remove_2.q | 1 - .../queries/clientpositive/union_remove_20.q | 1 - .../queries/clientpositive/union_remove_21.q | 1 - .../queries/clientpositive/union_remove_22.q | 1 - .../queries/clientpositive/union_remove_23.q | 1 - .../queries/clientpositive/union_remove_24.q | 1 - .../queries/clientpositive/union_remove_25.q | 1 - .../queries/clientpositive/union_remove_3.q | 1 - .../queries/clientpositive/union_remove_4.q | 1 - .../queries/clientpositive/union_remove_5.q | 1 - .../queries/clientpositive/union_remove_6.q | 1 - .../queries/clientpositive/union_remove_7.q | 1 - .../queries/clientpositive/union_remove_8.q | 1 - .../queries/clientpositive/union_remove_9.q | 1 - spark-client/pom.xml | 205 ------ .../hive/spark/client/BaseProtocol.java | 164 ----- .../org/apache/hive/spark/client/Job.java | 32 - .../apache/hive/spark/client/JobContext.java | 67 -- .../hive/spark/client/JobContextImpl.java | 84 --- .../apache/hive/spark/client/JobHandle.java | 104 --- .../hive/spark/client/JobHandleImpl.java | 244 ------ .../hive/spark/client/MetricsCollection.java | 288 -------- .../hive/spark/client/MonitorCallback.java | 30 - .../hive/spark/client/RemoteDriver.java | 550 -------------- .../apache/hive/spark/client/SparkClient.java | 113 --- .../hive/spark/client/SparkClientFactory.java | 91 --- .../hive/spark/client/SparkClientImpl.java | 692 ------------------ .../spark/client/SparkClientUtilities.java | 189 ----- .../spark/client/metrics/DataReadMethod.java | 29 - .../spark/client/metrics/InputMetrics.java | 47 -- .../hive/spark/client/metrics/Metrics.java | 130 ---- .../client/metrics/ShuffleReadMetrics.java | 76 -- .../client/metrics/ShuffleWriteMetrics.java | 54 -- .../spark/client/rpc/KryoMessageCodec.java | 175 ----- .../apache/hive/spark/client/rpc/README.md | 33 - .../org/apache/hive/spark/client/rpc/Rpc.java | 504 ------------- .../spark/client/rpc/RpcConfiguration.java | 211 ------ .../hive/spark/client/rpc/RpcDispatcher.java | 200 ----- .../hive/spark/client/rpc/RpcException.java | 29 - .../hive/spark/client/rpc/RpcServer.java | 367 ---------- .../hive/spark/client/rpc/SaslHandler.java | 114 --- .../hive/spark/counter/SparkCounter.java | 109 --- .../hive/spark/counter/SparkCounterGroup.java | 85 --- .../hive/spark/counter/SparkCounters.java | 159 ---- .../hive/spark/client/TestJobHandle.java | 104 --- .../spark/client/TestMetricsCollection.java | 173 ----- .../hive/spark/client/TestSparkClient.java | 442 ----------- .../client/rpc/TestKryoMessageCodec.java | 233 ------ .../apache/hive/spark/client/rpc/TestRpc.java | 443 ----------- .../src/test/resources/log4j2.properties | 38 - .../hive/ptest/execution/conf/TestParser.java | 7 +- 260 files changed, 36 insertions(+), 27133 deletions(-) delete mode 100644 common/src/java/org/apache/hadoop/hive/common/jsonexplain/spark/SparkJsonParser.java delete mode 100644 data/conf/perf-reg/spark/hive-site.xml delete mode 100644 data/conf/spark/local/hive-site.xml delete mode 100644 data/conf/spark/log4j2.properties delete mode 100644 data/conf/spark/standalone/hive-site.xml delete mode 100644 data/conf/spark/yarn-client/hive-site.xml delete mode 100644 itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java delete mode 100644 itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java delete mode 100644 itests/qtest-spark/pom.xml delete mode 100644 itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestLocalSparkCliDriver.java delete mode 100644 itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java delete mode 100644 itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java delete mode 100644 itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java delete mode 100644 itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkPerfCliDriver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/GroupByShuffler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveVoidFunction.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapTran.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ReduceTran.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ShuffleTran.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SmallTableCache.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkShuffler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistic.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticGroup.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistics.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsBuilder.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsNames.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobState.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SparkMetricsUtils.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkMapJoinProcessor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkSMBMapJoinInfo.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/SparkHashTableSinkDesc.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java delete mode 100644 ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkPartitionPruningSinkDesc.java delete mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java delete mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java delete mode 100644 ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java delete mode 100644 spark-client/pom.xml delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/Job.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/metrics/DataReadMethod.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/KryoMessageCodec.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/README.md delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcException.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounter.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounterGroup.java delete mode 100644 spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java delete mode 100644 spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java delete mode 100644 spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java delete mode 100644 spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java delete mode 100644 spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java delete mode 100644 spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java delete mode 100644 spark-client/src/test/resources/log4j2.properties diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/JsonParserFactory.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/JsonParserFactory.java index c2e9cfd7979..b89795879a9 100644 --- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/JsonParserFactory.java +++ b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/JsonParserFactory.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hive.common.jsonexplain; -import org.apache.hadoop.hive.common.jsonexplain.spark.SparkJsonParser; import org.apache.hadoop.hive.common.jsonexplain.tez.TezJsonParser; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/spark/SparkJsonParser.java b/common/src/java/org/apache/hadoop/hive/common/jsonexplain/spark/SparkJsonParser.java deleted file mode 100644 index 9afa7582a21..00000000000 --- a/common/src/java/org/apache/hadoop/hive/common/jsonexplain/spark/SparkJsonParser.java +++ /dev/null @@ -1,35 +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.hadoop.hive.common.jsonexplain.spark; - -import org.apache.hadoop.hive.common.jsonexplain.DagJsonParser; - - -public class SparkJsonParser extends DagJsonParser { - - @Override - public String mapEdgeType(String edgeName) { - return edgeName; - } - - @Override - public String getFrameworkName() { - return "Spark"; - } -} \ No newline at end of file diff --git a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java index af0f87bac3b..cd75934b006 100644 --- a/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java +++ b/common/src/java/org/apache/hadoop/hive/common/metrics/common/MetricsConstant.java @@ -52,8 +52,6 @@ public class MetricsConstant { // The number of map reduce tasks executed by the HiveServer2 since the last restart public static final String HIVE_MR_TASKS = "hive_mapred_tasks"; - // The number of spark tasks executed by the HiveServer2 since the last restart - public static final String HIVE_SPARK_TASKS = "hive_spark_tasks"; // The number of tez tasks executed by the HiveServer2 since the last restart public static final String HIVE_TEZ_TASKS = "hive_tez_tasks"; public static final String HS2_OPEN_SESSIONS = "hs2_open_sessions"; diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index c7a40f92f7f..a2d4687a346 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -91,21 +91,11 @@ public class HiveConf extends Configuration { private static final Map metaConfs = new HashMap(); private final List restrictList = new ArrayList(); private final Set hiddenSet = new HashSet(); - private final List rscList = new ArrayList<>(); private Pattern modWhiteListPattern = null; - private volatile boolean isSparkConfigUpdated = false; private volatile boolean isMr3ConfigUpdated = false; private static final int LOG_PREFIX_LENGTH = 64; - public boolean getSparkConfigUpdated() { - return isSparkConfigUpdated; - } - - public void setSparkConfigUpdated(boolean isSparkConfigUpdated) { - this.isSparkConfigUpdated = isSparkConfigUpdated; - } - public boolean getMr3ConfigUpdated() { return isMr3ConfigUpdated; } @@ -1657,7 +1647,7 @@ public static enum ConfVars { HIVEMAPJOINUSEOPTIMIZEDTABLE("hive.mapjoin.optimized.hashtable", true, "Whether Hive should use memory-optimized hash table for MapJoin.\n" + - "Only works on Tez and Spark, because memory-optimized hashtable cannot be serialized."), + "Only works on Tez, because memory-optimized hashtable cannot be serialized."), HIVEMAPJOINOPTIMIZEDTABLEPROBEPERCENT("hive.mapjoin.optimized.hashtable.probe.percent", (float) 0.5, "Probing space percentage of the optimized hashtable"), HIVEUSEHYBRIDGRACEHASHJOIN("hive.mapjoin.hybridgrace.hashtable", true, "Whether to use hybrid" + @@ -1867,7 +1857,6 @@ public static enum ConfVars { HIVEMERGEMAPREDFILES("hive.merge.mapredfiles", false, "Merge small files at the end of a map-reduce job"), HIVEMERGETEZFILES("hive.merge.tezfiles", false, "Merge small files at the end of a Tez DAG"), - HIVEMERGESPARKFILES("hive.merge.sparkfiles", false, "Merge small files at the end of a Spark DAG Transformation"), HIVEMERGEMAPFILESSIZE("hive.merge.size.per.task", (long) (256 * 1000 * 1000), "Size of merged files at the end of the job"), HIVEMERGEMAPFILESAVGSIZE("hive.merge.smallfiles.avgsize", (long) (16 * 1000 * 1000), @@ -2078,8 +2067,7 @@ public static enum ConfVars { "Whether to grant access to the hs2/hive user for queries"), HIVEQUERYNAME ("hive.query.name", null, "This named is used by Tez to set the dag name. This name in turn will appear on \n" + - "the Tez UI representing the work that was done. Used by Spark to set the query name, will show up in the\n" + - "Spark UI."), + "the Tez UI representing the work that was done."), HIVEOPTIMIZEBUCKETINGSORTING("hive.optimize.bucketingsorting", true, "Don't create a reducer for enforcing \n" + @@ -2263,9 +2251,6 @@ public static enum ConfVars { "When shared work optimizer is enabled, whether we should reuse the cache for the broadcast side\n" + "of mapjoin operators that share same broadcast input. Requires hive.optimize.shared.work\n" + "to be set to true. Tez only."), - HIVE_COMBINE_EQUIVALENT_WORK_OPTIMIZATION("hive.combine.equivalent.work.optimization", true, "Whether to " + - "combine equivalent work objects during physical optimization.\n This optimization looks for equivalent " + - "work objects and combines them if they meet certain preconditions. Spark only."), HIVE_REMOVE_SQ_COUNT_CHECK("hive.optimize.remove.sq_count_check", false, "Whether to remove an extra join with sq_count_check for scalar subqueries " + "with constant group by keys."), @@ -2939,9 +2924,6 @@ public static enum ConfVars { HIVE_EXPLAIN_USER("hive.explain.user", true, "Whether to show explain result at user level.\n" + "When enabled, will log EXPLAIN output for the query at user level. Tez only."), - HIVE_SPARK_EXPLAIN_USER("hive.spark.explain.user", false, - "Whether to show explain result at user level.\n" + - "When enabled, will log EXPLAIN output for the query at user level. Spark only."), // prefix used to auto generated column aliases (this should be started with '_') HIVE_AUTOGEN_COLUMNALIAS_PREFIX_LABEL("hive.autogen.columnalias.prefix.label", "_c", @@ -3488,7 +3470,7 @@ public static enum ConfVars { HIVE_SERVER2_JOB_CREDENTIAL_PROVIDER_PATH("hive.server2.job.credential.provider.path", "", "If set, this configuration property should provide a comma-separated list of URLs that indicates the type and " + "location of providers to be used by hadoop credential provider API. It provides HiveServer2 the ability to provide job-specific " + - "credential providers for jobs run using MR and Spark execution engines. This functionality has not been tested against Tez."), + "credential providers for jobs run using MR execution engine. This functionality has not been tested against Tez."), HIVE_MOVE_FILES_THREAD_COUNT("hive.mv.files.thread", 15, new SizeValidator(0L, true, 1024L, true), "Number of threads" + " used to move files in move task. Set it to 0 to disable multi-threaded file moves. This parameter is also used by" + " MSCK to check tables."), @@ -3550,7 +3532,7 @@ public static enum ConfVars { HIVE_EXECUTION_ENGINE("hive.execution.engine", "mr3", new StringSet(true, "mr3", "tez"), "Chooses execution engine. Options are: mr3 or tez. While MR\n" + "remains the default engine for historical reasons, it is itself a historical engine\n" + - "and is deprecated in Hive 2 line. It may be removed without further warning. tez and spark are not supported."), + "and is deprecated in Hive 2 line. It may be removed without further warning."), HIVE_EXECUTION_MODE("hive.execution.mode", "container", new StringSet("container", "llap"), "Chooses whether query fragments will run in container or in llap"), @@ -3705,11 +3687,8 @@ public static enum ConfVars { "Whether to generate the splits locally or in the AM (tez only)"), HIVE_TEZ_GENERATE_CONSISTENT_SPLITS("hive.tez.input.generate.consistent.splits", true, "Whether to generate consistent split locations when generating splits in the AM"), - HIVE_PREWARM_ENABLED("hive.prewarm.enabled", false, "Enables container prewarm for Tez/Spark (Hadoop 2 only)"), - HIVE_PREWARM_NUM_CONTAINERS("hive.prewarm.numcontainers", 10, "Controls the number of containers to prewarm for Tez/Spark (Hadoop 2 only)"), - HIVE_PREWARM_SPARK_TIMEOUT("hive.prewarm.spark.timeout", "5000ms", - new TimeValidator(TimeUnit.MILLISECONDS), - "Time to wait to finish prewarming spark executors"), + HIVE_PREWARM_ENABLED("hive.prewarm.enabled", false, "Enables container prewarm for Tez (Hadoop 2 only)"), + HIVE_PREWARM_NUM_CONTAINERS("hive.prewarm.numcontainers", 10, "Controls the number of containers to prewarm for Tez(Hadoop 2 only)"), HIVESTAGEIDREARRANGE("hive.stageid.rearrange", "none", new StringSet("none", "idonly", "traverse", "execution"), ""), HIVEEXPLAINDEPENDENCYAPPENDTASKTYPES("hive.explain.dependency.append.tasktype", false, ""), HIVEUSEGOOGLEREGEXENGINE("hive.use.googleregex.engine",false,"whether to use google regex engine or not, default regex engine is java.util.regex"), @@ -3829,8 +3808,6 @@ public static enum ConfVars { + " only if the execution engine is tez."), TEZ_DAG_STATUS_CHECK_INTERVAL("hive.tez.dag.status.check.interval", "500ms", new TimeValidator(TimeUnit.MILLISECONDS), "Interval between subsequent DAG status invocation."), - SPARK_EXEC_INPLACE_PROGRESS("hive.spark.exec.inplace.progress", true, - "Updates spark job execution progress in-place in the terminal."), TEZ_CONTAINER_MAX_JAVA_HEAP_FRACTION("hive.tez.container.max.java.heap.fraction", 0.8f, "This is to override the tez setting with the same name"), TEZ_TASK_SCALE_MEMORY_RESERVE_FRACTION_MIN("hive.tez.task.scale.memory.reserve-fraction.min", @@ -4267,68 +4244,6 @@ public static enum ConfVars { "validated for all SQL operations after every defined interval (default: 500ms) and corresponding action\n" + "defined in the trigger will be taken"), - SPARK_USE_OP_STATS("hive.spark.use.op.stats", true, - "Whether to use operator stats to determine reducer parallelism for Hive on Spark.\n" + - "If this is false, Hive will use source table stats to determine reducer\n" + - "parallelism for all first level reduce tasks, and the maximum reducer parallelism\n" + - "from all parents for all the rest (second level and onward) reducer tasks."), - SPARK_USE_TS_STATS_FOR_MAPJOIN("hive.spark.use.ts.stats.for.mapjoin", false, - "If this is set to true, mapjoin optimization in Hive/Spark will use statistics from\n" + - "TableScan operators at the root of operator tree, instead of parent ReduceSink\n" + - "operators of the Join operator."), - SPARK_OPTIMIZE_SHUFFLE_SERDE("hive.spark.optimize.shuffle.serde", false, - "If this is set to true, Hive on Spark will register custom serializers for data types\n" + - "in shuffle. This should result in less shuffled data."), - SPARK_CLIENT_FUTURE_TIMEOUT("hive.spark.client.future.timeout", - "60s", new TimeValidator(TimeUnit.SECONDS), - "Timeout for requests from Hive client to remote Spark driver."), - SPARK_JOB_MONITOR_TIMEOUT("hive.spark.job.monitor.timeout", - "60s", new TimeValidator(TimeUnit.SECONDS), - "Timeout for job monitor to get Spark job state."), - SPARK_RPC_CLIENT_CONNECT_TIMEOUT("hive.spark.client.connect.timeout", - "1000ms", new TimeValidator(TimeUnit.MILLISECONDS), - "Timeout for remote Spark driver in connecting back to Hive client."), - SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT("hive.spark.client.server.connect.timeout", - "90000ms", new TimeValidator(TimeUnit.MILLISECONDS), - "Timeout for handshake between Hive client and remote Spark driver. Checked by both processes."), - SPARK_RPC_SECRET_RANDOM_BITS("hive.spark.client.secret.bits", "256", - "Number of bits of randomness in the generated secret for communication between Hive client and remote Spark driver. " + - "Rounded down to the nearest multiple of 8."), - SPARK_RPC_MAX_THREADS("hive.spark.client.rpc.threads", 8, - "Maximum number of threads for remote Spark driver's RPC event loop."), - SPARK_RPC_MAX_MESSAGE_SIZE("hive.spark.client.rpc.max.size", 50 * 1024 * 1024, - "Maximum message size in bytes for communication between Hive client and remote Spark driver. Default is 50MB."), - SPARK_RPC_CHANNEL_LOG_LEVEL("hive.spark.client.channel.log.level", null, - "Channel logging level for remote Spark driver. One of {DEBUG, ERROR, INFO, TRACE, WARN}."), - SPARK_RPC_SASL_MECHANISM("hive.spark.client.rpc.sasl.mechanisms", "DIGEST-MD5", - "Name of the SASL mechanism to use for authentication."), - SPARK_RPC_SERVER_ADDRESS("hive.spark.client.rpc.server.address", "", - "The server address of HiverServer2 host to be used for communication between Hive client and remote Spark driver. " + - "Default is empty, which means the address will be determined in the same way as for hive.server2.thrift.bind.host." + - "This is only necessary if the host has multiple network addresses and if a different network address other than " + - "hive.server2.thrift.bind.host is to be used."), - SPARK_RPC_SERVER_PORT("hive.spark.client.rpc.server.port", "", "A list of port ranges which can be used by RPC server " + - "with the format of 49152-49222,49228 and a random one is selected from the list. Default is empty, which randomly " + - "selects one port from all available ones."), - SPARK_DYNAMIC_PARTITION_PRUNING( - "hive.spark.dynamic.partition.pruning", false, - "When dynamic pruning is enabled, joins on partition keys will be processed by writing\n" + - "to a temporary HDFS file, and read later for removing unnecessary partitions."), - SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE( - "hive.spark.dynamic.partition.pruning.max.data.size", 100*1024*1024L, - "Maximum total data size in dynamic pruning."), - SPARK_DYNAMIC_PARTITION_PRUNING_MAP_JOIN_ONLY( - "hive.spark.dynamic.partition.pruning.map.join.only", false, - "Turn on dynamic partition pruning only for map joins.\n" + - "If hive.spark.dynamic.partition.pruning is set to true, this parameter value is ignored."), - SPARK_USE_GROUPBY_SHUFFLE( - "hive.spark.use.groupby.shuffle", true, - "Spark groupByKey transformation has better performance but uses unbounded memory." + - "Turn this off when there is a memory issue."), - SPARK_JOB_MAX_TASKS("hive.spark.job.max.tasks", -1, "The maximum number of tasks a Spark job may have.\n" + - "If a Spark job contains more tasks than the maximum, it will be cancelled. A value of -1 means no limit."), - SPARK_STAGE_MAX_TASKS("hive.spark.stage.max.tasks", -1, "The maximum number of tasks a stage in a Spark job may have.\n" + - "If a Spark job stage contains more tasks than the maximum, the job will be cancelled. A value of -1 means no limit."), NWAYJOINREORDER("hive.reorder.nway.joins", true, "Runs reordering of tables within single n-way join (i.e.: picks streamtable)"), HIVE_MERGE_NWAY_JOINS("hive.merge.nway.joins", true, @@ -4384,22 +4299,12 @@ public static enum ConfVars { "hive.server2.authentication.ldap.customLDAPQuery," + "hive.privilege.synchronizer," + "hive.privilege.synchronizer.interval," + - "hive.spark.client.connect.timeout," + - "hive.spark.client.server.connect.timeout," + - "hive.spark.client.channel.log.level," + - "hive.spark.client.rpc.max.size," + - "hive.spark.client.rpc.threads," + - "hive.spark.client.secret.bits," + - "hive.spark.client.rpc.server.address," + - "hive.spark.client.rpc.server.port," + - "hive.spark.client.rpc.sasl.mechanisms," + "bonecp.,"+ "hive.druid.broker.address.default,"+ "hive.druid.coordinator.address.default,"+ "hikari.,"+ "hadoop.bin.path,"+ - "yarn.bin.path,"+ - "spark.home", + "yarn.bin.path", "Comma separated list of configuration options which are immutable at runtime"), HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list", METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname @@ -4417,11 +4322,6 @@ public static enum ConfVars { HIVE_CONF_INTERNAL_VARIABLE_LIST("hive.conf.internal.variable.list", "hive.added.files.path,hive.added.jars.path,hive.added.archives.path", "Comma separated list of variables which are used internally and should not be configurable."), - HIVE_SPARK_RSC_CONF_LIST("hive.spark.rsc.conf.list", - SPARK_OPTIMIZE_SHUFFLE_SERDE.varname + "," + - SPARK_CLIENT_FUTURE_TIMEOUT.varname, - "Comma separated list of variables which are related to remote spark context.\n" + - "Changing these variables will result in re-creating the spark session."), HIVE_QUERY_TIMEOUT_SECONDS("hive.query.timeout.seconds", "0s", new TimeValidator(TimeUnit.SECONDS), "Timeout for Running Query in seconds. A nonpositive value means infinite. " + @@ -4910,9 +4810,6 @@ public void verifyAndSet(String name, String value) throws IllegalArgumentExcept // When either name or value is null, the set method below will fail, // and throw IllegalArgumentException set(name, value); - if (isSparkRelatedConfig(name)) { - isSparkConfigUpdated = true; - } } } @@ -4930,32 +4827,6 @@ public static boolean isEncodedPar(String name) { return false; } - /** - * check whether spark related property is updated, which includes spark configurations, - * RSC configurations and yarn configuration in Spark on YARN mode. - * @param name - * @return - */ - private boolean isSparkRelatedConfig(String name) { - boolean result = false; - if (name.startsWith("spark")) { // Spark property. - // for now we don't support changing spark app name on the fly - result = !name.equals("spark.app.name"); - } else if (name.startsWith("yarn")) { // YARN property in Spark on YARN mode. - String sparkMaster = get("spark.master"); - if (sparkMaster != null && sparkMaster.startsWith("yarn")) { - result = true; - } - } else if (rscList.stream().anyMatch(rscVar -> rscVar.equals(name))) { // Remote Spark Context property. - result = true; - } else if (name.equals("mapreduce.job.queuename")) { - // a special property starting with mapreduce that we would also like to effect if it changes - result = true; - } - - return result; - } - public static int getIntVar(Configuration conf, ConfVars var) { assert (var.valClass == Integer.class) : var.varname; if (var.altName != null) { @@ -5173,17 +5044,6 @@ public static void setBoolVar(Configuration conf, ConfVars var, boolean val) { conf.setBoolean(var.varname, val); } - /* Dynamic partition pruning is enabled in some or all cases if either - * hive.spark.dynamic.partition.pruning is true or - * hive.spark.dynamic.partition.pruning.map.join.only is true - */ - public static boolean isSparkDPPAny(Configuration conf) { - return (conf.getBoolean(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING.varname, - ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING.defaultBoolVal) || - conf.getBoolean(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAP_JOIN_ONLY.varname, - ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAP_JOIN_ONLY.defaultBoolVal)); - } - public boolean getBoolVar(ConfVars var) { return getBoolVar(this, var); } @@ -5318,7 +5178,6 @@ public HiveConf(HiveConf other) { super(other); hiveJar = other.hiveJar; auxJars = other.auxJars; - isSparkConfigUpdated = other.isSparkConfigUpdated; origProp = (Properties)other.origProp.clone(); restrictList.addAll(other.restrictList); hiddenSet.addAll(other.hiddenSet); @@ -5436,7 +5295,6 @@ private void initialize(Class cls) { setupRestrictList(); hiddenSet.clear(); hiddenSet.addAll(HiveConfUtil.getHiddenSet(this)); - setupRSCList(); } /** @@ -5832,17 +5690,6 @@ private void setupRestrictList() { restrictList.add(ConfVars.HIVE_CONF_RESTRICTED_LIST.varname); restrictList.add(ConfVars.HIVE_CONF_HIDDEN_LIST.varname); restrictList.add(ConfVars.HIVE_CONF_INTERNAL_VARIABLE_LIST.varname); - restrictList.add(ConfVars.HIVE_SPARK_RSC_CONF_LIST.varname); - } - - private void setupRSCList() { - rscList.clear(); - String vars = this.getVar(ConfVars.HIVE_SPARK_RSC_CONF_LIST); - if (vars != null) { - for (String var : vars.split(",")) { - rscList.add(var.trim()); - } - } } /** @@ -5866,21 +5713,6 @@ public boolean isWebUiQueryInfoCacheEnabled() { return isWebUiEnabled() && this.getIntVar(ConfVars.HIVE_SERVER2_WEBUI_MAX_HISTORIC_QUERIES) > 0; } - /* Dynamic partition pruning is enabled in some or all cases - */ - public boolean isSparkDPPAny() { - return isSparkDPPAny(this); - } - - /* Dynamic partition pruning is enabled only for map join - * hive.spark.dynamic.partition.pruning is false and - * hive.spark.dynamic.partition.pruning.map.join.only is true - */ - public boolean isSparkDPPOnlyMapjoin() { - return (!this.getBoolVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING) && - this.getBoolVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAP_JOIN_ONLY)); - } - public static boolean isLoadMetastoreConfig() { return loadMetastoreConfig; } diff --git a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java index 390ba3bea98..7d60f77e23f 100644 --- a/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java +++ b/common/src/java/org/apache/hadoop/hive/ql/log/PerfLogger.java @@ -80,19 +80,6 @@ public class PerfLogger { public static final String MR3_CREATE_VERTEX = "MR3CreateVertex"; public static final String MR3_RUN_VERTEX = "MR3RunVertex"; - public static final String SPARK_SUBMIT_TO_RUNNING = "SparkSubmitToRunning"; - public static final String SPARK_BUILD_PLAN = "SparkBuildPlan"; - public static final String SPARK_BUILD_RDD_GRAPH = "SparkBuildRDDGraph"; - public static final String SPARK_SUBMIT_JOB = "SparkSubmitJob"; - public static final String SPARK_RUN_JOB = "SparkRunJob"; - public static final String SPARK_CREATE_TRAN = "SparkCreateTran."; - public static final String SPARK_RUN_STAGE = "SparkRunStage."; - public static final String SPARK_INIT_OPERATORS = "SparkInitializeOperators"; - public static final String SPARK_GENERATE_TASK_TREE = "SparkGenerateTaskTree"; - public static final String SPARK_OPTIMIZE_OPERATOR_TREE = "SparkOptimizeOperatorTree"; - public static final String SPARK_OPTIMIZE_TASK_TREE = "SparkOptimizeTaskTree"; - public static final String SPARK_FLUSH_HASHTABLE = "SparkFlushHashTable."; - protected final Map startTimes = new HashMap(); protected final Map endTimes = new HashMap(); diff --git a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java index 780a708dab9..c2f01c614d1 100644 --- a/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java +++ b/common/src/test/org/apache/hadoop/hive/conf/TestHiveConf.java @@ -168,19 +168,6 @@ public void testHiddenConfig() throws Exception { } } - @Test - public void testSparkConfigUpdate(){ - HiveConf conf = new HiveConf(); - Assert.assertFalse(conf.getSparkConfigUpdated()); - - conf.verifyAndSet("spark.master", "yarn"); - Assert.assertTrue(conf.getSparkConfigUpdated()); - conf.verifyAndSet("hive.execution.engine", "spark"); - Assert.assertTrue("Expected spark config updated.", conf.getSparkConfigUpdated()); - - conf.setSparkConfigUpdated(false); - Assert.assertFalse(conf.getSparkConfigUpdated()); - } @Test public void testEncodingDecoding() throws UnsupportedEncodingException { HiveConf conf = new HiveConf(); diff --git a/data/conf/perf-reg/spark/hive-site.xml b/data/conf/perf-reg/spark/hive-site.xml deleted file mode 100644 index e52a2f9fa77..00000000000 --- a/data/conf/perf-reg/spark/hive-site.xml +++ /dev/null @@ -1,284 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - datanucleus.autoCreateSchema - true - - - - datanucleus.fixedDatastore - false - - - - hive.metastore.schema.verification - false - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - true - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - true - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge that stores temporary hive statistics. Currently, fs type is supported - - - - hive.execution.engine - spark - Whether to use MR or Spark - - - - hive.prewarm.enabled - true - - - - hive.prewarm.numcontainers - 1 - - - - spark.master - local-cluster[1,2,1024] - - - - hive.prewarm.spark.timeout - 30s - - - - spark.serializer - org.apache.spark.serializer.KryoSerializer - - - - spark.akka.logLifecycleEvents - true - - - - hive.spark.log.dir - ${spark.home}/logs/ - - - - spark.driver.extraClassPath - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar:${maven.local.repository}/org/antlr/antlr-runtime/${antlr.version}/antlr-runtime-${antlr.version}.jar - - - - spark.local.dir - ${test.tmp.dir}/spark-local-dir - - - - hive.aux.jars.path - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar - - - - hive.users.in.admin.role - hive_admin_user - - - - hive.in.test - true - Internal marker for test. Used for masking env-dependent values - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - - - - hive.query.results.cache.enabled - false - - - - yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage - 99 - - - - diff --git a/data/conf/spark/local/hive-site.xml b/data/conf/spark/local/hive-site.xml deleted file mode 100644 index c26bfaa008e..00000000000 --- a/data/conf/spark/local/hive-site.xml +++ /dev/null @@ -1,274 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - datanucleus.schema.autoCreateAll - true - - - - hive.metastore.schema.verification - false - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge that stores temporary hive statistics. Currently, fs type is supported - - - - hive.execution.engine - spark - Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark - - - - spark.master - local[*] - - - - hive.prewarm.enabled - true - - - - hive.prewarm.numcontainers - 1 - - - - hive.prewarm.spark.timeout - 30s - - - - spark.serializer - org.apache.spark.serializer.KryoSerializer - - - - spark.akka.logLifecycleEvents - true - - - - hive.spark.log.dir - ${spark.home}/logs/ - - - - spark.driver.extraClassPath - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar:${maven.local.repository}/org/antlr/antlr-runtime/${antlr.version}/antlr-runtime-${antlr.version}.jar - - - - spark.local.dir - ${test.tmp.dir}/spark-local-dir - - - - hive.aux.jars.path - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar - - - - hive.users.in.admin.role - hive_admin_user - - - - hive.in.test - true - Internal marker for test. Used for masking env-dependent values - - - - - hive.stats.column.autogather - false - - - - hive.query.results.cache.enabled - false - - - diff --git a/data/conf/spark/log4j2.properties b/data/conf/spark/log4j2.properties deleted file mode 100644 index c2deb4f9f88..00000000000 --- a/data/conf/spark/log4j2.properties +++ /dev/null @@ -1,69 +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. - -status = INFO -name = SparkLog4j2Test -packages = org.apache.hadoop.hive.ql.log - -# list of properties -property.spark.log.level = DEBUG -property.spark.root.logger = DRFA -property.spark.log.dir = target/tmp/log -property.spark.log.file = spark.log - -# list of all appenders -appenders = console, DRFA - -# console appender -appender.console.type = Console -appender.console.name = console -appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n - -# daily rolling file appender -appender.DRFA.type = RollingFile -appender.DRFA.name = DRFA -appender.DRFA.fileName = ${sys:spark.log.dir}/${sys:spark.log.file} -appender.DRFA.filePattern = ${sys:spark.log.dir}/${sys:spark.log.file}.%d{yyyy-MM-dd} -appender.DRFA.layout.type = PatternLayout -appender.DRFA.layout.pattern = %d{ISO8601} %5p [%t] %c{2}: %m%n -appender.DRFA.policies.type = Policies -appender.DRFA.policies.time.type = TimeBasedTriggeringPolicy -appender.DRFA.policies.time.interval = 1 -appender.DRFA.policies.time.modulate = true -appender.DRFA.strategy.type = DefaultRolloverStrategy -appender.DRFA.strategy.max = 30 - -# list of all loggers -loggers = SparkIMain, SparkILoop, Jetty, AbstractLifeCycle - -logger.SparkIMain.name = org.apache.spark.repl.SparkIMain$exprTyper -logger.SparkIMain.level = INFO - -logger.SparkILoop.name = org.apache.spark.repl.SparkILoop$SparkILoopInterpreter -logger.SparkILoop.level = INFO - -logger.Jetty.name = org.eclipse.jetty -logger.Jetty.level = WARN - -logger.AbstractLifeCycle.name = org.eclipse.jetty.util.component.AbstractLifeCycle -logger.AbstractLifeCycle.level = ERROR - -# root logger -rootLogger.level = ${sys:spark.log.level} -rootLogger.appenderRefs = root -rootLogger.appenderRef.root.ref = ${sys:spark.root.logger} diff --git a/data/conf/spark/standalone/hive-site.xml b/data/conf/spark/standalone/hive-site.xml deleted file mode 100644 index 79e388ec06f..00000000000 --- a/data/conf/spark/standalone/hive-site.xml +++ /dev/null @@ -1,279 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - datanucleus.schema.autoCreateAll - true - - - - hive.metastore.schema.verification - false - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge that stores temporary hive statistics. Currently, fs type is supported - - - - hive.execution.engine - spark - Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark - - - - spark.master - local-cluster[1,2,1024] - - - - hive.prewarm.enabled - true - - - - hive.prewarm.numcontainers - 1 - - - - hive.prewarm.spark.timeout - 30s - - - - spark.serializer - org.apache.spark.serializer.KryoSerializer - - - - spark.akka.logLifecycleEvents - true - - - - hive.spark.log.dir - ${spark.home}/logs/ - - - - spark.driver.extraClassPath - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar:${maven.local.repository}/org/antlr/antlr-runtime/${antlr.version}/antlr-runtime-${antlr.version}.jar - - - - spark.local.dir - ${test.tmp.dir}/spark-local-dir - - - - hive.aux.jars.path - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar - - - - hive.users.in.admin.role - hive_admin_user - - - - hive.in.test - true - Internal marker for test. Used for masking env-dependent values - - - - hive.spark.client.connect.timeout - 30000ms - - - - - hive.stats.column.autogather - false - - - - hive.query.results.cache.enabled - false - - - diff --git a/data/conf/spark/yarn-client/hive-site.xml b/data/conf/spark/yarn-client/hive-site.xml deleted file mode 100644 index f527ca06e22..00000000000 --- a/data/conf/spark/yarn-client/hive-site.xml +++ /dev/null @@ -1,319 +0,0 @@ - - - - - - - - - - - - - - - hadoop.tmp.dir - ${test.tmp.dir}/hadoop-tmp - A base for other temporary directories. - - - - hive.exec.scratchdir - ${test.tmp.dir}/scratchdir - Scratch space for Hive jobs - - - - hive.exec.local.scratchdir - ${test.tmp.dir}/localscratchdir/ - Local scratch space for Hive jobs - - - - datanucleus.schema.autoCreateAll - true - - - - hive.metastore.schema.verification - false - - - - javax.jdo.option.ConnectionURL - jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true - - - - javax.jdo.option.ConnectionDriverName - org.apache.derby.jdbc.EmbeddedDriver - - - - javax.jdo.option.ConnectionUserName - APP - - - - javax.jdo.option.ConnectionPassword - mine - - - - - hive.metastore.warehouse.dir - ${test.warehouse.dir} - - - - - hive.metastore.metadb.dir - file://${test.tmp.dir}/metadb/ - - Required by metastore server or if the uris argument below is not supplied - - - - - test.log.dir - ${test.tmp.dir}/log/ - - - - - test.data.files - ${hive.root}/data/files - - - - - hive.jar.path - ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - - test.data.scripts - ${hive.root}/data/scripts - - - - - hive.metastore.rawstore.impl - org.apache.hadoop.hive.metastore.ObjectStore - Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database - - - - hive.querylog.location - ${test.tmp.dir}/tmp - Location of the structured hive logs - - - - hive.exec.pre.hooks - org.apache.hadoop.hive.ql.hooks.PreExecutePrinter, org.apache.hadoop.hive.ql.hooks.EnforceReadOnlyTables - Pre Execute Hook for Tests - - - - hive.exec.post.hooks - org.apache.hadoop.hive.ql.hooks.PostExecutePrinter - Post Execute Hook for Tests - - - - hive.support.concurrency - false - Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. - - - - fs.pfile.impl - org.apache.hadoop.fs.ProxyLocalFileSystem - A proxy for local file system used for cross file system testing - - - - hive.exec.mode.local.auto - false - - Let hive determine whether to run in local mode automatically - Disabling this for tests so that minimr is not affected - - - - - hive.auto.convert.join - false - Whether Hive enable the optimization about converting common join into mapjoin based on the input file size - - - - hive.ignore.mapjoin.hint - false - Whether Hive ignores the mapjoin hint - - - - io.sort.mb - 10 - - - - hive.input.format - org.apache.hadoop.hive.ql.io.CombineHiveInputFormat - The default input format, if it is not specified, the system assigns it. It is set to HiveInputFormat for hadoop versions 17, 18 and 19, whereas it is set to CombineHiveInputFormat for hadoop 20. The user can always overwrite it - if there is a bug in CombineHiveInputFormat, it can always be manually set to HiveInputFormat. - - - - hive.default.rcfile.serde - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - The default SerDe hive will use for the rcfile format - - - - hive.stats.dbclass - fs - The default storatge that stores temporary hive statistics. Currently, fs type is supported - - - - hive.execution.engine - spark - Chooses execution engine. Options are: mr (Map reduce, default), tez (hadoop 2 only), spark - - - - spark.master - yarn - - - - spark.submit.deployMode - cluster - - - - spark.serializer - org.apache.spark.serializer.KryoSerializer - - - - spark.akka.logLifecycleEvents - true - - - - hive.spark.log.dir - ${spark.home}/logs/ - - - - spark.driver.extraClassPath - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar:${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar - - - - spark.executor.instances - 2 - - - - spark.executor.cores - 2 - - - - spark.executor.memory - 512m - - - - spark.yarn.executor.memoryOverhead - 0 - - - - spark.driver.memory - 512m - - - - spark.yarn.driver.memoryOverhead - 0 - - - - spark.local.dir - ${test.tmp.dir}/spark-local-dir - - - - hive.prewarm.enabled - true - - - - hive.prewarm.spark.timeout - 30s - - - - spark.testing - true - - - - hive.aux.jars.path - ${maven.local.repository}/org/apache/hive/hive-it-util/${hive.version}/hive-it-util-${hive.version}.jar - - - - hive.users.in.admin.role - hive_admin_user - - - - hive.in.test - true - Internal marker for test. Used for masking env-dependent values - - - - hive.spark.client.connect.timeout - 30000ms - - - - yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage - 99 - - - - - hive.stats.column.autogather - false - - - - hive.query.results.cache.enabled - false - - - diff --git a/itests/hive-unit/pom.xml b/itests/hive-unit/pom.xml index a5710f4b221..6cad063826a 100644 --- a/itests/hive-unit/pom.xml +++ b/itests/hive-unit/pom.xml @@ -29,7 +29,6 @@ ../.. - ${basedir}/${hive.path.to.root}/itests/hive-unit/target/spark @@ -487,60 +486,6 @@ - - spark-test - - - !skipSparkTests - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - com.esotericsoftware.kryo - kryo - - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - download-spark - generate-sources - - run - - - - - - - - - - - - - - diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java deleted file mode 100644 index be3b501110b..00000000000 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/exec/spark/TestSparkStatistics.java +++ /dev/null @@ -1,100 +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.hadoop.hive.ql.exec.spark; - -import com.google.common.collect.Lists; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistic; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; -import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; -import org.apache.hadoop.hive.ql.session.SessionState; - -import org.junit.Assert; -import org.junit.Test; - -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class TestSparkStatistics { - - @Test - public void testSparkStatistics() { - HiveConf conf = new HiveConf(); - conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, - SQLStdHiveAuthorizerFactory.class.getName()); - conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - conf.setVar(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "spark"); - conf.set("spark.master", "local-cluster[1,2,1024]"); - - SessionState.start(conf); - - Driver driver = null; - - try { - driver = new Driver(new QueryState.Builder() - .withGenerateNewQueryId(true) - .withHiveConf(conf).build(), - null, null); - - Assert.assertEquals(0, driver.run("create table test (col int)").getResponseCode()); - Assert.assertEquals(0, driver.compile("select * from test order by col")); - - List sparkTasks = Utilities.getSparkTasks(driver.getPlan().getRootTasks()); - Assert.assertEquals(1, sparkTasks.size()); - - SparkTask sparkTask = sparkTasks.get(0); - - DriverContext driverCxt = new DriverContext(driver.getContext()); - driverCxt.prepare(driver.getPlan()); - - sparkTask.initialize(driver.getQueryState(), driver.getPlan(), driverCxt, driver.getContext() - .getOpContext()); - Assert.assertEquals(0, sparkTask.execute(driverCxt)); - - Assert.assertNotNull(sparkTask.getSparkStatistics()); - - List sparkStats = Lists.newArrayList(sparkTask.getSparkStatistics() - .getStatisticGroup(SparkStatisticsNames.SPARK_GROUP_NAME).getStatistics()); - - Assert.assertEquals(18, sparkStats.size()); - - Map statsMap = sparkStats.stream().collect( - Collectors.toMap(SparkStatistic::getName, SparkStatistic::getValue)); - - Assert.assertTrue(Long.parseLong(statsMap.get(SparkStatisticsNames.TASK_DURATION_TIME)) > 0); - Assert.assertTrue(Long.parseLong(statsMap.get(SparkStatisticsNames.EXECUTOR_CPU_TIME)) > 0); - Assert.assertTrue( - Long.parseLong(statsMap.get(SparkStatisticsNames.EXECUTOR_DESERIALIZE_CPU_TIME)) > 0); - Assert.assertTrue( - Long.parseLong(statsMap.get(SparkStatisticsNames.EXECUTOR_DESERIALIZE_TIME)) > 0); - Assert.assertTrue(Long.parseLong(statsMap.get(SparkStatisticsNames.EXECUTOR_RUN_TIME)) > 0); - } finally { - if (driver != null) { - Assert.assertEquals(0, driver.run("drop table if exists test").getResponseCode()); - driver.destroy(); - } - } - } -} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java deleted file mode 100644 index 2a4da20df95..00000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcWithLocalClusterSpark.java +++ /dev/null @@ -1,318 +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.hive.jdbc; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.HashMap; -import java.util.Map; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.jdbc.miniHS2.MiniHS2; -import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.session.HiveSessionHook; -import org.apache.hive.service.cli.session.HiveSessionHookContext; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -/** - * This class is cloned from TestJdbcWithMiniMR, except use Spark as the execution engine. - */ -public class TestJdbcWithLocalClusterSpark { - public static final String TEST_TAG = "miniHS2.localClusterSpark.tag"; - public static final String TEST_TAG_VALUE = "miniHS2.localClusterSpark.value"; - public static class LocalClusterSparkSessionHook implements HiveSessionHook { - @Override - public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException { - sessionHookContext.getSessionConf().set(TEST_TAG, TEST_TAG_VALUE); - } - } - - private static MiniHS2 miniHS2 = null; - private static HiveConf conf; - private static Path dataFilePath; - private static String dbName = "mrTestDb"; - private Connection hs2Conn = null; - private Statement stmt; - - private static HiveConf createHiveConf() { - HiveConf conf = new HiveConf(); - conf.set("hive.execution.engine", "spark"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.set("spark.master", "local-cluster[2,2,1024]"); - conf.set("hive.spark.client.connect.timeout", "30000ms"); - // FIXME: Hadoop3 made the incompatible change for dfs.client.datanode-restart.timeout - // while spark2 is still using Hadoop2. - // Spark requires Hive to support Hadoop3 first then Spark can start - // working on Hadoop3 support. Remove this after Spark supports Hadoop3. - conf.set("dfs.client.datanode-restart.timeout", "30"); - return conf; - } - - @BeforeClass - public static void beforeTest() throws Exception { - Class.forName(MiniHS2.getJdbcDriverName()); - conf = createHiveConf(); - conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - String dataFileDir = conf.get("test.data.files").replace('\\', '/') - .replace("c:", ""); - dataFilePath = new Path(dataFileDir, "kv1.txt"); - DriverManager.setLoginTimeout(0); - conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - miniHS2 = new MiniHS2(conf, MiniClusterType.MR); - Map overlayProps = new HashMap(); - overlayProps.put(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname, - LocalClusterSparkSessionHook.class.getName()); - miniHS2.start(overlayProps); - createDb(); - } - - // setup DB - private static void createDb() throws Exception { - Connection conn = DriverManager. - getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), "bar"); - Statement stmt2 = conn.createStatement(); - stmt2.execute("DROP DATABASE IF EXISTS " + dbName + " CASCADE"); - stmt2.execute("CREATE DATABASE " + dbName); - stmt2.close(); - conn.close(); - } - - @Before - public void setUp() throws Exception { - hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL(dbName), - System.getProperty("user.name"), "bar"); - stmt = hs2Conn.createStatement(); - stmt.execute("USE " + dbName); - } - - @After - public void tearDown() throws Exception { - if (hs2Conn != null) { - hs2Conn.close(); - } - } - - @AfterClass - public static void afterTest() throws Exception { - if (miniHS2 != null && miniHS2.isStarted()) { - miniHS2.stop(); - } - } - - /** - * Verify that the connection to HS2 with MiniMr is successful. - * @throws Exception - */ - @Test - public void testConnection() throws Exception { - // the session hook should set the property - verifyProperty(TEST_TAG, TEST_TAG_VALUE); - } - - /** - * Run nonMr query. - * @throws Exception - */ - @Test - public void testNonSparkQuery() throws Exception { - String tableName = "testTab1"; - String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tableName; - - testKvQuery(tableName, queryStr, resultVal); - } - - /** - * Run nonMr query. - * @throws Exception - */ - @Test - public void testSparkQuery() throws Exception { - String tableName = "testTab2"; - String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tableName - + " where value = '" + resultVal + "'"; - - testKvQuery(tableName, queryStr, resultVal); - } - - @Test - public void testPermFunc() throws Exception { - - // This test assumes the hive-contrib JAR has been built as part of the Hive build. - // Also dependent on the UDFExampleAdd class within that JAR. - String udfClassName = "org.apache.hadoop.hive.contrib.udf.example.UDFExampleAdd"; - String mvnRepo = System.getProperty("maven.local.repository"); - String hiveVersion = System.getProperty("hive.version"); - String jarFileName = "hive-contrib-" + hiveVersion + ".jar"; - String[] pathParts = { - "org", "apache", "hive", - "hive-contrib", hiveVersion, jarFileName - }; - - // Create path to hive-contrib JAR on local filesystem - Path contribJarPath = new Path(mvnRepo); - for (String pathPart : pathParts) { - contribJarPath = new Path(contribJarPath, pathPart); - } - FileSystem localFs = FileSystem.getLocal(conf); - assertTrue("Hive contrib JAR exists at " + contribJarPath, localFs.exists(contribJarPath)); - - String hdfsJarPathStr = "hdfs:///" + jarFileName; - Path hdfsJarPath = new Path(hdfsJarPathStr); - - // Copy JAR to DFS - FileSystem dfs = miniHS2.getDFS().getFileSystem(); - dfs.copyFromLocalFile(contribJarPath, hdfsJarPath); - assertTrue("Verify contrib JAR copied to HDFS at " + hdfsJarPath, dfs.exists(hdfsJarPath)); - - // Register function - String queryStr = "CREATE FUNCTION example_add AS '" + udfClassName + "'" - + " USING JAR '" + hdfsJarPathStr + "'"; - stmt.execute(queryStr); - - // Call describe - ResultSet res; - res = stmt.executeQuery("DESCRIBE FUNCTION " + dbName + ".example_add"); - checkForNotExist(res); - - // Use UDF in query - String tableName = "testTab3"; - setupKv1Tabs(tableName); - res = stmt.executeQuery("SELECT EXAMPLE_ADD(1, 2) FROM " + tableName + " LIMIT 1"); - assertTrue("query has results", res.next()); - assertEquals(3, res.getInt(1)); - assertFalse("no more results", res.next()); - - // A new connection should be able to call describe/use function without issue - Connection conn2 = DriverManager.getConnection(miniHS2.getJdbcURL(dbName), - System.getProperty("user.name"), "bar"); - Statement stmt2 = conn2.createStatement(); - stmt2.execute("USE " + dbName); - res = stmt2.executeQuery("DESCRIBE FUNCTION " + dbName + ".example_add"); - checkForNotExist(res); - - res = stmt2.executeQuery("SELECT " + dbName + ".example_add(1, 1) FROM " + tableName + " LIMIT 1"); - assertTrue("query has results", res.next()); - assertEquals(2, res.getInt(1)); - assertFalse("no more results", res.next()); - - stmt.execute("DROP TABLE " + tableName); - } - - @Test - public void testTempTable() throws Exception { - // Create temp table with current connection - String tempTableName = "tmp1"; - stmt.execute("CREATE TEMPORARY TABLE " + tempTableName + " (key string, value string)"); - stmt.execute("load data local inpath '" - + dataFilePath.toString() + "' into table " + tempTableName); - - String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tempTableName - + " where value = '" + resultVal + "'"; - verifyResult(queryStr, resultVal, 2); - - // A second connection should not be able to see the table - Connection conn2 = DriverManager.getConnection(miniHS2.getJdbcURL(dbName), - System.getProperty("user.name"), "bar"); - Statement stmt2 = conn2.createStatement(); - stmt2.execute("USE " + dbName); - boolean gotException = false; - try { - stmt2.executeQuery(queryStr); - } catch (SQLException err) { - // This is expected to fail. - assertTrue("Expecting table not found error, instead got: " + err, - err.getMessage().contains("Table not found")); - gotException = true; - } - assertTrue("Exception while querying non-existing temp table", gotException); - } - - private void checkForNotExist(ResultSet res) throws Exception { - int numRows = 0; - while (res.next()) { - numRows++; - String strVal = res.getString(1); - assertEquals("Should not find 'not exist'", -1, strVal.toLowerCase().indexOf("not exist")); - } - assertTrue("Rows returned from describe function", numRows > 0); - } - - /** - * Verify if the given property contains the expected value. - * @param propertyName - * @param expectedValue - * @throws Exception - */ - private void verifyProperty(String propertyName, String expectedValue) throws Exception { - Statement stmt = hs2Conn .createStatement(); - ResultSet res = stmt.executeQuery("set " + propertyName); - assertTrue(res.next()); - String[] results = res.getString(1).split("="); - assertEquals("Property should be set", results.length, 2); - assertEquals("Property should be set", expectedValue, results[1]); - } - - // create tables, verify query - private void testKvQuery(String tableName, String queryStr, String resultVal) - throws SQLException { - setupKv1Tabs(tableName); - verifyResult(queryStr, resultVal, 2); - stmt.execute("DROP TABLE " + tableName); - } - - // create table and pupulate with kv1.txt - private void setupKv1Tabs(String tableName) throws SQLException { - Statement stmt = hs2Conn.createStatement(); - // create table - stmt.execute("CREATE TABLE " + tableName - + " (under_col INT COMMENT 'the under column', value STRING)" - + " COMMENT ' test table'"); - - // load data - stmt.execute("load data local inpath '" - + dataFilePath.toString() + "' into table " + tableName); - } - - // run given query and validate expecated result - private void verifyResult(String queryStr, String expString, int colPos) - throws SQLException { - ResultSet res = stmt.executeQuery(queryStr); - assertTrue(res.next()); - assertEquals(expString, res.getString(colPos)); - res.close(); - } -} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java deleted file mode 100644 index 9f72e51187d..00000000000 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestMultiSessionsHS2WithLocalClusterSpark.java +++ /dev/null @@ -1,256 +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.hive.jdbc; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hive.jdbc.miniHS2.MiniHS2; -import org.apache.hive.jdbc.miniHS2.MiniHS2.MiniClusterType; -import org.apache.hive.service.cli.HiveSQLException; -import org.apache.hive.service.cli.session.HiveSessionHook; -import org.apache.hive.service.cli.session.HiveSessionHookContext; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -public class TestMultiSessionsHS2WithLocalClusterSpark { - public static final String TEST_TAG = "miniHS2.localClusterSpark.tag"; - public static final String TEST_TAG_VALUE = "miniHS2.localClusterSpark.value"; - private static final int PARALLEL_NUMBER = 3; - - public static class LocalClusterSparkSessionHook implements HiveSessionHook { - @Override - public void run(HiveSessionHookContext sessionHookContext) throws HiveSQLException { - sessionHookContext.getSessionConf().set(TEST_TAG, TEST_TAG_VALUE); - } - } - - private static MiniHS2 miniHS2 = null; - private static HiveConf conf; - private static Path dataFilePath; - private static String dbName = "sparkTestDb"; - private ThreadLocal localConnection = new ThreadLocal(); - private ThreadLocal localStatement = new ThreadLocal(); - private ExecutorService pool = null; - - - private static HiveConf createHiveConf() { - HiveConf conf = new HiveConf(); - conf.set("hive.exec.parallel", "true"); - conf.set("hive.execution.engine", "spark"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.set("spark.master", "local-cluster[2,2,1024]"); - conf.set("spark.deploy.defaultCores", "2"); - conf.set("hive.spark.client.connect.timeout", "30000ms"); - // FIXME: Hadoop3 made the incompatible change for dfs.client.datanode-restart.timeout - // while spark2 is still using Hadoop2. - // Spark requires Hive to support Hadoop3 first then Spark can start - // working on Hadoop3 support. Remove this after Spark supports Hadoop3. - conf.set("dfs.client.datanode-restart.timeout", "30"); - return conf; - } - - @BeforeClass - public static void beforeTest() throws Exception { - Class.forName(MiniHS2.getJdbcDriverName()); - conf = createHiveConf(); - conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - String dataFileDir = conf.get("test.data.files").replace('\\', '/') - .replace("c:", ""); - dataFilePath = new Path(dataFileDir, "kv1.txt"); - DriverManager.setLoginTimeout(0); - conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false); - miniHS2 = new MiniHS2(conf, MiniClusterType.MR); - Map overlayProps = new HashMap(); - overlayProps.put(ConfVars.HIVE_SERVER2_SESSION_HOOK.varname, - LocalClusterSparkSessionHook.class.getName()); - miniHS2.start(overlayProps); - createDb(); - } - - // setup DB - private static void createDb() throws Exception { - Connection conn = DriverManager. - getConnection(miniHS2.getJdbcURL(), System.getProperty("user.name"), "bar"); - Statement stmt2 = conn.createStatement(); - stmt2.execute("DROP DATABASE IF EXISTS " + dbName + " CASCADE"); - stmt2.execute("CREATE DATABASE " + dbName); - stmt2.close(); - conn.close(); - } - - @Before - public void setUp() throws Exception { - pool = Executors.newFixedThreadPool(PARALLEL_NUMBER, - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Test-Thread-%d").build()); - createConnection(); - } - - @After - public void tearDown() throws Exception { - pool.shutdownNow(); - closeConnection(); - } - - private void createConnection() throws Exception { - Connection connection = DriverManager.getConnection(miniHS2.getJdbcURL(dbName), - System.getProperty("user.name"), "bar"); - Statement statement = connection.createStatement(); - localConnection.set(connection); - localStatement.set(statement); - statement.execute("USE " + dbName); - } - - private void closeConnection() throws SQLException { - if (localStatement.get() != null) { - localStatement.get().close(); - } - - if (localConnection.get() != null) { - localConnection.get().close(); - } - } - - @AfterClass - public static void afterTest() throws Exception { - if (miniHS2 != null && miniHS2.isStarted()) { - miniHS2.stop(); - } - } - - /** - * Run nonSpark query - * - * @throws Exception - */ - @Test - public void testNonSparkQuery() throws Exception { - String tableName = "kvTable1"; - setupTable(tableName); - Callable runNonSparkQuery = getNonSparkQueryCallable(tableName); - runInParallel(runNonSparkQuery); - dropTable(tableName); - } - - /** - * Run spark query - * - * @throws Exception - */ - @Test - public void testSparkQuery() throws Exception { - String tableName = "kvTable2"; - setupTable(tableName); - Callable runSparkQuery = getSparkQueryCallable(tableName); - runInParallel(runSparkQuery); - dropTable(tableName); - } - - private void runInParallel(Callable runNonSparkQuery) throws InterruptedException, ExecutionException { - List futureList = new LinkedList(); - for (int i = 0; i < PARALLEL_NUMBER; i++) { - Future future = pool.submit(runNonSparkQuery); - futureList.add(future); - } - - for (Future future : futureList) { - future.get(); - } - } - - private Callable getNonSparkQueryCallable(final String tableName) { - return new Callable() { - @Override - public Void call() throws Exception { - String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tableName; - testKvQuery(queryStr, resultVal); - return null; - } - }; - } - - private Callable getSparkQueryCallable(final String tableName) { - return new Callable() { - @Override - public Void call() throws Exception { - String resultVal = "val_238"; - String queryStr = "SELECT * FROM " + tableName + - " where value = '" + resultVal + "'"; - testKvQuery(queryStr, resultVal); - return null; - } - }; - } - - private void testKvQuery(String queryStr, String resultVal) - throws Exception { - createConnection(); - verifyResult(queryStr, resultVal, 2); - closeConnection(); - } - - // create table and load kv1.txt - private void setupTable(String tableName) throws SQLException { - Statement statement = localStatement.get(); - // create table - statement.execute("CREATE TABLE " + tableName - + " (under_col INT COMMENT 'the under column', value STRING)" - + " COMMENT ' test table'"); - - // load data - statement.execute("LOAD DATA LOCAL INPATH '" - + dataFilePath.toString() + "' INTO TABLE " + tableName); - } - - private void dropTable(String tableName) throws SQLException { - localStatement.get().execute("DROP TABLE " + tableName); - } - - // run given query and validate expected result - private void verifyResult(String queryStr, String expString, int colPos) - throws SQLException { - ResultSet res = localStatement.get().executeQuery(queryStr); - assertTrue(res.next()); - assertEquals(expString, res.getString(colPos)); - res.close(); - } -} diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java index cb005bf0327..4cbf35708cd 100644 --- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java +++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestRestrictedList.java @@ -87,26 +87,15 @@ public static void startServices() throws Exception { addToExpectedRestrictedMap("hive.server2.authentication.ldap.userMembershipKey"); addToExpectedRestrictedMap("hive.server2.authentication.ldap.groupClassKey"); addToExpectedRestrictedMap("hive.server2.authentication.ldap.customLDAPQuery"); - addToExpectedRestrictedMap("hive.spark.client.channel.log.level"); - addToExpectedRestrictedMap("hive.spark.client.secret.bits"); - addToExpectedRestrictedMap("hive.spark.client.rpc.server.address"); - addToExpectedRestrictedMap("hive.spark.client.rpc.server.port"); - addToExpectedRestrictedMap("hive.spark.client.rpc.sasl.mechanisms"); addToExpectedRestrictedMap("bonecp.test"); addToExpectedRestrictedMap("hive.druid.broker.address.default"); addToExpectedRestrictedMap("hive.druid.coordinator.address.default"); addToExpectedRestrictedMap("hikari.test"); addToExpectedRestrictedMap("hadoop.bin.path"); addToExpectedRestrictedMap("yarn.bin.path"); - addToExpectedRestrictedMap("hive.spark.client.connect.timeout"); - addToExpectedRestrictedMap("hive.spark.client.server.connect.timeout"); - addToExpectedRestrictedMap("hive.spark.client.rpc.max.size"); - addToExpectedRestrictedMap("hive.spark.client.rpc.threads"); addToExpectedRestrictedMap("_hive.local.session.path"); addToExpectedRestrictedMap("_hive.tmp_table_space"); addToExpectedRestrictedMap("_hive.hdfs.session.path"); - addToExpectedRestrictedMap("hive.spark.client.rpc.server.address"); - addToExpectedRestrictedMap("spark.home"); addToExpectedRestrictedMap("hive.privilege.synchronizer.interval"); } diff --git a/itests/pom.xml b/itests/pom.xml index a4dc8a9597f..920eb48f194 100644 --- a/itests/pom.xml +++ b/itests/pom.xml @@ -50,69 +50,6 @@ - - spark-test - - - !skipSparkTests - - - - qtest-spark - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - download-spark - generate-sources - - run - - - - - set -x - /bin/pwd - BASE_DIR=./target - HIVE_ROOT=$BASE_DIR/../../../ - DOWNLOAD_DIR=./../thirdparty - download() { - url=$1; - finalName=$2 - tarName=$(basename $url) - rm -rf $BASE_DIR/$finalName - if [[ ! -f $DOWNLOAD_DIR/$tarName ]] - then - curl -Sso $DOWNLOAD_DIR/$tarName $url - else - local md5File="$tarName".md5sum - curl -Sso $DOWNLOAD_DIR/$md5File "$url".md5sum - cd $DOWNLOAD_DIR - if type md5sum >/dev/null && ! md5sum -c $md5File; then - curl -Sso $DOWNLOAD_DIR/$tarName $url || return 1 - fi - - cd - - fi - tar -zxf $DOWNLOAD_DIR/$tarName -C $BASE_DIR - mv $BASE_DIR/spark-${spark.version}-bin-hadoop3-beta1-without-hive $BASE_DIR/$finalName - } - mkdir -p $DOWNLOAD_DIR - download "http://d3jw87u4immizc.cloudfront.net/spark-tarball/spark-${spark.version}-bin-hadoop3-beta1-without-hive.tgz" "spark" - cp -f $HIVE_ROOT/data/conf/spark/log4j2.properties $BASE_DIR/spark/conf/ - - - - - - - - - diff --git a/itests/qtest-spark/pom.xml b/itests/qtest-spark/pom.xml deleted file mode 100644 index 73e4e9435df..00000000000 --- a/itests/qtest-spark/pom.xml +++ /dev/null @@ -1,474 +0,0 @@ - - - - 4.0.0 - - - org.apache.hive - hive-it - 3.1.3 - ../pom.xml - - - hive-it-qfile-spark - jar - Hive Integration - QFile Spark Tests - - - ../.. - - OFF - - - 2.21 - - - false - false - ${hadoop.version} - -mkdir -p - ${basedir}/${hive.path.to.root}/itests/qtest-spark/target/spark - - - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - - - org.eclipse.jetty - jetty-util - ${jetty.version} - test - - - org.eclipse.jetty - jetty-security - ${jetty.version} - test - - - org.eclipse.jetty - jetty-plus - ${jetty.version} - test - - - org.eclipse.jetty - jetty-server - ${jetty.version} - test - - - org.eclipse.jetty - jetty-servlet - ${jetty.version} - test - - - com.esotericsoftware.kryo - kryo - ${spark.kryo.version} - test - - - org.mockito - mockito-all - ${mockito-all.version} - test - - - - - org.apache.hive - hive-common - ${project.version} - test - - - org.apache.hive - hive-contrib - ${project.version} - test - - - org.apache.hive - hive-standalone-metastore - ${project.version} - test - - - org.apache.hive - hive-standalone-metastore - ${project.version} - tests - test - - - org.apache.hive - hive-it-custom-serde - ${project.version} - test - - - org.apache.hive - hive-it-util - ${project.version} - - - org.apache.hive - hive-it-druid - - - test - - - org.apache.hive - hive-serde - ${project.version} - test - - - org.apache.hive - hive-exec - ${project.version} - test - - - - - com.sun.jersey - jersey-servlet - ${jersey.version} - test - - - org.apache.hadoop - hadoop-archives - ${hadoop.version} - test - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-common - ${hadoop.version} - tests - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - tests - test - - - org.apache.hadoop - hadoop-hdfs - ${hadoop.version} - test - - - org.apache.hadoop - hadoop-mapreduce-client-jobclient - ${hadoop.version} - tests - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-mapreduce-client-hs - ${hadoop.version} - test - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - - - org.apache.hadoop - hadoop-mapreduce-client-core - ${hadoop.version} - test - - - org.apache.hadoop - hadoop-yarn-server-tests - ${hadoop.version} - test - tests - - - org.apache.hadoop - hadoop-yarn-client - ${hadoop.version} - test - - - org.apache.hbase - hbase-common - ${hbase.version} - test - - - org.apache.hbase - hbase-common - ${hbase.version} - test - tests - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - test - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - test - tests - - - org.apache.hbase - hbase-hadoop2-compat - ${hbase.version} - test - - - org.apache.hbase - hbase-hadoop2-compat - ${hbase.version} - test - tests - - - org.apache.hbase - hbase-server - ${hbase.version} - test - - - org.apache.hbase - hbase-server - ${hbase.version} - tests - test - - - org.apache.hbase - hbase-mapreduce - ${hbase.version} - test - - - junit - junit - ${junit.version} - test - - - - - - - org.codehaus.mojo - properties-maven-plugin - 1.0-alpha-2 - - - initialize - - read-project-properties - - - - ${basedir}/../src/test/resources/testconfiguration.properties - - - - - - - org.apache.maven.plugins - maven-antrun-plugin - - - download-spark - generate-sources - - run - - - - - - - - - - - generate-tests-sources - generate-test-sources - - - - - - - - - - - - - - - run - - - - - - org.codehaus.mojo - build-helper-maven-plugin - ${maven.build-helper.plugin.version} - - - add-test-sources - generate-test-sources - - add-test-source - - - - target/generated-test-sources/java - - - - - - - - - - qsplits - - - - org.apache.maven.plugins - maven-antrun-plugin - - - generate-split-tests - generate-sources - - - - - - - - - - - run - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-test-source - generate-sources - - add-test-source - - - - target/generated-test-sources - - - - - - - - - - diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestLocalSparkCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestLocalSparkCliDriver.java deleted file mode 100644 index 41da5fe84fd..00000000000 --- a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestLocalSparkCliDriver.java +++ /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.hadoop.hive.cli; - -import java.io.File; -import java.util.List; - -import org.apache.hadoop.hive.cli.control.CliAdapter; -import org.apache.hadoop.hive.cli.control.CliConfigs; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestLocalSparkCliDriver { - - static CliAdapter adapter = new CliConfigs.LocalSparkCliConfig().getCliAdapter(); - - @Parameters(name = "{0}") - public static List getParameters() throws Exception { - return adapter.getParameters(); - } - - @ClassRule - public static TestRule cliClassRule = adapter.buildClassRule(); - - @Rule - public TestRule cliTestRule = adapter.buildTestRule(); - - private String name; - private File qfile; - - public TestLocalSparkCliDriver(String name, File qfile) { - this.name = name; - this.qfile = qfile; - } - - @Test - public void testCliDriver() throws Exception { - adapter.runTest(name, qfile); - } -} diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java deleted file mode 100644 index c19d4dbe5ae..00000000000 --- a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestMiniSparkOnYarnCliDriver.java +++ /dev/null @@ -1,66 +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.hadoop.hive.cli; - -import java.io.File; -import java.util.List; - -import org.apache.hadoop.hive.cli.control.CliAdapter; -import org.apache.hadoop.hive.cli.control.CliConfigs; -import org.apache.hadoop.hive.cli.control.SplitSupport; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestMiniSparkOnYarnCliDriver { - - private static final int N_SPLITS = 5; - - static CliAdapter adapter = new CliConfigs.SparkOnYarnCliConfig().getCliAdapter(); - - @Parameters(name = "{0}") - public static List getParameters() throws Exception { - return SplitSupport.process(adapter.getParameters(), TestMiniSparkOnYarnCliDriver.class, N_SPLITS); - } - - @ClassRule - public static TestRule cliClassRule = adapter.buildClassRule(); - - @Rule - public TestRule cliTestRule = adapter.buildTestRule(); - - private String name; - private File qfile; - - public TestMiniSparkOnYarnCliDriver(String name, File qfile) { - this.name = name; - this.qfile = qfile; - } - - @Test - public void testCliDriver() throws Exception { - adapter.runTest(name, qfile); - } - -} diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java deleted file mode 100644 index bf200b67b50..00000000000 --- a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkCliDriver.java +++ /dev/null @@ -1,66 +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.hadoop.hive.cli; - -import java.io.File; -import java.util.List; - -import org.apache.hadoop.hive.cli.control.CliAdapter; -import org.apache.hadoop.hive.cli.control.CliConfigs; -import org.apache.hadoop.hive.cli.control.SplitSupport; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestSparkCliDriver { - - private static final int N_SPLITS = 11; - - static CliAdapter adapter = new CliConfigs.SparkCliConfig().getCliAdapter(); - - @Parameters(name = "{0}") - public static List getParameters() throws Exception { - return SplitSupport.process(adapter.getParameters(), TestSparkCliDriver.class, N_SPLITS); - } - - @ClassRule - public static TestRule cliClassRule = adapter.buildClassRule(); - - @Rule - public TestRule cliTestRule = adapter.buildTestRule(); - - private String name; - private File qfile; - - public TestSparkCliDriver(String name, File qfile) { - this.name = name; - this.qfile = qfile; - } - - @Test - public void testCliDriver() throws Exception { - adapter.runTest(name, qfile); - } - -} diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java deleted file mode 100644 index 9f8db15f7de..00000000000 --- a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkNegativeCliDriver.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.cli; - -import java.io.File; -import java.util.List; - -import org.apache.hadoop.hive.cli.control.CliAdapter; -import org.apache.hadoop.hive.cli.control.CliConfigs; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestSparkNegativeCliDriver { - - static CliAdapter adapter = new CliConfigs.SparkNegativeCliConfig().getCliAdapter(); - - @Parameters(name = "{0}") - public static List getParameters() throws Exception { - return adapter.getParameters(); - } - - @ClassRule - public static TestRule cliClassRule = adapter.buildClassRule(); - - @Rule - public TestRule cliTestRule = adapter.buildTestRule(); - - private String name; - private File qfile; - - public TestSparkNegativeCliDriver(String name, File qfile) { - this.name = name; - this.qfile = qfile; - } - - @Test - public void testCliDriver() throws Exception { - adapter.runTest(name, qfile); - } - -} diff --git a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkPerfCliDriver.java b/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkPerfCliDriver.java deleted file mode 100644 index 49aeff1eaf7..00000000000 --- a/itests/qtest-spark/src/test/java/org/apache/hadoop/hive/cli/TestSparkPerfCliDriver.java +++ /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.hadoop.hive.cli; - -import java.io.File; -import java.util.List; - -import org.apache.hadoop.hive.cli.control.CliAdapter; -import org.apache.hadoop.hive.cli.control.CliConfigs; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TestRule; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; - -@RunWith(Parameterized.class) -public class TestSparkPerfCliDriver { - - static CliAdapter adapter = new CliConfigs.SparkPerfCliConfig().getCliAdapter(); - - @Parameters(name = "{0}") - public static List getParameters() throws Exception { - return adapter.getParameters(); - } - - @ClassRule - public static TestRule cliClassRule = adapter.buildClassRule(); - - @Rule - public TestRule cliTestRule = adapter.buildTestRule(); - - private String name; - private File qfile; - - public TestSparkPerfCliDriver(String name, File qfile) { - this.name = name; - this.qfile = qfile; - } - - @Test - public void testCliDriver() throws Exception { - adapter.runTest(name, qfile); - } - -} diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java index 91121aee84d..77e0b037e87 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java @@ -322,28 +322,6 @@ public TezPerfCliConfig() { } } - public static class SparkPerfCliConfig extends AbstractCliConfig { - public SparkPerfCliConfig() { - super(CorePerfCliDriver.class); - try { - setQueryDir("ql/src/test/queries/clientpositive/perf"); - - excludesFrom(testConfigProps, "spark.perf.disabled.query.files"); - - setResultsDir("ql/src/test/results/clientpositive/perf/spark"); - setLogDir("itests/qtest/target/qfile-results/clientpositive/spark"); - - setInitScript("q_perf_test_init.sql"); - setCleanupScript("q_perf_test_cleanup.sql"); - - setHiveConfDir("data/conf/perf-reg/spark"); - setClusterType(MiniClusterType.spark); - } catch (Exception e) { - throw new RuntimeException("can't construct cliconfig", e); - } - } - } - public static class CompareCliConfig extends AbstractCliConfig { public CompareCliConfig() { super(CoreCompareCliDriver.class); @@ -533,96 +511,6 @@ public AccumuloCliConfig() { } } - public static class SparkCliConfig extends AbstractCliConfig { - public SparkCliConfig() { - super(CoreCliDriver.class); - try { - setQueryDir("ql/src/test/queries/clientpositive"); - - includesFrom(testConfigProps, "spark.query.files"); - includesFrom(testConfigProps, "spark.only.query.files"); - - setResultsDir("ql/src/test/results/clientpositive/spark"); - setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark"); - - setInitScript("q_test_init.sql"); - setCleanupScript("q_test_cleanup.sql"); - - setHiveConfDir("data/conf/spark/standalone"); - setClusterType(MiniClusterType.spark); - } catch (Exception e) { - throw new RuntimeException("can't construct cliconfig", e); - } - } - } - - public static class LocalSparkCliConfig extends AbstractCliConfig { - public LocalSparkCliConfig() { - super(CoreCliDriver.class); - try { - setQueryDir("ql/src/test/queries/clientpositive"); - - includesFrom(testConfigProps, "localSpark.only.query.files"); - - setResultsDir("ql/src/test/results/clientpositive/spark"); - setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark"); - - setInitScript("q_test_init.sql"); - setCleanupScript("q_test_cleanup.sql"); - - setHiveConfDir("data/conf/spark/local"); - setClusterType(MiniClusterType.spark); - } catch (Exception e) { - throw new RuntimeException("can't construct cliconfig", e); - } - } - } - - public static class SparkOnYarnCliConfig extends AbstractCliConfig { - public SparkOnYarnCliConfig() { - super(CoreCliDriver.class); - try { - setQueryDir("ql/src/test/queries/clientpositive"); - - includesFrom(testConfigProps, "miniSparkOnYarn.query.files"); - includesFrom(testConfigProps, "miniSparkOnYarn.only.query.files"); - - setResultsDir("ql/src/test/results/clientpositive/spark"); - setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark"); - - setInitScript("q_test_init.sql"); - setCleanupScript("q_test_cleanup.sql"); - - setHiveConfDir("data/conf/spark/yarn-client"); - setClusterType(MiniClusterType.miniSparkOnYarn); - } catch (Exception e) { - throw new RuntimeException("can't construct cliconfig", e); - } - } - } - - public static class SparkNegativeCliConfig extends AbstractCliConfig { - public SparkNegativeCliConfig() { - super(CoreNegativeCliDriver.class); - try { - setQueryDir("ql/src/test/queries/clientnegative"); - - includesFrom(testConfigProps, "spark.query.negative.files"); - - setResultsDir("ql/src/test/results/clientnegative/spark"); - setLogDir("itests/qtest-spark/target/qfile-results/clientnegative/spark"); - - setInitScript("q_test_init.sql"); - setCleanupScript("q_test_cleanup.sql"); - - setHiveConfDir("data/conf/spark/standalone"); - setClusterType(MiniClusterType.spark); - } catch (Exception e) { - throw new RuntimeException("can't construct cliconfig", e); - } - } - } - public static class BlobstoreCliConfig extends AbstractCliConfig { public BlobstoreCliConfig() { super(CoreBlobstoreCliDriver.class); diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java index ebb4ad8e782..14b9ef08a91 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -91,8 +91,6 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; import org.apache.hadoop.hive.ql.exec.tez.TezSessionState; import org.apache.hadoop.hive.ql.lockmgr.zookeeper.CuratorFrameworkSingleton; import org.apache.hadoop.hive.ql.lockmgr.zookeeper.ZooKeeperHiveLockManager; @@ -196,7 +194,6 @@ public class QTestUtil { private MiniLlapCluster llapCluster = null; private String hadoopVer = null; private QTestSetup setup = null; - private SparkSession sparkSession = null; private boolean isSessionStateStarted = false; private static final String javaVersion = getJavaVersion(); private QOutProcessor qOutProcessor; @@ -485,8 +482,7 @@ private void createRemoteDirs() { private enum CoreClusterType { MR, - TEZ, - SPARK + TEZ } public enum FsType { @@ -500,8 +496,6 @@ public enum MiniClusterType { mr(CoreClusterType.MR, FsType.hdfs), tez(CoreClusterType.TEZ, FsType.hdfs), tez_local(CoreClusterType.TEZ, FsType.local), - spark(CoreClusterType.SPARK, FsType.local), - miniSparkOnYarn(CoreClusterType.SPARK, FsType.hdfs), llap(CoreClusterType.TEZ, FsType.hdfs), llap_local(CoreClusterType.TEZ, FsType.local), none(CoreClusterType.MR, FsType.local), @@ -534,10 +528,6 @@ public static MiniClusterType valueForString(String type) { return tez; } else if (type.equals("tez_local")) { return tez_local; - } else if (type.equals("spark")) { - return spark; - } else if (type.equals("miniSparkOnYarn")) { - return miniSparkOnYarn; } else if (type.equals("llap")) { return llap; } else if (type.equals("llap_local")) { @@ -747,8 +737,6 @@ private void setupMiniCluster(HadoopShims shims, String confDir) throws mr = shims.getMiniTezCluster(conf, numTrackers, uriString, isLlapIoEnabled); } - } else if (clusterType == MiniClusterType.miniSparkOnYarn) { - mr = shims.getMiniSparkCluster(conf, 2, uriString, 1); } else if (clusterType == MiniClusterType.mr) { mr = shims.getMiniMrCluster(conf, 2, uriString, 1); } @@ -815,15 +803,6 @@ public void shutdown() throws Exception { kafkaCluster = null; } setup.tearDown(); - if (sparkSession != null) { - try { - SparkSessionManagerImpl.getInstance().closeSession(sparkSession); - } catch (Exception ex) { - LOG.error("Error closing spark session.", ex); - } finally { - sparkSession = null; - } - } if (mr != null) { mr.shutdown(); mr = null; @@ -1411,13 +1390,6 @@ private void restartSessions(boolean canReuseSession, CliSessionState ss, Sessio ss.setTezSession(tezSessionState); oldSs.close(); } - - if (oldSs != null && clusterType.getCoreClusterType() == CoreClusterType.SPARK) { - sparkSession = oldSs.getSparkSession(); - ss.setSparkSession(sparkSession); - oldSs.setSparkSession(null); - oldSs.close(); - } } private CliSessionState startSessionState(boolean canReuseSession) throws IOException { diff --git a/packaging/pom.xml b/packaging/pom.xml index 3b9ee34679c..8f173de795a 100644 --- a/packaging/pom.xml +++ b/packaging/pom.xml @@ -302,11 +302,14 @@ hive-webhcat-java-client ${project.version} + + diff --git a/packaging/src/main/assembly/src.xml b/packaging/src/main/assembly/src.xml index 58a4f1cad61..0175d2b9465 100644 --- a/packaging/src/main/assembly/src.xml +++ b/packaging/src/main/assembly/src.xml @@ -95,14 +95,13 @@ service-rpc/**/* service/**/* shims/**/* - spark-client/**/* storage-api/**/* standalone-metastore/**/* streaming/**/* testutils/**/* upgrade-acid/**/* vector-code-gen/**/* - kryo-registrator/**/* + / diff --git a/pom.xml b/pom.xml index 021deb249da..bf4bd418d71 100644 --- a/pom.xml +++ b/pom.xml @@ -56,8 +56,7 @@ llap-tez llap-server shims - spark-client - kryo-registrator + testutils packaging standalone-metastore @@ -93,7 +92,6 @@ - 1.0b3 @@ -172,8 +170,6 @@ 1.1 9.3.25.v20180904 1.19 - - 2.22.2 2.12 2.0.2 2.9.9 @@ -201,7 +197,7 @@ 2.6.1.mr3 0.9.1.mr3.1.0 2.2.0 - 2.3.0 + 2.3.0 2.11 2.11.12 1.1 @@ -597,11 +593,12 @@ jackson-databind ${jackson.version} - + + com.fasterxml.jackson.dataformat jackson-dataformat-smile @@ -934,17 +931,6 @@ scala-library ${scala.version} - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - - - org.apache.hadoop - hadoop-core - - - org.jamon jamon-runtime @@ -1254,7 +1240,6 @@ **/ql/exec/vector/udf/generic/*.java **/TestHiveServer2Concurrency.java ${test.excludes.additional} - ${skip.spark.files} true false @@ -1270,10 +1255,6 @@ en_US.UTF-8 ${test.conf.dir}:${basedir}/${hive.path.to.root}/conf ${test.hive.hadoop.classpath} - ${spark.home}/lib/spark-assembly-${spark.version}-hadoop2.4.0.jar:${test.hive.hadoop.classpath} - -Dorg.xerial.snappy.tempdir=/tmp -Dorg.xerial.snappy.lib.name=libsnappyjava.jnilib - 2.11 - ${spark.home} ${env.PATH}${test.extra.path} @@ -1296,7 +1277,6 @@ true ${test.tmp.dir} - ${spark.home} ${test.tmp.dir} @@ -1311,8 +1291,6 @@ src,src1,srcbucket,srcbucket2,src_json,src_thrift,src_sequencefile,srcpart,alltypesorc,alltypesparquet,src_hbase,cbo_t1,cbo_t2,cbo_t3,src_cbo,part,lineitem ${test.conf.dir}/krb5.conf - - ${antlr.version} ${qfile} ${initScript} ${clustermode} @@ -1559,19 +1537,6 @@ file:/ - - spark-test - - - !skipSparkTests - - - - - **/ql/exec/spark/session/TestSparkSessionManagerImpl.java,**/TestMultiSessionsHS2WithLocalClusterSpark.java,**/TestJdbcWithLocalClusterSpark.java - - - itests diff --git a/ql/pom.xml b/ql/pom.xml index c2a2067b857..c52ab105a75 100644 --- a/ql/pom.xml +++ b/ql/pom.xml @@ -83,9 +83,9 @@ ${project.version} - org.apache.hive - hive-spark-client - ${project.version} + org.scala-lang + scala-library + ${scala.version} @@ -706,51 +706,11 @@ ${mr3.version} provided - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - true - - - com.esotericsoftware.kryo - kryo - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - org.glassfish.jersey.containers - * - - - org.glassfish.jersey.core - * - - - com.sun.jersey jersey-servlet test - - org.glassfish.jersey.core - jersey-server - ${glassfish.jersey.version} - test - - - org.glassfish.jersey.containers - jersey-container-servlet-core - ${glassfish.jersey.version} - test - org.hamcrest hamcrest-all @@ -969,7 +929,6 @@ org.codehaus.jackson:jackson-mapper-asl com.google.guava:guava net.sf.opencsv:opencsv - org.apache.hive:hive-spark-client org.apache.hive:hive-storage-api org.apache.orc:orc-core org.apache.orc:orc-shims diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java index 6509fe35425..163799112e5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java @@ -2292,8 +2292,7 @@ private void execute() throws CommandProcessorResponse { setQueryDisplays(plan.getRootTasks()); int mrJobs = Utilities.getMRTasks(plan.getRootTasks()).size(); - int jobs = mrJobs + Utilities.getTezTasks(plan.getRootTasks()).size() - + Utilities.getSparkTasks(plan.getRootTasks()).size(); + int jobs = mrJobs + Utilities.getTezTasks(plan.getRootTasks()).size(); if (jobs > 0) { logMrWarning(mrJobs); console.printInfo("Query ID = " + queryId); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java index 4419ae19aed..172a54665fc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java @@ -498,10 +498,6 @@ public enum ErrorMsg { FILE_NOT_FOUND(20012, "File not found: {0}", "64000", true), WRONG_FILE_FORMAT(20013, "Wrong file format. Please check the file's format.", "64000", true), - SPARK_CREATE_CLIENT_INVALID_QUEUE(20014, "Spark job is submitted to an invalid queue: {0}." - + " Please fix and try again.", true), - SPARK_RUNTIME_OOM(20015, "Spark job failed because of out of memory."), - //if the error message is changed for REPL_EVENTS_MISSING_IN_METASTORE, then need modification in getNextNotification //method in HiveMetaStoreClient REPL_EVENTS_MISSING_IN_METASTORE(20016, "Notification events are missing in the meta store."), @@ -575,36 +571,7 @@ public enum ErrorMsg { CONCATENATE_UNSUPPORTED_TABLE_TRANSACTIONAL(30035, "Concatenate/Merge can not be performed on transactional tables"), - SPARK_GET_JOB_INFO_TIMEOUT(30036, - "Spark job timed out after {0} seconds while getting job info", true), - SPARK_JOB_MONITOR_TIMEOUT(30037, "Job hasn''t been submitted after {0}s." + - " Aborting it.\nPossible reasons include network issues, " + - "errors in remote driver or the cluster has no available resources, etc.\n" + - "Please check YARN or Spark driver''s logs for further information.\n" + - "The timeout is controlled by " + HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT + ".", true), - - // Various errors when creating Spark client - SPARK_CREATE_CLIENT_TIMEOUT(30038, - "Timed out while creating Spark client for session {0}.", true), - SPARK_CREATE_CLIENT_QUEUE_FULL(30039, - "Failed to create Spark client because job queue is full: {0}.", true), - SPARK_CREATE_CLIENT_INTERRUPTED(30040, - "Interrupted while creating Spark client for session {0}", true), - SPARK_CREATE_CLIENT_ERROR(30041, - "Failed to create Spark client for Spark session {0}", true), - SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST(30042, - "Failed to create Spark client due to invalid resource request: {0}", true), - SPARK_CREATE_CLIENT_CLOSED_SESSION(30043, - "Cannot create Spark client on a closed session {0}", true), - - SPARK_JOB_INTERRUPTED(30044, "Spark job was interrupted while executing"), - - REPL_FILE_SYSTEM_OPERATION_RETRY(30045, "Replication file system operation retry expired."), - - //========================== 40000 range starts here ========================// - - SPARK_JOB_RUNTIME_ERROR(40001, - "Spark job failed during runtime. Please check stacktrace for the root cause.") + REPL_FILE_SYSTEM_OPERATION_RETRY(30045, "Replication file system operation retry expired.") ; private int errorCode; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java index 75d3388ad74..3bbce23ce6a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExplainTask.java @@ -62,7 +62,6 @@ import org.apache.hadoop.hive.ql.plan.HiveOperation; import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TezWork; import org.apache.hadoop.hive.ql.plan.api.StageType; import org.apache.hadoop.hive.ql.security.authorization.AuthorizationFactory; @@ -573,36 +572,6 @@ else if (ent.getValue() instanceof List) { json.accumulate(ent.getKey().toString(), jsonDep); } } - } else if (ent.getValue() != null && !((List) ent.getValue()).isEmpty() - && ((List) ent.getValue()).get(0) != null && - ((List) ent.getValue()).get(0) instanceof SparkWork.Dependency) { - if (out != null) { - boolean isFirst = true; - for (SparkWork.Dependency dep: (List) ent.getValue()) { - if (!isFirst) { - out.print(", "); - } else { - out.print("<- "); - isFirst = false; - } - out.print(dep.getName()); - out.print(" ("); - out.print(dep.getShuffleType()); - out.print(", "); - out.print(dep.getNumPartitions()); - out.print(")"); - } - out.println(); - } - if (jsonOutput) { - for (SparkWork.Dependency dep: (List) ent.getValue()) { - JSONObject jsonDep = new JSONObject(new LinkedHashMap<>()); - jsonDep.put("parent", dep.getName()); - jsonDep.put("type", dep.getShuffleType()); - jsonDep.put("partitions", dep.getNumPartitions()); - json.accumulate(ent.getKey().toString(), jsonDep); - } - } } else { if (out != null) { out.print(ent.getValue().toString()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java index 9ea81b47229..17ca8e8898a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe; import org.apache.hadoop.hive.ql.exec.persistence.ObjectContainer; import org.apache.hadoop.hive.ql.exec.persistence.UnwrapRowContainer; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.exec.tez.LlapObjectCache; import org.apache.hadoop.hive.ql.exec.tez.LlapObjectSubCache; import org.apache.hadoop.hive.ql.io.HiveKey; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java index b61d37ed1d5..3a7a4afdad2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -36,15 +36,11 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorReduceSinkOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSparkPartitionPruningSinkOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorTopNKeyOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkCommonOperator; import org.apache.hadoop.hive.ql.exec.vector.ptf.VectorPTFOperator; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc; import org.apache.hadoop.hive.ql.plan.AbstractVectorDesc; import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; @@ -75,7 +71,6 @@ import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; import org.apache.hadoop.hive.ql.plan.ScriptDesc; import org.apache.hadoop.hive.ql.plan.SelectDesc; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.TopNKeyDesc; import org.apache.hadoop.hive.ql.plan.UDTFDesc; @@ -117,13 +112,11 @@ public final class OperatorFactory { opvec.put(LateralViewForwardDesc.class, LateralViewForwardOperator.class); opvec.put(HashTableDummyDesc.class, HashTableDummyOperator.class); opvec.put(HashTableSinkDesc.class, HashTableSinkOperator.class); - opvec.put(SparkHashTableSinkDesc.class, SparkHashTableSinkOperator.class); opvec.put(DummyStoreDesc.class, DummyStoreOperator.class); opvec.put(DemuxDesc.class, DemuxOperator.class); opvec.put(MuxDesc.class, MuxOperator.class); opvec.put(AppMasterEventDesc.class, AppMasterEventOperator.class); opvec.put(DynamicPruningEventDesc.class, AppMasterEventOperator.class); - opvec.put(SparkPartitionPruningSinkDesc.class, SparkPartitionPruningSinkOperator.class); opvec.put(RCFileMergeDesc.class, RCFileMergeOperator.class); opvec.put(OrcFileMergeDesc.class, OrcFileMergeOperator.class); opvec.put(CommonMergeJoinDesc.class, CommonMergeJoinOperator.class); @@ -134,8 +127,6 @@ public final class OperatorFactory { static { vectorOpvec.put(AppMasterEventDesc.class, VectorAppMasterEventOperator.class); vectorOpvec.put(DynamicPruningEventDesc.class, VectorAppMasterEventOperator.class); - vectorOpvec.put( - SparkPartitionPruningSinkDesc.class, VectorSparkPartitionPruningSinkOperator.class); vectorOpvec.put(SelectDesc.class, VectorSelectOperator.class); vectorOpvec.put(GroupByDesc.class, VectorGroupByOperator.class); vectorOpvec.put(MapJoinDesc.class, VectorMapJoinOperator.class); @@ -145,7 +136,6 @@ public final class OperatorFactory { vectorOpvec.put(FilterDesc.class, VectorFilterOperator.class); vectorOpvec.put(LimitDesc.class, VectorLimitOperator.class); vectorOpvec.put(PTFDesc.class, VectorPTFOperator.class); - vectorOpvec.put(SparkHashTableSinkDesc.class, VectorSparkHashTableSinkOperator.class); vectorOpvec.put(TopNKeyDesc.class, VectorTopNKeyOperator.class); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java index c6333495d57..ec92c610323 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java @@ -29,7 +29,6 @@ import java.util.Stack; import org.apache.hadoop.hive.ql.exec.NodeUtils.Function; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.MapJoinDesc; import org.apache.hadoop.hive.ql.plan.MapWork; @@ -354,25 +353,6 @@ public static void findRoots(Operator op, Collection> roots) { } } - /** - * Remove the branch that contains the specified operator. Do nothing if there's no branching, - * i.e. all the upstream operators have only one child. - */ - public static void removeBranch(SparkPartitionPruningSinkOperator op) { - Operator child = op; - Operator curr = op; - - while (curr.getChildOperators().size() <= 1) { - child = curr; - if (curr.getParentOperators() == null || curr.getParentOperators().isEmpty()) { - return; - } - curr = curr.getParentOperators().get(0); - } - - curr.removeChild(child); - } - /** * Remove operator from the tree, disconnecting it from its * parents and children. @@ -401,20 +381,6 @@ public static String getOpNamePretty(Operator op) { return op.toString(); } - /** - * Return true if contain branch otherwise return false - */ - public static boolean isInBranch(SparkPartitionPruningSinkOperator op) { - Operator curr = op; - while (curr.getChildOperators().size() <= 1) { - if (curr.getParentOperators() == null || curr.getParentOperators().isEmpty()) { - return false; - } - curr = curr.getParentOperators().get(0); - } - return true; - } - public static Set> getOp(BaseWork work, Class clazz) { Set> ops = new HashSet>(); if (work instanceof MapWork) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java index 166e6ea9588..1fd56a042ab 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java @@ -55,9 +55,6 @@ import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkEnv; -import org.apache.spark.SparkFiles; /** * ScriptOperator. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java index e03429bc37f..0422677e9ad 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java @@ -53,8 +53,6 @@ import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.serde2.Serializer; @@ -252,8 +250,6 @@ public Kryo create() { kryo.register(SequenceFileInputFormat.class); kryo.register(RCFileInputFormat.class); kryo.register(HiveSequenceFileOutputFormat.class); - kryo.register(SparkEdgeProperty.class); - kryo.register(SparkWork.class); kryo.register(Pair.class); kryo.register(MemoryMonitorInfo.class); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java deleted file mode 100644 index 78ae9a18e82..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java +++ /dev/null @@ -1,207 +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.hadoop.hive.ql.exec; - -import java.io.BufferedOutputStream; -import java.io.ObjectOutputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Set; - -import org.apache.commons.io.FileExistsException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinPersistableTableContainer; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.SparkBucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.api.OperatorType; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class SparkHashTableSinkOperator - extends TerminalOperator implements Serializable { - private static final long serialVersionUID = 1L; - private final String CLASS_NAME = this.getClass().getName(); - private final transient PerfLogger perfLogger = SessionState.getPerfLogger(); - protected static final Logger LOG = LoggerFactory.getLogger(SparkHashTableSinkOperator.class.getName()); - private static final String MAPRED_FILE_REPLICATION = "mapreduce.client.submit.file.replication"; - private static final int DEFAULT_REPLICATION = 10; - - private final HashTableSinkOperator htsOperator; - - private short numReplication; - - /** Kryo ctor. */ - protected SparkHashTableSinkOperator() { - super(); - htsOperator = null; // Kryo will set this; or so we hope. - } - - public SparkHashTableSinkOperator(CompilationOpContext ctx) { - super(ctx); - htsOperator = new HashTableSinkOperator(ctx); - } - - @Override - protected void initializeOp(Configuration hconf) throws HiveException { - super.initializeOp(hconf); - ObjectInspector[] inputOIs = new ObjectInspector[conf.getTagLength()]; - byte tag = conf.getTag(); - inputOIs[tag] = inputObjInspectors[0]; - conf.setTagOrder(new Byte[]{ tag }); - numReplication = (short) hconf.getInt(MAPRED_FILE_REPLICATION, DEFAULT_REPLICATION); - htsOperator.setConf(conf); - htsOperator.initialize(hconf, inputOIs); - } - - @Override - public void process(Object row, int tag) throws HiveException { - // Ignore the tag passed in, which should be 0, not what we want - htsOperator.process(row, conf.getTag()); - } - - @Override - public void closeOp(boolean abort) throws HiveException { - try { - MapJoinPersistableTableContainer[] mapJoinTables = htsOperator.mapJoinTables; - byte tag = conf.getTag(); - if (mapJoinTables == null || mapJoinTables.length < tag - || mapJoinTables[tag] == null) { - LOG.debug("mapJoinTable is null"); - } else if (abort) { - if (LOG.isDebugEnabled()) { - LOG.debug("Aborting, skip dumping side-table for tag: " + tag); - } - } else { - String method = PerfLogger.SPARK_FLUSH_HASHTABLE + getName(); - perfLogger.PerfLogBegin(CLASS_NAME, method); - try { - flushToFile(mapJoinTables[tag], tag); - } finally { - perfLogger.PerfLogEnd(CLASS_NAME, method); - } - } - super.closeOp(abort); - } catch (HiveException e) { - throw e; - } catch (Exception e) { - throw new HiveException(e); - } - } - - protected void flushToFile(MapJoinPersistableTableContainer tableContainer, - byte tag) throws Exception { - MapredLocalWork localWork = getExecContext().getLocalWork(); - BucketMapJoinContext mapJoinCtx = localWork.getBucketMapjoinContext(); - Path inputPath = getExecContext().getCurrentInputPath(); - String bigInputPath = null; - if (inputPath != null && mapJoinCtx != null) { - Set aliases = - ((SparkBucketMapJoinContext)mapJoinCtx).getPosToAliasMap().get((int)tag); - bigInputPath = mapJoinCtx.getMappingBigFile( - aliases.iterator().next(), inputPath.toString()); - } - - // get tmp file URI - Path tmpURI = localWork.getTmpHDFSPath(); - LOG.info("Temp URI for side table: " + tmpURI); - // get current bucket file name - String fileName = localWork.getBucketFileName(bigInputPath); - // get the tmp URI path; it will be a hdfs path if not local mode - String dumpFilePrefix = conf.getDumpFilePrefix(); - Path path = Utilities.generatePath(tmpURI, dumpFilePrefix, tag, fileName); - FileSystem fs = path.getFileSystem(htsOperator.getConfiguration()); - - fs.mkdirs(path); // Create the folder and its parents if not there - while (true) { - path = new Path(path, getOperatorId() - + "-" + Math.abs(Utilities.randGen.nextInt())); - try { - // This will guarantee file name uniqueness. - if (fs.createNewFile(path)) { - break; - } - } catch (FileExistsException e) { - // No problem, use a new name - } - } - - htsOperator.console.printInfo(Utilities.now() + "\tDump the side-table for tag: " + tag - + " with group count: " + tableContainer.size() + " into file: " + path); - try { - // get the hashtable file and path - OutputStream os = null; - ObjectOutputStream out = null; - MapJoinTableContainerSerDe mapJoinTableSerde = htsOperator.mapJoinTableSerdes[tag]; - try { - os = fs.create(path, numReplication); - out = new ObjectOutputStream(new BufferedOutputStream(os, 4096)); - mapJoinTableSerde.persist(out, tableContainer); - } finally { - if (out != null) { - out.close(); - } else if (os != null) { - os.close(); - } - } - - FileStatus status = fs.getFileStatus(path); - htsOperator.console.printInfo(Utilities.now() + "\tUploaded 1 File to: " + path - + " (" + status.getLen() + " bytes)"); - } catch (Exception e) { - // Failed to dump the side-table, remove the partial file - try { - fs.delete(path, false); - } catch (Exception ex) { - LOG.warn("Got exception in deleting partial side-table dump for tag: " - + tag + ", file " + path, ex); - } - throw e; - } - tableContainer.clear(); - } - - /** - * Implements the getName function for the Node Interface. - * - * @return the name of the operator - */ - @Override - public String getName() { - return SparkHashTableSinkOperator.getOperatorName(); - } - - public static String getOperatorName() { - return HashTableSinkOperator.getOperatorName(); - } - - @Override - public OperatorType getType() { - return OperatorType.HASHTABLESINK; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java index 47a802f4f7b..8781979fc0c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.exec.repl.ReplStateLogWork; import org.apache.hadoop.hive.ql.exec.repl.ReplLoadTask; import org.apache.hadoop.hive.ql.exec.repl.ReplLoadWork; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.io.merge.MergeFileTask; import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; @@ -51,7 +50,6 @@ import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.ReplCopyWork; import org.apache.hadoop.hive.ql.plan.ReplTxnWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TezWork; import com.google.common.annotations.VisibleForTesting; @@ -107,7 +105,6 @@ public TaskTuple(Class workClass, Class> taskClass) { taskvec.add(new TaskTuple(DependencyCollectionWork.class, DependencyCollectionTask.class)); taskvec.add(new TaskTuple(TezWork.class, TezTask.class)); - taskvec.add(new TaskTuple(SparkWork.class, SparkTask.class)); taskvec.add(new TaskTuple<>(ReplDumpWork.class, ReplDumpTask.class)); taskvec.add(new TaskTuple<>(ReplLoadWork.class, ReplLoadTask.class)); taskvec.add(new TaskTuple<>(ReplStateLogWork.class, ReplStateLogTask.class)); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 3e43d2eb234..9d2bccfabde 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -119,7 +119,6 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.hive.ql.exec.mr.ExecReducer; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.DagUtils; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.exec.util.DAGTraversal; @@ -2641,16 +2640,12 @@ public static List getTezTasks(List> tasks return getTasks(tasks, new TaskFilterFunction<>(TezTask.class)); } - public static List getSparkTasks(List> tasks) { - return getTasks(tasks, new TaskFilterFunction<>(SparkTask.class)); - } - public static List getMRTasks(List> tasks) { return getTasks(tasks, new TaskFilterFunction<>(ExecDriver.class)); } public static int getNumClusterJobs(List> tasks) { - return getMRTasks(tasks).size() + getTezTasks(tasks).size() + getSparkTasks(tasks).size(); + return getMRTasks(tasks).size() + getTezTasks(tasks).size(); } static class TaskFilterFunction implements DAGTraversal.Function { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java deleted file mode 100644 index 4b77ac9b007..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/CacheTran.java +++ /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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.io.WritableComparable; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.storage.StorageLevel; - -public abstract class CacheTran - implements SparkTran { - // whether to cache current RDD. - private boolean caching = false; - private JavaPairRDD cachedRDD; - protected final String name; - - protected CacheTran(boolean cache, String name) { - this.caching = cache; - this.name = name; - } - - @Override - public JavaPairRDD transform( - JavaPairRDD input) { - if (caching) { - if (cachedRDD == null) { - cachedRDD = doTransform(input); - cachedRDD.persist(StorageLevel.MEMORY_AND_DISK()); - } - return cachedRDD.setName(this.name + " (" + cachedRDD.getNumPartitions() + ", cached)"); - } else { - JavaPairRDD rdd = doTransform(input); - return rdd.setName(this.name + " (" + rdd.getNumPartitions() + ")"); - } - } - - public Boolean isCacheEnable() { - return caching; - } - - protected abstract JavaPairRDD doTransform(JavaPairRDD input); - - @Override - public String getName() { - return name; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/GroupByShuffler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/GroupByShuffler.java deleted file mode 100644 index 4bfde25f15d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/GroupByShuffler.java +++ /dev/null @@ -1,41 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; - -public class GroupByShuffler implements SparkShuffler> { - - @Override - public JavaPairRDD> shuffle( - JavaPairRDD input, int numPartitions) { - if (numPartitions > 0) { - return input.groupByKey(numPartitions); - } - return input.groupByKey(); - } - - @Override - public String getName() { - return "GroupBy"; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java deleted file mode 100644 index cf27e92bafd..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java +++ /dev/null @@ -1,210 +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.hadoop.hive.ql.exec.spark; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.HashTableSinkOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.MapredContext; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.TemporaryHashSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; -import org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectSerDeContext; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe; -import org.apache.hadoop.hive.ql.exec.vector.VectorizationOperator; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkBucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.VectorMapJoinDesc; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.mapred.JobConf; - -/** - * HashTableLoader for Spark to load the hashtable for MapJoins. - */ -public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTableLoader { - - private static final Logger LOG = LoggerFactory.getLogger(HashTableLoader.class.getName()); - - private ExecMapperContext context; - private Configuration hconf; - - private MapJoinOperator joinOp; - private MapJoinDesc desc; - - private boolean useFastContainer = false; - - @Override - public void init(ExecMapperContext context, MapredContext mrContext, Configuration hconf, - MapJoinOperator joinOp) { - this.context = context; - this.hconf = hconf; - this.joinOp = joinOp; - this.desc = joinOp.getConf(); - if (desc.getVectorMode() && HiveConf.getBoolVar( - hconf, HiveConf.ConfVars.HIVE_VECTORIZATION_MAPJOIN_NATIVE_FAST_HASHTABLE_ENABLED)) { - if (joinOp instanceof VectorizationOperator) { - VectorMapJoinDesc vectorDesc = (VectorMapJoinDesc) ((VectorizationOperator) joinOp).getVectorDesc(); - useFastContainer = vectorDesc != null && vectorDesc.getHashTableImplementationType() == - VectorMapJoinDesc.HashTableImplementationType.FAST; - } - } - } - - @Override - public void load(MapJoinTableContainer[] mapJoinTables, - MapJoinTableContainerSerDe[] mapJoinTableSerdes) - throws HiveException { - - // Note: it's possible that a MJ operator is in a ReduceWork, in which case the - // currentInputPath will be null. But, since currentInputPath is only interesting - // for bucket join case, and for bucket join the MJ operator will always be in - // a MapWork, this should be OK. - String currentInputPath = - context.getCurrentInputPath() == null ? null : context.getCurrentInputPath().toString(); - - LOG.info("******* Load from HashTable for input file: " + currentInputPath); - MapredLocalWork localWork = context.getLocalWork(); - try { - if (localWork.getDirectFetchOp() != null) { - loadDirectly(mapJoinTables, currentInputPath); - } - // All HashTables share the same base dir, - // which is passed in as the tmp path - Path baseDir = localWork.getTmpPath(); - if (baseDir == null) { - return; - } - FileSystem fs = FileSystem.get(baseDir.toUri(), hconf); - BucketMapJoinContext mapJoinCtx = localWork.getBucketMapjoinContext(); - boolean firstContainer = true; - boolean useOptimizedContainer = !useFastContainer && HiveConf.getBoolVar( - hconf, HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE); - for (int pos = 0; pos < mapJoinTables.length; pos++) { - if (pos == desc.getPosBigTable() || mapJoinTables[pos] != null) { - continue; - } - if (useOptimizedContainer) { - MapJoinObjectSerDeContext keyCtx = mapJoinTableSerdes[pos].getKeyContext(); - ObjectInspector keyOI = keyCtx.getSerDe().getObjectInspector(); - if (!MapJoinBytesTableContainer.isSupportedKey(keyOI)) { - if (firstContainer) { - LOG.warn("Not using optimized table container." + - "Only a subset of mapjoin keys is supported."); - useOptimizedContainer = false; - HiveConf.setBoolVar(hconf, HiveConf.ConfVars.HIVEMAPJOINUSEOPTIMIZEDTABLE, false); - } else { - throw new HiveException("Only a subset of mapjoin keys is supported."); - } - } - } - firstContainer = false; - String bigInputPath = currentInputPath; - if (currentInputPath != null && mapJoinCtx != null) { - if (!desc.isBucketMapJoin()) { - bigInputPath = null; - } else { - Set aliases = - ((SparkBucketMapJoinContext) mapJoinCtx).getPosToAliasMap().get(pos); - String alias = aliases.iterator().next(); - // Any one small table input path - String smallInputPath = - mapJoinCtx.getAliasBucketFileNameMapping().get(alias).get(bigInputPath).get(0); - bigInputPath = mapJoinCtx.getMappingBigFile(alias, smallInputPath); - } - } - String fileName = localWork.getBucketFileName(bigInputPath); - Path path = Utilities.generatePath(baseDir, desc.getDumpFilePrefix(), (byte) pos, fileName); - mapJoinTables[pos] = load(fs, path, mapJoinTableSerdes[pos]); - } - } catch (Exception e) { - throw new HiveException(e); - } - } - - private MapJoinTableContainer load(FileSystem fs, Path path, - MapJoinTableContainerSerDe mapJoinTableSerde) throws HiveException { - LOG.info("\tLoad back all hashtable files from tmp folder uri:" + path); - if (!SparkUtilities.isDedicatedCluster(hconf)) { - return useFastContainer ? mapJoinTableSerde.loadFastContainer(desc, fs, path, hconf) : - mapJoinTableSerde.load(fs, path, hconf); - } - MapJoinTableContainer mapJoinTable = SmallTableCache.get(path); - if (mapJoinTable == null) { - synchronized (path.toString().intern()) { - mapJoinTable = SmallTableCache.get(path); - if (mapJoinTable == null) { - mapJoinTable = useFastContainer ? - mapJoinTableSerde.loadFastContainer(desc, fs, path, hconf) : - mapJoinTableSerde.load(fs, path, hconf); - SmallTableCache.cache(path, mapJoinTable); - } - } - } - return mapJoinTable; - } - - private void loadDirectly(MapJoinTableContainer[] mapJoinTables, String inputFileName) - throws Exception { - MapredLocalWork localWork = context.getLocalWork(); - List> directWorks = localWork.getDirectFetchOp().get(joinOp); - if (directWorks == null || directWorks.isEmpty()) { - return; - } - JobConf job = new JobConf(hconf); - MapredLocalTask localTask = new MapredLocalTask(localWork, job, false); - - HashTableSinkOperator sink = new TemporaryHashSinkOperator(new CompilationOpContext(), desc); - sink.setParentOperators(new ArrayList>(directWorks)); - - for (Operator operator : directWorks) { - if (operator != null) { - operator.setChildOperators(Arrays.>asList(sink)); - } - } - localTask.setExecContext(context); - localTask.startForward(inputFileName); - - MapJoinTableContainer[] tables = sink.getMapJoinTables(); - for (int i = 0; i < sink.getNumParent(); i++) { - if (sink.getParentOperators().get(i) != null) { - mapJoinTables[i] = tables[i]; - } - } - - Arrays.fill(tables, null); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java deleted file mode 100644 index c75158b3c79..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveBaseFunctionResultList.java +++ /dev/null @@ -1,123 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Iterator; -import java.util.NoSuchElementException; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.OutputCollector; - -import scala.Tuple2; - -import com.google.common.base.Preconditions; - -/** - * Base class for - * - collecting Map/Reduce function output and - * - providing an Iterable interface for fetching output records. Input records - * are processed in lazy fashion i.e when output records are requested - * through Iterator interface. - */ -@SuppressWarnings("rawtypes") -public abstract class HiveBaseFunctionResultList - implements Iterator, OutputCollector, Serializable { - private static final long serialVersionUID = -1L; - private final Iterator inputIterator; - private boolean isClosed = false; - - // Contains results from last processed input record. - private final HiveKVResultCache lastRecordOutput; - - public HiveBaseFunctionResultList(Iterator inputIterator) { - this.inputIterator = inputIterator; - this.lastRecordOutput = new HiveKVResultCache(); - } - - @Override - public void collect(HiveKey key, BytesWritable value) throws IOException { - lastRecordOutput.add(SparkUtilities.copyHiveKey(key), - SparkUtilities.copyBytesWritable(value)); - } - - /** Process the given record. */ - protected abstract void processNextRecord(T inputRecord) throws IOException; - - /** - * @return true if current state of the record processor is done. - */ - protected abstract boolean processingDone(); - - /** Close the record processor. */ - protected abstract void closeRecordProcessor(); - - @Override - public boolean hasNext() { - // Return remaining records (if any) from last processed input record. - if (lastRecordOutput.hasNext()) { - return true; - } - - // Process the records in the input iterator until - // - new output records are available for serving downstream operator, - // - input records are exhausted or - // - processing is completed. - while (inputIterator.hasNext() && !processingDone()) { - try { - processNextRecord(inputIterator.next()); - if (lastRecordOutput.hasNext()) { - return true; - } - } catch (IOException ex) { - throw new IllegalStateException("Error while processing input.", ex); - } - } - - // At this point we are done processing the input. Close the record processor - if (!isClosed) { - closeRecordProcessor(); - isClosed = true; - } - - // It is possible that some operators add records after closing the processor, so make sure - // to check the lastRecordOutput - if (lastRecordOutput.hasNext()) { - return true; - } - - lastRecordOutput.clear(); - return false; - } - - @Override - public Tuple2 next() { - if (hasNext()) { - return lastRecordOutput.next(); - } - throw new NoSuchElementException("There are no more elements"); - } - - @Override - public void remove() { - throw new UnsupportedOperationException("Iterator.remove() is not supported"); - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java deleted file mode 100644 index 413946b39d7..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveKVResultCache.java +++ /dev/null @@ -1,267 +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.hadoop.hive.ql.exec.spark; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; - -import scala.Tuple2; - -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; - -/** - * A cache with fixed buffer. If the buffer is full, new entries will - * be written to disk. This class is thread safe since multiple threads - * could access it (doesn't have to be concurrently), for example, - * the StreamThread in ScriptOperator. - */ -@SuppressWarnings("unchecked") -class HiveKVResultCache { - private static final Logger LOG = LoggerFactory.getLogger(HiveKVResultCache.class); - - @VisibleForTesting - static final int IN_MEMORY_NUM_ROWS = 1024; - - private ObjectPair[] writeBuffer; - private ObjectPair[] readBuffer; - - private File parentFile; - private File tmpFile; - - private int readCursor = 0; - private int writeCursor = 0; - - // Indicate if the read buffer has data, for example, - // when in reading, data on disk could be pull in - private boolean readBufferUsed = false; - private int rowsInReadBuffer = 0; - - private Input input; - private Output output; - - public HiveKVResultCache() { - writeBuffer = new ObjectPair[IN_MEMORY_NUM_ROWS]; - readBuffer = new ObjectPair[IN_MEMORY_NUM_ROWS]; - for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) { - writeBuffer[i] = new ObjectPair(); - readBuffer[i] = new ObjectPair(); - } - } - - private void switchBufferAndResetCursor() { - ObjectPair[] tmp = readBuffer; - rowsInReadBuffer = writeCursor; - readBuffer = writeBuffer; - readBufferUsed = true; - readCursor = 0; - writeBuffer = tmp; - writeCursor = 0; - } - - private void setupOutput() throws IOException { - if (parentFile == null) { - while (true) { - parentFile = File.createTempFile("hive-resultcache", ""); - if (parentFile.delete() && parentFile.mkdir()) { - parentFile.deleteOnExit(); - break; - } - if (LOG.isDebugEnabled()) { - LOG.debug("Retry creating tmp result-cache directory..."); - } - } - } - - if (tmpFile == null || input != null) { - tmpFile = File.createTempFile("ResultCache", ".tmp", parentFile); - LOG.info("ResultCache created temp file " + tmpFile.getAbsolutePath()); - tmpFile.deleteOnExit(); - } - - FileOutputStream fos = null; - try { - fos = new FileOutputStream(tmpFile); - output = new Output(fos); - } finally { - if (output == null && fos != null) { - fos.close(); - } - } - } - - private BytesWritable readValue(Input input) { - return new BytesWritable(input.readBytes(input.readInt())); - } - - private void writeValue(Output output, BytesWritable bytesWritable) { - int size = bytesWritable.getLength(); - output.writeInt(size); - output.writeBytes(bytesWritable.getBytes(), 0, size); - } - - private HiveKey readHiveKey(Input input) { - HiveKey hiveKey = new HiveKey( - input.readBytes(input.readInt()), input.readInt()); - hiveKey.setDistKeyLength(input.readInt()); - return hiveKey; - } - - private void writeHiveKey(Output output, HiveKey hiveKey) { - int size = hiveKey.getLength(); - output.writeInt(size); - output.writeBytes(hiveKey.getBytes(), 0, size); - output.writeInt(hiveKey.hashCode()); - output.writeInt(hiveKey.getDistKeyLength()); - } - - public synchronized void add(HiveKey key, BytesWritable value) { - if (writeCursor >= IN_MEMORY_NUM_ROWS) { // Write buffer is full - if (!readBufferUsed) { // Read buffer isn't used, switch buffer - switchBufferAndResetCursor(); - } else { - // Need to spill from write buffer to disk - try { - if (output == null) { - setupOutput(); - } - for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) { - ObjectPair pair = writeBuffer[i]; - writeHiveKey(output, pair.getFirst()); - writeValue(output, pair.getSecond()); - pair.setFirst(null); - pair.setSecond(null); - } - writeCursor = 0; - } catch (Exception e) { - clear(); // Clean up the cache - throw new RuntimeException("Failed to spill rows to disk", e); - } - } - } - ObjectPair pair = writeBuffer[writeCursor++]; - pair.setFirst(key); - pair.setSecond(value); - } - - public synchronized void clear() { - writeCursor = readCursor = rowsInReadBuffer = 0; - readBufferUsed = false; - - if (parentFile != null) { - if (input != null) { - try { - input.close(); - } catch (Throwable ignored) { - } - input = null; - } - if (output != null) { - try { - output.close(); - } catch (Throwable ignored) { - } - output = null; - } - try { - FileUtil.fullyDelete(parentFile); - } catch (Throwable ignored) { - } - parentFile = null; - tmpFile = null; - } - } - - public synchronized boolean hasNext() { - return readBufferUsed || writeCursor > 0; - } - - public synchronized Tuple2 next() { - Preconditions.checkState(hasNext()); - if (!readBufferUsed) { - try { - if (input == null && output != null) { - // Close output stream if open - output.close(); - output = null; - - FileInputStream fis = null; - try { - fis = new FileInputStream(tmpFile); - input = new Input(fis); - } finally { - if (input == null && fis != null) { - fis.close(); - } - } - } - if (input != null) { - // Load next batch from disk - for (int i = 0; i < IN_MEMORY_NUM_ROWS; i++) { - ObjectPair pair = readBuffer[i]; - pair.setFirst(readHiveKey(input)); - pair.setSecond(readValue(input)); - } - if (input.eof()) { - input.close(); - input = null; - } - rowsInReadBuffer = IN_MEMORY_NUM_ROWS; - readBufferUsed = true; - readCursor = 0; - } else if (writeCursor == 1) { - ObjectPair pair = writeBuffer[0]; - Tuple2 row = new Tuple2( - pair.getFirst(), pair.getSecond()); - pair.setFirst(null); - pair.setSecond(null); - writeCursor = 0; - return row; - } else { - // No record on disk, more data in write buffer - switchBufferAndResetCursor(); - } - } catch (Exception e) { - clear(); // Clean up the cache - throw new RuntimeException("Failed to load rows from disk", e); - } - } - ObjectPair pair = readBuffer[readCursor]; - Tuple2 row = new Tuple2( - pair.getFirst(), pair.getSecond()); - pair.setFirst(null); - pair.setSecond(null); - if (++readCursor >= rowsInReadBuffer) { - readBufferUsed = false; - rowsInReadBuffer = 0; - readCursor = 0; - } - return row; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java deleted file mode 100644 index 9296aaad922..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunction.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.exec.spark; - -import java.util.Iterator; - -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.io.merge.MergeFileMapper; -import org.apache.hadoop.io.BytesWritable; - -import scala.Tuple2; - -public class HiveMapFunction extends HivePairFlatMapFunction< - Iterator>, HiveKey, BytesWritable> { - - private static final long serialVersionUID = 1L; - - public HiveMapFunction(byte[] jobConfBuffer, SparkReporter sparkReporter) { - super(jobConfBuffer, sparkReporter); - } - - @SuppressWarnings("unchecked") - @Override - public Iterator> - call(Iterator> it) throws Exception { - initJobConf(); - - SparkRecordHandler mapRecordHandler; - - // need different record handler for MergeFileWork - if (MergeFileMapper.class.getName().equals(jobConf.get(Utilities.MAPRED_MAPPER_CLASS))) { - mapRecordHandler = new SparkMergeFileRecordHandler(); - } else { - mapRecordHandler = new SparkMapRecordHandler(); - } - - HiveMapFunctionResultList result = new HiveMapFunctionResultList(it, mapRecordHandler); - mapRecordHandler.init(jobConf, result, sparkReporter); - - return result; - } - - @Override - protected boolean isMap() { - return true; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java deleted file mode 100644 index 8ba9e18a024..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveMapFunctionResultList.java +++ /dev/null @@ -1,60 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.Iterator; - -import org.apache.hadoop.io.BytesWritable; - -import scala.Tuple2; - -public class HiveMapFunctionResultList extends - HiveBaseFunctionResultList> { - private static final long serialVersionUID = 1L; - private final SparkRecordHandler recordHandler; - - /** - * Instantiate result set Iterable for Map function output. - * - * @param inputIterator Input record iterator. - * @param handler Initialized {@link SparkMapRecordHandler} instance. - */ - public HiveMapFunctionResultList( - Iterator> inputIterator, - SparkRecordHandler handler) { - super(inputIterator); - recordHandler = handler; - } - - @Override - protected void processNextRecord(Tuple2 inputRecord) - throws IOException { - recordHandler.processRow(inputRecord._1(), inputRecord._2()); - } - - @Override - protected boolean processingDone() { - return recordHandler.getDone(); - } - - @Override - protected void closeRecordProcessor() { - recordHandler.close(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java deleted file mode 100644 index 7b019b9c4d9..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HivePairFlatMapFunction.java +++ /dev/null @@ -1,87 +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.hadoop.hive.ql.exec.spark; - -import java.text.NumberFormat; - -import org.apache.hadoop.mapred.JobConf; -import org.apache.spark.TaskContext; -import org.apache.spark.api.java.function.PairFlatMapFunction; - - -public abstract class HivePairFlatMapFunction implements PairFlatMapFunction { - private final NumberFormat taskIdFormat = NumberFormat.getInstance(); - private final NumberFormat stageIdFormat = NumberFormat.getInstance(); - { - taskIdFormat.setGroupingUsed(false); - taskIdFormat.setMinimumIntegerDigits(6); - stageIdFormat.setGroupingUsed(false); - stageIdFormat.setMinimumIntegerDigits(4); - } - - protected transient JobConf jobConf; - protected SparkReporter sparkReporter; - - private byte[] buffer; - - public HivePairFlatMapFunction(byte[] buffer, SparkReporter sparkReporter) { - this.buffer = buffer; - this.sparkReporter = sparkReporter; - } - - protected void initJobConf() { - if (jobConf == null) { - jobConf = KryoSerializer.deserializeJobConf(this.buffer); - SmallTableCache.initialize(jobConf); - setupMRLegacyConfigs(); - } - } - - protected abstract boolean isMap(); - - // Some Hive features depends on several MR configuration legacy, build and add - // these configuration to JobConf here. - private void setupMRLegacyConfigs() { - StringBuilder taskAttemptIdBuilder = new StringBuilder("attempt_"); - taskAttemptIdBuilder.append(System.currentTimeMillis()) - .append("_") - .append(stageIdFormat.format(TaskContext.get().stageId())) - .append("_"); - - if (isMap()) { - taskAttemptIdBuilder.append("m_"); - } else { - taskAttemptIdBuilder.append("r_"); - } - - // Hive requires this TaskAttemptId to be unique. MR's TaskAttemptId is composed - // of "attempt_timestamp_jobNum_m/r_taskNum_attemptNum". The counterpart for - // Spark should be "attempt_timestamp_stageNum_m/r_partitionId_attemptNum". - // When there're multiple attempts for a task, Hive will rely on the partitionId - // to figure out if the data are duplicate or not when collecting the final outputs - // (see org.apache.hadoop.hive.ql.exec.Utils.removeTempOrDuplicateFiles) - taskAttemptIdBuilder.append(taskIdFormat.format(TaskContext.get().partitionId())) - .append("_").append(TaskContext.get().attemptNumber()); - - String taskAttemptIdStr = taskAttemptIdBuilder.toString(); - jobConf.set("mapred.task.id", taskAttemptIdStr); - jobConf.set("mapreduce.task.attempt.id", taskAttemptIdStr); - jobConf.setInt("mapred.task.partition", TaskContext.get().partitionId()); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java deleted file mode 100644 index 9c7d15a3e30..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunction.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.exec.spark; - -import java.util.Iterator; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; - -import scala.Tuple2; - -public class HiveReduceFunction extends HivePairFlatMapFunction< - Iterator>, HiveKey, BytesWritable> { - - private static final long serialVersionUID = 1L; - - public HiveReduceFunction(byte[] buffer, SparkReporter sparkReporter) { - super(buffer, sparkReporter); - } - - @SuppressWarnings("unchecked") - @Override - public Iterator> - call(Iterator> it) throws Exception { - initJobConf(); - - SparkReduceRecordHandler reducerRecordhandler = new SparkReduceRecordHandler(); - HiveReduceFunctionResultList result = - new HiveReduceFunctionResultList(it, reducerRecordhandler); - reducerRecordhandler.init(jobConf, result, sparkReporter); - - return result; - } - - @Override - protected boolean isMap() { - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java deleted file mode 100644 index 7cc2e9af6ed..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveReduceFunctionResultList.java +++ /dev/null @@ -1,69 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.Iterator; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; - -import scala.Tuple2; - -public class HiveReduceFunctionResultList extends - HiveBaseFunctionResultList> { - private static final long serialVersionUID = 1L; - private final SparkReduceRecordHandler reduceRecordHandler; - - /** - * Instantiate result set Iterable for Reduce function output. - * - * @param inputIterator Input record iterator. - * @param reducer Initialized {@link org.apache.hadoop.hive.ql.exec.mr.ExecReducer} instance. - */ - public HiveReduceFunctionResultList( - Iterator> inputIterator, - SparkReduceRecordHandler reducer) { - super(inputIterator); - this.reduceRecordHandler = reducer; - } - - @Override - protected void processNextRecord(Tuple2 inputRecord) - throws IOException { - HiveKey key = inputRecord._1(); - V value = inputRecord._2(); - if (value instanceof Iterable) { - @SuppressWarnings("unchecked") - Iterable values = (Iterable)value; - reduceRecordHandler.processRow(key, values.iterator()); - } else { - reduceRecordHandler.processRow(key, value); - } - } - - @Override - protected boolean processingDone() { - return false; - } - - @Override - protected void closeRecordProcessor() { - reduceRecordHandler.close(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java deleted file mode 100644 index 8a44c1b0a7d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClient.java +++ /dev/null @@ -1,54 +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.hadoop.hive.ql.exec.spark; - -import java.io.Closeable; -import java.io.Serializable; - -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.spark.SparkConf; - -public interface HiveSparkClient extends Serializable, Closeable { - /** - * HiveSparkClient should generate Spark RDD graph by given sparkWork and driverContext, - * and submit RDD graph to Spark cluster. - * @param driverContext - * @param sparkWork - * @return SparkJobRef could be used to track spark job progress and metrics. - * @throws Exception - */ - SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception; - - /** - * @return spark configuration - */ - SparkConf getSparkConf(); - - /** - * @return the number of executors - */ - int getExecutorCount() throws Exception; - - /** - * For standalone mode, this can be used to get total number of cores. - * @return default parallelism. - */ - int getDefaultParallelism() throws Exception; -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java deleted file mode 100644 index c22fb8923d9..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java +++ /dev/null @@ -1,269 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.commons.compress.utils.CharsetNames; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.hive.common.LogUtils; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hive.spark.client.SparkClientUtilities; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConfUtil; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hive.spark.client.rpc.RpcConfiguration; -import org.apache.spark.SparkConf; - -import com.google.common.base.Joiner; -import com.google.common.base.Splitter; -import com.google.common.base.Strings; -import com.google.common.collect.Sets; - -public class HiveSparkClientFactory { - protected static final transient Logger LOG = LoggerFactory.getLogger(HiveSparkClientFactory.class); - - private static final String SPARK_DEFAULT_CONF_FILE = "spark-defaults.conf"; - private static final String SPARK_DEFAULT_MASTER = "yarn"; - private static final String SPARK_DEFAULT_DEPLOY_MODE = "cluster"; - private static final String SPARK_DEFAULT_APP_NAME = "Hive on Spark"; - private static final String SPARK_DEFAULT_SERIALIZER = "org.apache.spark.serializer.KryoSerializer"; - private static final String SPARK_DEFAULT_REFERENCE_TRACKING = "false"; - private static final String SPARK_WAIT_APP_COMPLETE = "spark.yarn.submit.waitAppCompletion"; - private static final String SPARK_DEPLOY_MODE = "spark.submit.deployMode"; - @VisibleForTesting - public static final String SPARK_CLONE_CONFIGURATION = "spark.hadoop.cloneConf"; - - public static HiveSparkClient createHiveSparkClient(HiveConf hiveconf, String sessionId) throws Exception { - Map sparkConf = initiateSparkConf(hiveconf, sessionId); - - // Submit spark job through local spark context while spark master is local mode, otherwise submit - // spark job through remote spark context. - String master = sparkConf.get("spark.master"); - if (master.equals("local") || master.startsWith("local[")) { - // With local spark context, all user sessions share the same spark context. - return LocalHiveSparkClient.getInstance(generateSparkConf(sparkConf), hiveconf); - } else { - return new RemoteHiveSparkClient(hiveconf, sparkConf, sessionId); - } - } - - public static Map initiateSparkConf(HiveConf hiveConf, String sessionId) { - Map sparkConf = new HashMap(); - HBaseConfiguration.addHbaseResources(hiveConf); - - // set default spark configurations. - sparkConf.put("spark.master", SPARK_DEFAULT_MASTER); - final String appNameKey = "spark.app.name"; - String appName = hiveConf.get(appNameKey); - final String sessionIdString = " (sessionId = " + sessionId + ")"; - if (appName == null) { - if (sessionId == null) { - appName = SPARK_DEFAULT_APP_NAME; - } else { - appName = SPARK_DEFAULT_APP_NAME + sessionIdString; - } - } else { - appName = appName + sessionIdString; - } - sparkConf.put(appNameKey, appName); - sparkConf.put("spark.serializer", SPARK_DEFAULT_SERIALIZER); - sparkConf.put("spark.kryo.referenceTracking", SPARK_DEFAULT_REFERENCE_TRACKING); - - // load properties from spark-defaults.conf. - InputStream inputStream = null; - try { - inputStream = HiveSparkClientFactory.class.getClassLoader() - .getResourceAsStream(SPARK_DEFAULT_CONF_FILE); - if (inputStream != null) { - LOG.info("loading spark properties from: " + SPARK_DEFAULT_CONF_FILE); - Properties properties = new Properties(); - properties.load(new InputStreamReader(inputStream, CharsetNames.UTF_8)); - for (String propertyName : properties.stringPropertyNames()) { - if (propertyName.startsWith("spark")) { - String value = properties.getProperty(propertyName); - sparkConf.put(propertyName, properties.getProperty(propertyName)); - LOG.debug(String.format( - "load spark property from %s (%s -> %s).", - SPARK_DEFAULT_CONF_FILE, propertyName, LogUtils.maskIfPassword(propertyName,value))); - } - } - } - } catch (IOException e) { - LOG.info("Failed to open spark configuration file: " - + SPARK_DEFAULT_CONF_FILE, e); - } finally { - if (inputStream != null) { - try { - inputStream.close(); - } catch (IOException e) { - LOG.debug("Failed to close inputstream.", e); - } - } - } - - // load properties from hive configurations, including both spark.* properties, - // properties for remote driver RPC, and yarn properties for Spark on YARN mode. - String sparkMaster = hiveConf.get("spark.master"); - if (sparkMaster == null) { - sparkMaster = sparkConf.get("spark.master"); - hiveConf.set("spark.master", sparkMaster); - } - String deployMode = null; - if (!SparkClientUtilities.isLocalMaster(sparkMaster)) { - deployMode = hiveConf.get(SPARK_DEPLOY_MODE); - if (deployMode == null) { - deployMode = sparkConf.get(SPARK_DEPLOY_MODE); - if (deployMode == null) { - deployMode = SparkClientUtilities.getDeployModeFromMaster(sparkMaster); - } - if (deployMode == null) { - deployMode = SPARK_DEFAULT_DEPLOY_MODE; - } - hiveConf.set(SPARK_DEPLOY_MODE, deployMode); - } - } - if (SessionState.get() != null && SessionState.get().getConf() != null) { - SessionState.get().getConf().set("spark.master", sparkMaster); - if (deployMode != null) { - SessionState.get().getConf().set(SPARK_DEPLOY_MODE, deployMode); - } - } - if (SparkClientUtilities.isYarnClusterMode(sparkMaster, deployMode)) { - sparkConf.put("spark.yarn.maxAppAttempts", "1"); - } - for (Map.Entry entry : hiveConf) { - String propertyName = entry.getKey(); - if (propertyName.startsWith("spark")) { - String value = hiveConf.get(propertyName); - sparkConf.put(propertyName, value); - LOG.debug(String.format( - "load spark property from hive configuration (%s -> %s).", - propertyName, LogUtils.maskIfPassword(propertyName,value))); - } else if (propertyName.startsWith("yarn") && - SparkClientUtilities.isYarnMaster(sparkMaster)) { - String value = hiveConf.get(propertyName); - // Add spark.hadoop prefix for yarn properties as SparkConf only accept properties - // started with spark prefix, Spark would remove spark.hadoop prefix lately and add - // it to its hadoop configuration. - sparkConf.put("spark.hadoop." + propertyName, value); - LOG.debug(String.format( - "load yarn property from hive configuration in %s mode (%s -> %s).", - sparkMaster, propertyName, LogUtils.maskIfPassword(propertyName,value))); - } else if (propertyName.equals(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY)) { - String value = hiveConf.get(propertyName); - if (value != null && !value.isEmpty()) { - sparkConf.put("spark.hadoop." + propertyName, value); - } - } else if (propertyName.startsWith("hbase") || propertyName.startsWith("zookeeper.znode")) { - // Add HBase related configuration to Spark because in security mode, Spark needs it - // to generate hbase delegation token for Spark. This is a temp solution to deal with - // Spark problem. - String value = hiveConf.get(propertyName); - sparkConf.put("spark.hadoop." + propertyName, value); - LOG.debug(String.format( - "load HBase configuration (%s -> %s).", propertyName, LogUtils.maskIfPassword(propertyName,value))); - } else if (propertyName.startsWith("oozie")) { - String value = hiveConf.get(propertyName); - sparkConf.put("spark." + propertyName, value); - LOG.debug(String.format( - "Pass Oozie configuration (%s -> %s).", propertyName, LogUtils.maskIfPassword(propertyName,value))); - } - - if (RpcConfiguration.HIVE_SPARK_RSC_CONFIGS.contains(propertyName)) { - String value = RpcConfiguration.getValue(hiveConf, propertyName); - sparkConf.put(propertyName, value); - LOG.debug(String.format( - "load RPC property from hive configuration (%s -> %s).", - propertyName, LogUtils.maskIfPassword(propertyName,value))); - } - } - - final boolean optShuffleSerDe = hiveConf.getBoolVar( - HiveConf.ConfVars.SPARK_OPTIMIZE_SHUFFLE_SERDE); - - Set classes = Sets.newHashSet( - Splitter.on(",").trimResults().omitEmptyStrings().split( - Strings.nullToEmpty(sparkConf.get("spark.kryo.classesToRegister")))); - classes.add(Writable.class.getName()); - classes.add(VectorizedRowBatch.class.getName()); - if (!optShuffleSerDe) { - classes.add(HiveKey.class.getName()); - classes.add(BytesWritable.class.getName()); - } else { - sparkConf.put("spark.kryo.registrator", SparkClientUtilities.HIVE_KRYO_REG_NAME); - } - sparkConf.put("spark.kryo.classesToRegister", Joiner.on(",").join(classes)); - - // set yarn queue name - final String sparkQueueNameKey = "spark.yarn.queue"; - if (SparkClientUtilities.isYarnMaster(sparkMaster) && hiveConf.get(sparkQueueNameKey) == null) { - String queueName = hiveConf.get("mapreduce.job.queuename"); - if (queueName != null) { - sparkConf.put(sparkQueueNameKey, queueName); - } - } - - // Disable it to avoid verbose app state report in yarn-cluster mode - if (SparkClientUtilities.isYarnClusterMode(sparkMaster, deployMode) && - sparkConf.get(SPARK_WAIT_APP_COMPLETE) == null) { - sparkConf.put(SPARK_WAIT_APP_COMPLETE, "false"); - } - - // Force Spark configs to be cloned by default - sparkConf.putIfAbsent(SPARK_CLONE_CONFIGURATION, "true"); - - - // Set the credential provider passwords if found, if there is job specific password - // the credential provider location is set directly in the execute method of LocalSparkClient - // and submit method of RemoteHiveSparkClient when the job config is created - String password = HiveConfUtil.getJobCredentialProviderPassword(hiveConf); - if(password != null) { - addCredentialProviderPassword(sparkConf, password); - } - return sparkConf; - } - - private static void addCredentialProviderPassword(Map sparkConf, - String jobCredstorePassword) { - sparkConf.put("spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD", jobCredstorePassword); - sparkConf.put("spark.executorEnv.HADOOP_CREDSTORE_PASSWORD", jobCredstorePassword); - } - - static SparkConf generateSparkConf(Map conf) { - SparkConf sparkConf = new SparkConf(false); - for (Map.Entry entry : conf.entrySet()) { - sparkConf.set(entry.getKey(), entry.getValue()); - } - return sparkConf; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveVoidFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveVoidFunction.java deleted file mode 100644 index f79cc96a772..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveVoidFunction.java +++ /dev/null @@ -1,46 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.function.VoidFunction; - -import scala.Tuple2; - -/** - * Implementation of a voidFunction that does nothing. - */ -public class HiveVoidFunction implements VoidFunction> { - private static final long serialVersionUID = 1L; - - private static HiveVoidFunction instance = new HiveVoidFunction(); - - public static HiveVoidFunction getInstance() { - return instance; - } - - private HiveVoidFunction() { - } - - @Override - public void call(Tuple2 t) throws Exception { - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java deleted file mode 100644 index f77e92cd2a0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/KryoSerializer.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.exec.spark; - -import java.io.ByteArrayInputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import org.apache.commons.io.output.ByteArrayOutputStream; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; -import org.apache.hadoop.mapred.JobConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; - -public class KryoSerializer { - private static final Logger LOG = LoggerFactory.getLogger(KryoSerializer.class); - - public static byte[] serialize(Object object) { - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - Output output = new Output(stream); - - Kryo kryo = SerializationUtilities.borrowKryo(); - kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); - try { - kryo.writeObject(output, object); - } finally { - SerializationUtilities.releaseKryo(kryo); - } - - output.close(); // close() also calls flush() - return stream.toByteArray(); - } - - public static T deserialize(byte[] buffer, Class clazz) { - Kryo kryo = SerializationUtilities.borrowKryo(); - kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); - T result = null; - try { - result = kryo.readObject(new Input(new ByteArrayInputStream(buffer)), clazz); - } finally { - SerializationUtilities.releaseKryo(kryo); - } - return result; - } - - public static byte[] serializeJobConf(JobConf jobConf) { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - try { - jobConf.write(new DataOutputStream(out)); - } catch (IOException e) { - LOG.error("Error serializing job configuration: " + e, e); - return null; - } finally { - try { - out.close(); - } catch (IOException e) { - LOG.error("Error closing output stream: " + e, e); - } - } - - return out.toByteArray(); - - } - - public static JobConf deserializeJobConf(byte[] buffer) { - JobConf conf = new JobConf(); - try { - conf.readFields(new DataInputStream(new ByteArrayInputStream(buffer))); - } catch (IOException e) { - String msg = "Error de-serializing job configuration: " + e; - throw new IllegalStateException(msg, e); - } - return conf; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java deleted file mode 100644 index dff3b0f569c..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/LocalHiveSparkClient.java +++ /dev/null @@ -1,245 +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.hadoop.hive.ql.exec.spark; - -import java.io.File; -import java.io.FileNotFoundException; -import java.net.MalformedURLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.ql.exec.DagUtils; -import org.apache.hive.spark.client.SparkClientUtilities; -import org.apache.spark.util.CallSite; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConfUtil; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.JobMetricsListener; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.LocalSparkJobStatus; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import com.google.common.base.Splitter; -import com.google.common.base.Strings; - -/** - * LocalSparkClient submit Spark job in local driver, it's responsible for build spark client - * environment and execute spark work. - */ -public class LocalHiveSparkClient implements HiveSparkClient { - private static final long serialVersionUID = 1L; - - private static final String MR_JAR_PROPERTY = "tmpjars"; - protected static final transient Logger LOG = LoggerFactory - .getLogger(LocalHiveSparkClient.class); - - private static final Splitter CSV_SPLITTER = Splitter.on(",").omitEmptyStrings(); - - private static LocalHiveSparkClient client; - - public static synchronized LocalHiveSparkClient getInstance( - SparkConf sparkConf, HiveConf hiveConf) throws FileNotFoundException, MalformedURLException { - if (client == null) { - client = new LocalHiveSparkClient(sparkConf, hiveConf); - } - return client; - } - - private final JavaSparkContext sc; - - private final List localJars = new ArrayList(); - - private final List localFiles = new ArrayList(); - - private final JobMetricsListener jobMetricsListener; - - private LocalHiveSparkClient(SparkConf sparkConf, HiveConf hiveConf) - throws FileNotFoundException, MalformedURLException { - String regJar = null; - // the registrator jar should already be in CP when not in test mode - if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVE_IN_TEST)) { - String kryoReg = sparkConf.get("spark.kryo.registrator", ""); - if (SparkClientUtilities.HIVE_KRYO_REG_NAME.equals(kryoReg)) { - regJar = SparkClientUtilities.findKryoRegistratorJar(hiveConf); - SparkClientUtilities.addJarToContextLoader(new File(regJar)); - } - } - sc = new JavaSparkContext(sparkConf); - if (regJar != null) { - sc.addJar(regJar); - } - jobMetricsListener = new JobMetricsListener(); - sc.sc().addSparkListener(jobMetricsListener); - } - - @Override - public SparkConf getSparkConf() { - return sc.sc().conf(); - } - - @Override - public int getExecutorCount() { - return sc.sc().getExecutorMemoryStatus().size(); - } - - @Override - public int getDefaultParallelism() throws Exception { - return sc.sc().defaultParallelism(); - } - - @Override - public SparkJobRef execute(DriverContext driverContext, SparkWork sparkWork) throws Exception { - Context ctx = driverContext.getCtx(); - HiveConf hiveConf = (HiveConf) ctx.getConf(); - refreshLocalResources(sparkWork, hiveConf); - JobConf jobConf = new JobConf(hiveConf); - - // Create temporary scratch dir - Path emptyScratchDir; - emptyScratchDir = ctx.getMRTmpPath(); - FileSystem fs = emptyScratchDir.getFileSystem(jobConf); - fs.mkdirs(emptyScratchDir); - - // Update credential provider location - // the password to the credential provider in already set in the sparkConf - // in HiveSparkClientFactory - HiveConfUtil.updateJobCredentialProviders(jobConf); - - SparkCounters sparkCounters = new SparkCounters(sc); - Map> prefixes = sparkWork.getRequiredCounterPrefix(); - if (prefixes != null) { - for (String group : prefixes.keySet()) { - for (String counterName : prefixes.get(group)) { - sparkCounters.createCounter(group, counterName); - } - } - } - SparkReporter sparkReporter = new SparkReporter(sparkCounters); - - // Generate Spark plan - SparkPlanGenerator gen = - new SparkPlanGenerator(sc, ctx, jobConf, emptyScratchDir, sparkReporter); - SparkPlan plan = gen.generate(sparkWork); - - if (driverContext.isShutdown()) { - throw new HiveException("Operation is cancelled."); - } - - // Execute generated plan. - JavaPairRDD finalRDD = plan.generateGraph(); - - sc.setJobGroup("queryId = " + sparkWork.getQueryId(), DagUtils.getQueryName(jobConf)); - - // We use Spark RDD async action to submit job as it's the only way to get jobId now. - JavaFutureAction future = finalRDD.foreachAsync(HiveVoidFunction.getInstance()); - - // As we always use foreach action to submit RDD graph, it would only trigger one job. - int jobId = future.jobIds().get(0); - LocalSparkJobStatus sparkJobStatus = new LocalSparkJobStatus( - sc, jobId, jobMetricsListener, sparkCounters, plan.getCachedRDDIds(), future); - return new LocalSparkJobRef(Integer.toString(jobId), hiveConf, sparkJobStatus, sc); - } - - /** - * At this point single SparkContext is used by more than one thread, so make this - * method synchronized. - * - * This method can't remove a jar/resource from SparkContext. Looks like this is an - * issue we have to live with until multiple SparkContexts are supported in a single JVM. - */ - private synchronized void refreshLocalResources(SparkWork sparkWork, HiveConf conf) { - // add hive-exec jar - addJars((new JobConf(this.getClass())).getJar()); - - // add aux jars - addJars(conf.getAuxJars()); - addJars(SessionState.get() == null ? null : SessionState.get().getReloadableAuxJars()); - - // add added jars - String addedJars = Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDJARS, addedJars); - addJars(addedJars); - - // add plugin module jars on demand - // jobConf will hold all the configuration for hadoop, tez, and hive - JobConf jobConf = new JobConf(conf); - jobConf.set(MR_JAR_PROPERTY, ""); - for (BaseWork work : sparkWork.getAllWork()) { - work.configureJobConf(jobConf); - } - addJars(jobConf.get(MR_JAR_PROPERTY)); - - // add added files - String addedFiles = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDFILES, addedFiles); - addResources(addedFiles); - - // add added archives - String addedArchives = Utilities.getResourceFiles(conf, SessionState.ResourceType.ARCHIVE); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDARCHIVES, addedArchives); - addResources(addedArchives); - } - - private void addResources(String addedFiles) { - for (String addedFile : CSV_SPLITTER.split(Strings.nullToEmpty(addedFiles))) { - if (!localFiles.contains(addedFile)) { - localFiles.add(addedFile); - sc.addFile(addedFile); - } - } - } - - private void addJars(String addedJars) { - for (String addedJar : CSV_SPLITTER.split(Strings.nullToEmpty(addedJars))) { - if (!localJars.contains(addedJar)) { - localJars.add(addedJar); - sc.addJar(addedJar); - } - } - } - - @Override - public void close() { - synchronized (LocalHiveSparkClient.class) { - client = null; - } - if (sc != null) { - sc.stop(); - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java deleted file mode 100644 index b1a0d55367f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapInput.java +++ /dev/null @@ -1,101 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.io.WritableUtils; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.storage.StorageLevel; - -import scala.Tuple2; - -import com.google.common.base.Preconditions; - - -public class MapInput implements SparkTran { - private JavaPairRDD hadoopRDD; - private boolean toCache; - private final SparkPlan sparkPlan; - private final String name; - - public MapInput(SparkPlan sparkPlan, JavaPairRDD hadoopRDD) { - this(sparkPlan, hadoopRDD, false, "MapInput"); - } - - public MapInput(SparkPlan sparkPlan, - JavaPairRDD hadoopRDD, boolean toCache, String name) { - this.hadoopRDD = hadoopRDD; - this.toCache = toCache; - this.sparkPlan = sparkPlan; - this.name = name; - } - - public void setToCache(boolean toCache) { - this.toCache = toCache; - } - - @Override - public JavaPairRDD transform( - JavaPairRDD input) { - Preconditions.checkArgument(input == null, - "AssertionError: MapInput doesn't take any input"); - JavaPairRDD result; - if (toCache) { - result = hadoopRDD.mapToPair(new CopyFunction()); - sparkPlan.addCachedRDDId(result.id()); - result = result.persist(StorageLevel.MEMORY_AND_DISK()); - } else { - result = hadoopRDD; - } - result.setName(this.name); - return result; - } - - private static class CopyFunction implements PairFunction, - WritableComparable, Writable> { - - private transient Configuration conf; - - @Override - public Tuple2 - call(Tuple2 tuple) throws Exception { - if (conf == null) { - conf = new Configuration(); - } - - return new Tuple2(tuple._1(), - WritableUtils.clone(tuple._2(), conf)); - } - - } - - @Override - public String getName() { - return name; - } - - @Override - public Boolean isCacheEnable() { - return new Boolean(toCache); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapTran.java deleted file mode 100644 index b102f5147de..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/MapTran.java +++ /dev/null @@ -1,45 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; - -public class MapTran extends CacheTran { - private HiveMapFunction mapFunc; - - public MapTran() { - this(false, "MapTran"); - } - - public MapTran(boolean cache, String name) { - super(cache, name); - } - - @Override - public JavaPairRDD doTransform( - JavaPairRDD input) { - return input.mapPartitionsToPair(mapFunc); - } - - public void setMapFunction(HiveMapFunction mapFunc) { - this.mapFunc = mapFunc; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ReduceTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ReduceTran.java deleted file mode 100644 index 3b34c78a7a0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ReduceTran.java +++ /dev/null @@ -1,45 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; - -public class ReduceTran extends CacheTran { - private HiveReduceFunction reduceFunc; - - public ReduceTran() { - this(false, "Reduce"); - } - - public ReduceTran(boolean caching, String name) { - super(caching, name); - } - - @Override - public JavaPairRDD doTransform( - JavaPairRDD input) { - return input.mapPartitionsToPair(reduceFunc); - } - - public void setReduceFunction(HiveReduceFunction redFunc) { - this.reduceFunc = redFunc; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java deleted file mode 100644 index e8f39aeabb0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ /dev/null @@ -1,381 +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.hadoop.hive.ql.exec.spark; - -import com.google.common.base.Splitter; -import com.google.common.base.Strings; - -import java.io.IOException; -import java.io.Serializable; -import java.io.StringWriter; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hive.ql.io.NullScanFileSystem; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.conf.HiveConfUtil; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.DagUtils; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hive.spark.client.Job; -import org.apache.hive.spark.client.JobContext; -import org.apache.hive.spark.client.JobHandle; -import org.apache.hive.spark.client.SparkClient; -import org.apache.hive.spark.client.SparkClientFactory; -import org.apache.hive.spark.client.SparkClientUtilities; -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaPairRDD; - -/** - * RemoteSparkClient is a wrapper of {@link org.apache.hive.spark.client.SparkClient}, which - * wrap a spark job request and send to an remote SparkContext. - */ -public class RemoteHiveSparkClient implements HiveSparkClient { - private static final long serialVersionUID = 1L; - - private static final String MR_JAR_PROPERTY = "tmpjars"; - private static final String MR_CREDENTIALS_LOCATION_PROPERTY = "mapreduce.job.credentials.binary"; - private static final transient Logger LOG = LoggerFactory.getLogger(RemoteHiveSparkClient.class); - private static final transient Splitter CSV_SPLITTER = Splitter.on(",").omitEmptyStrings(); - - private transient Map conf; - private transient SparkClient remoteClient; - private transient SparkConf sparkConf; - private transient HiveConf hiveConf; - - private transient List localJars = new ArrayList(); - private transient List localFiles = new ArrayList(); - - private final transient long sparkClientTimtout; - private final String sessionId; - - RemoteHiveSparkClient(HiveConf hiveConf, Map conf, String sessionId) throws Exception { - this.hiveConf = hiveConf; - sparkClientTimtout = hiveConf.getTimeVar(HiveConf.ConfVars.SPARK_CLIENT_FUTURE_TIMEOUT, - TimeUnit.SECONDS); - sparkConf = HiveSparkClientFactory.generateSparkConf(conf); - this.conf = conf; - this.sessionId = sessionId; - createRemoteClient(); - } - - private void createRemoteClient() throws Exception { - remoteClient = SparkClientFactory.createClient(conf, hiveConf, sessionId); - - if (HiveConf.getBoolVar(hiveConf, ConfVars.HIVE_PREWARM_ENABLED) && - (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) || - SparkClientUtilities.isLocalMaster(hiveConf.get("spark.master")))) { - int minExecutors = getExecutorsToWarm(); - if (minExecutors <= 0) { - return; - } - - LOG.info("Prewarm Spark executors. The minimum number of executors to warm is " + minExecutors); - - // Spend at most HIVE_PREWARM_SPARK_TIMEOUT to wait for executors to come up. - int curExecutors = 0; - long maxPrewarmTime = HiveConf.getTimeVar(hiveConf, ConfVars.HIVE_PREWARM_SPARK_TIMEOUT, - TimeUnit.MILLISECONDS); - long ts = System.currentTimeMillis(); - do { - try { - curExecutors = getExecutorCount(maxPrewarmTime, TimeUnit.MILLISECONDS); - } catch (TimeoutException e) { - // let's don't fail on future timeout since we have a timeout for pre-warm - LOG.warn("Timed out getting executor count.", e); - } - if (curExecutors >= minExecutors) { - LOG.info("Finished prewarming Spark executors. The current number of executors is " + curExecutors); - return; - } - Thread.sleep(500); // sleep half a second - } while (System.currentTimeMillis() - ts < maxPrewarmTime); - - LOG.info("Timeout (" + maxPrewarmTime / 1000 + "s) occurred while prewarming executors. " + - "The current number of executors is " + curExecutors); - } - } - - /** - * Please note that the method is very tied with Spark documentation 1.4.1 regarding - * dynamic allocation, such as default values. - * @return - */ - private int getExecutorsToWarm() { - int minExecutors = - HiveConf.getIntVar(hiveConf, HiveConf.ConfVars.HIVE_PREWARM_NUM_CONTAINERS); - boolean dynamicAllocation = hiveConf.getBoolean("spark.dynamicAllocation.enabled", false); - if (dynamicAllocation) { - int min = sparkConf.getInt("spark.dynamicAllocation.minExecutors", 0); - int initExecutors = sparkConf.getInt("spark.dynamicAllocation.initialExecutors", min); - minExecutors = Math.min(minExecutors, initExecutors); - } else { - int execInstances = sparkConf.getInt("spark.executor.instances", 2); - minExecutors = Math.min(minExecutors, execInstances); - } - return minExecutors; - } - - private int getExecutorCount(long timeout, TimeUnit unit) throws Exception { - Future handler = remoteClient.getExecutorCount(); - return handler.get(timeout, unit); - } - - @Override - public SparkConf getSparkConf() { - return sparkConf; - } - - @Override - public int getExecutorCount() throws Exception { - return getExecutorCount(sparkClientTimtout, TimeUnit.SECONDS); - } - - @Override - public int getDefaultParallelism() throws Exception { - Future handler = remoteClient.getDefaultParallelism(); - return handler.get(sparkClientTimtout, TimeUnit.SECONDS); - } - - @Override - public SparkJobRef execute(final DriverContext driverContext, final SparkWork sparkWork) - throws Exception { - if (SparkClientUtilities.isYarnMaster(hiveConf.get("spark.master")) && - !remoteClient.isActive()) { - // Re-create the remote client if not active any more - close(); - createRemoteClient(); - } - - try { - return submit(driverContext, sparkWork); - } catch (Throwable cause) { - throw new Exception("Failed to submit Spark work, please retry later", cause); - } - } - - private SparkJobRef submit(final DriverContext driverContext, final SparkWork sparkWork) throws Exception { - final Context ctx = driverContext.getCtx(); - final HiveConf hiveConf = (HiveConf) ctx.getConf(); - refreshLocalResources(sparkWork, hiveConf); - final JobConf jobConf = new JobConf(hiveConf); - - //update the credential provider location in the jobConf - HiveConfUtil.updateJobCredentialProviders(jobConf); - - // Create temporary scratch dir - final Path emptyScratchDir = ctx.getMRTmpPath(); - FileSystem fs = emptyScratchDir.getFileSystem(jobConf); - fs.mkdirs(emptyScratchDir); - - // make sure NullScanFileSystem can be loaded - HIVE-18442 - jobConf.set("fs." + NullScanFileSystem.getBaseScheme() + ".impl", - NullScanFileSystem.class.getCanonicalName()); - - byte[] jobConfBytes = KryoSerializer.serializeJobConf(jobConf); - byte[] scratchDirBytes = KryoSerializer.serialize(emptyScratchDir); - byte[] sparkWorkBytes = KryoSerializer.serialize(sparkWork); - - JobStatusJob job = new JobStatusJob(jobConfBytes, scratchDirBytes, sparkWorkBytes); - if (driverContext.isShutdown()) { - throw new HiveException("Operation is cancelled."); - } - - JobHandle jobHandle = remoteClient.submit(job); - RemoteSparkJobStatus sparkJobStatus = new RemoteSparkJobStatus(remoteClient, jobHandle, sparkClientTimtout); - return new RemoteSparkJobRef(hiveConf, jobHandle, sparkJobStatus); - } - - private void refreshLocalResources(SparkWork sparkWork, HiveConf conf) throws IOException { - // add hive-exec jar - addJars((new JobConf(this.getClass())).getJar()); - - // add aux jars - addJars(conf.getAuxJars()); - addJars(SessionState.get() == null ? null : SessionState.get().getReloadableAuxJars()); - - // add added jars - String addedJars = Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDJARS, addedJars); - addJars(addedJars); - - // add plugin module jars on demand - // jobConf will hold all the configuration for hadoop, tez, and hive - JobConf jobConf = new JobConf(conf); - jobConf.set(MR_JAR_PROPERTY, ""); - for (BaseWork work : sparkWork.getAllWork()) { - work.configureJobConf(jobConf); - } - addJars(jobConf.get(MR_JAR_PROPERTY)); - - // remove the location of container tokens - conf.unset(MR_CREDENTIALS_LOCATION_PROPERTY); - // add added files - String addedFiles = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDFILES, addedFiles); - addResources(addedFiles); - - // add added archives - String addedArchives = Utilities.getResourceFiles(conf, SessionState.ResourceType.ARCHIVE); - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEADDEDARCHIVES, addedArchives); - addResources(addedArchives); - } - - private void addResources(String addedFiles) throws IOException { - for (String addedFile : CSV_SPLITTER.split(Strings.nullToEmpty(addedFiles))) { - try { - URI fileUri = FileUtils.getURI(addedFile); - if (fileUri != null && !localFiles.contains(fileUri)) { - localFiles.add(fileUri); - if (SparkUtilities.needUploadToHDFS(fileUri, sparkConf)) { - fileUri = SparkUtilities.uploadToHDFS(fileUri, hiveConf); - } - remoteClient.addFile(fileUri); - } - } catch (URISyntaxException e) { - LOG.warn("Failed to add file:" + addedFile, e); - } - } - } - - private void addJars(String addedJars) throws IOException { - for (String addedJar : CSV_SPLITTER.split(Strings.nullToEmpty(addedJars))) { - try { - URI jarUri = FileUtils.getURI(addedJar); - if (jarUri != null && !localJars.contains(jarUri)) { - localJars.add(jarUri); - if (SparkUtilities.needUploadToHDFS(jarUri, sparkConf)) { - jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf); - } - remoteClient.addJar(jarUri); - } - } catch (URISyntaxException e) { - LOG.warn("Failed to add jar:" + addedJar, e); - } - } - } - - @Override - public void close() { - if (remoteClient != null) { - remoteClient.stop(); - } - localFiles.clear(); - localJars.clear(); - } - - private static class JobStatusJob implements Job { - - private static final long serialVersionUID = 1L; - private final byte[] jobConfBytes; - private final byte[] scratchDirBytes; - private final byte[] sparkWorkBytes; - - @SuppressWarnings("unused") - private JobStatusJob() { - // For deserialization. - this(null, null, null); - } - - JobStatusJob(byte[] jobConfBytes, byte[] scratchDirBytes, byte[] sparkWorkBytes) { - this.jobConfBytes = jobConfBytes; - this.scratchDirBytes = scratchDirBytes; - this.sparkWorkBytes = sparkWorkBytes; - } - - @Override - public Serializable call(JobContext jc) throws Exception { - JobConf localJobConf = KryoSerializer.deserializeJobConf(jobConfBytes); - - // Add jar to current thread class loader dynamically, and add jar paths to JobConf as Spark - // may need to load classes from this jar in other threads. - Map addedJars = jc.getAddedJars(); - if (addedJars != null && !addedJars.isEmpty()) { - List localAddedJars = SparkClientUtilities.addToClassPath(addedJars, - localJobConf, jc.getLocalTmpDir()); - localJobConf.set(Utilities.HIVE_ADDED_JARS, StringUtils.join(localAddedJars, ";")); - } - - Path localScratchDir = KryoSerializer.deserialize(scratchDirBytes, Path.class); - SparkWork localSparkWork = KryoSerializer.deserialize(sparkWorkBytes, SparkWork.class); - logConfigurations(localJobConf); - - SparkCounters sparkCounters = new SparkCounters(jc.sc()); - Map> prefixes = localSparkWork.getRequiredCounterPrefix(); - if (prefixes != null) { - for (String group : prefixes.keySet()) { - for (String counterName : prefixes.get(group)) { - sparkCounters.createCounter(group, counterName); - } - } - } - SparkReporter sparkReporter = new SparkReporter(sparkCounters); - - // Generate Spark plan - SparkPlanGenerator gen = - new SparkPlanGenerator(jc.sc(), null, localJobConf, localScratchDir, sparkReporter); - SparkPlan plan = gen.generate(localSparkWork); - - jc.sc().setJobGroup("queryId = " + localSparkWork.getQueryId(), DagUtils.getQueryName(localJobConf)); - - // Execute generated plan. - JavaPairRDD finalRDD = plan.generateGraph(); - // We use Spark RDD async action to submit job as it's the only way to get jobId now. - JavaFutureAction future = finalRDD.foreachAsync(HiveVoidFunction.getInstance()); - jc.monitor(future, sparkCounters, plan.getCachedRDDIds()); - return null; - } - - private void logConfigurations(JobConf localJobConf) { - if (LOG.isDebugEnabled()) { - LOG.debug("Logging job configuration: "); - StringBuilder outWriter = new StringBuilder(); - // redact sensitive information before logging - HiveConfUtil.dumpConfig(localJobConf, outWriter); - LOG.debug(outWriter.toString()); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ShuffleTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ShuffleTran.java deleted file mode 100644 index 40ff01ab7a0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/ShuffleTran.java +++ /dev/null @@ -1,77 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.storage.StorageLevel; - -public class ShuffleTran implements SparkTran { - private final SparkShuffler shuffler; - private final int numOfPartitions; - private final boolean toCache; - private final SparkPlan sparkPlan; - private final String name; - private final SparkEdgeProperty edge; - - public ShuffleTran(SparkPlan sparkPlan, SparkShuffler sf, int n) { - this(sparkPlan, sf, n, false, "Shuffle", null); - } - - public ShuffleTran(SparkPlan sparkPlan, SparkShuffler sf, int n, boolean toCache, String name, - SparkEdgeProperty edge) { - shuffler = sf; - numOfPartitions = n; - this.toCache = toCache; - this.sparkPlan = sparkPlan; - this.name = name; - this.edge = edge; - } - - @Override - public JavaPairRDD transform(JavaPairRDD input) { - JavaPairRDD result = shuffler.shuffle(input, numOfPartitions); - if (toCache) { - sparkPlan.addCachedRDDId(result.id()); - result = result.persist(StorageLevel.MEMORY_AND_DISK()); - } - return result.setName(this.name + " (" + edge.getShuffleType() + ", " + numOfPartitions + - (toCache ? ", cached)" : ")")); - } - - public int getNoOfPartitions() { - return numOfPartitions; - } - - @Override - public String getName() { - return name; - } - - @Override - public Boolean isCacheEnable() { - return new Boolean(toCache); - } - - public SparkShuffler getShuffler() { - return shuffler; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SmallTableCache.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SmallTableCache.java deleted file mode 100644 index 3293100af96..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SmallTableCache.java +++ /dev/null @@ -1,73 +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.hadoop.hive.ql.exec.spark; - -import java.util.concurrent.ConcurrentHashMap; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer; - -public class SmallTableCache { - private static final Logger LOG = LoggerFactory.getLogger(SmallTableCache.class.getName()); - - private static final ConcurrentHashMap - tableContainerMap = new ConcurrentHashMap(); - private static volatile String queryId; - - /** - * Check if this is a new query. If so, clean up the cache - * that is for the previous query, and reset the current query id. - */ - public static void initialize(Configuration conf) { - String currentQueryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname); - if (!currentQueryId.equals(queryId)) { - if (!tableContainerMap.isEmpty()) { - synchronized (tableContainerMap) { - if (!currentQueryId.equals(queryId) && !tableContainerMap.isEmpty()) { - for (MapJoinTableContainer tableContainer: tableContainerMap.values()) { - tableContainer.clear(); - } - tableContainerMap.clear(); - if (LOG.isDebugEnabled()) { - LOG.debug("Cleaned up small table cache for query " + queryId); - } - } - } - } - queryId = currentQueryId; - } - } - - public static void cache(Path path, MapJoinTableContainer tableContainer) { - if (tableContainerMap.putIfAbsent(path, tableContainer) == null && LOG.isDebugEnabled()) { - LOG.debug("Cached small table file " + path + " for query " + queryId); - } - } - - public static MapJoinTableContainer get(Path path) { - MapJoinTableContainer tableContainer = tableContainerMap.get(path); - if (tableContainer != null && LOG.isDebugEnabled()) { - LOG.debug("Loaded small table file " + path + " from cache for query " + queryId); - } - return tableContainer; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.java deleted file mode 100644 index 22b598f0b45..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SortByShuffler.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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.HashPartitioner; -import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.storage.StorageLevel; - -public class SortByShuffler implements SparkShuffler { - - private final boolean totalOrder; - private final SparkPlan sparkPlan; - - /** - * @param totalOrder whether this shuffler provides total order shuffle. - */ - public SortByShuffler(boolean totalOrder, SparkPlan sparkPlan) { - this.totalOrder = totalOrder; - this.sparkPlan = sparkPlan; - } - - @Override - public JavaPairRDD shuffle( - JavaPairRDD input, int numPartitions) { - JavaPairRDD rdd; - if (totalOrder) { - if (numPartitions > 0) { - if (numPartitions > 1 && input.getStorageLevel() == StorageLevel.NONE()) { - input.persist(StorageLevel.DISK_ONLY()); - sparkPlan.addCachedRDDId(input.id()); - } - rdd = input.sortByKey(true, numPartitions); - } else { - rdd = input.sortByKey(true); - } - } else { - Partitioner partitioner = new HashPartitioner(numPartitions); - rdd = input.repartitionAndSortWithinPartitions(partitioner); - } - return rdd; - } - - @Override - public String getName() { - return "SortBy"; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java deleted file mode 100644 index ed889fad2d6..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java +++ /dev/null @@ -1,277 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.io.ObjectInputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import com.clearspring.analytics.util.Preconditions; -import javolution.testing.AssertionException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator; -import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluatorFactory; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.PartitionDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.ReflectionUtils; - -/** - * The spark version of DynamicPartitionPruner. - */ -public class SparkDynamicPartitionPruner { - private static final Logger LOG = LoggerFactory.getLogger(SparkDynamicPartitionPruner.class); - private final Map> sourceInfoMap = new LinkedHashMap>(); - private final BytesWritable writable = new BytesWritable(); - - public void prune(MapWork work, JobConf jobConf) throws HiveException, SerDeException { - sourceInfoMap.clear(); - initialize(work, jobConf); - if (sourceInfoMap.size() == 0) { - // Nothing to prune for this MapWork - return; - } - processFiles(work, jobConf); - prunePartitions(work); - } - - public void initialize(MapWork work, JobConf jobConf) throws SerDeException { - Map columnMap = new HashMap(); - Set sourceWorkIds = work.getEventSourceTableDescMap().keySet(); - - for (String id : sourceWorkIds) { - List tables = work.getEventSourceTableDescMap().get(id); - // Real column name - on which the operation is being performed - List columnNames = work.getEventSourceColumnNameMap().get(id); - // Column type - List columnTypes = work.getEventSourceColumnTypeMap().get(id); - List partKeyExprs = work.getEventSourcePartKeyExprMap().get(id); - - Iterator cit = columnNames.iterator(); - Iterator typit = columnTypes.iterator(); - Iterator pit = partKeyExprs.iterator(); - for (TableDesc t : tables) { - String columnName = cit.next(); - String columnType = typit.next(); - ExprNodeDesc partKeyExpr = pit.next(); - SourceInfo si = new SourceInfo(t, partKeyExpr, columnName, columnType, jobConf); - if (!sourceInfoMap.containsKey(id)) { - sourceInfoMap.put(id, new ArrayList()); - } - sourceInfoMap.get(id).add(si); - - // We could have multiple sources restrict the same column, need to take - // the union of the values in that case. - if (columnMap.containsKey(columnName)) { - si.values = columnMap.get(columnName).values; - } - columnMap.put(columnName, si); - } - } - } - - private void processFiles(MapWork work, JobConf jobConf) throws HiveException { - ObjectInputStream in = null; - try { - Path baseDir = work.getTmpPathForPartitionPruning(); - FileSystem fs = FileSystem.get(baseDir.toUri(), jobConf); - - // Find the SourceInfo to put values in. - for (String name : sourceInfoMap.keySet()) { - Path sourceDir = new Path(baseDir, name); - for (FileStatus fstatus : fs.listStatus(sourceDir)) { - LOG.info("Start processing pruning file: " + fstatus.getPath()); - in = new ObjectInputStream(fs.open(fstatus.getPath())); - final int numName = in.readInt(); - Set columnNames = new HashSet<>(); - for (int i = 0; i < numName; i++) { - columnNames.add(in.readUTF()); - } - - // make sure the dpp sink has output for all the corresponding part columns - for (SourceInfo si : sourceInfoMap.get(name)) { - Preconditions.checkArgument(columnNames.contains(si.columnName), - "AssertionError: no output for column " + si.columnName); - } - - // Read dpp outputs - while (in.available() > 0) { - writable.readFields(in); - - for (SourceInfo info : sourceInfoMap.get(name)) { - Object row = info.deserializer.deserialize(writable); - Object value = info.soi.getStructFieldData(row, info.field); - value = ObjectInspectorUtils.copyToStandardObject(value, info.fieldInspector); - info.values.add(value); - } - } - } - } - } catch (Exception e) { - throw new HiveException(e); - } finally { - try { - if (in != null) { - in.close(); - } - } catch (IOException e) { - throw new HiveException("error while trying to close input stream", e); - } - } - } - - private void prunePartitions(MapWork work) throws HiveException { - for (String source : sourceInfoMap.keySet()) { - for (SourceInfo info : sourceInfoMap.get(source)) { - prunePartitionSingleSource(info, work); - } - } - } - - private void prunePartitionSingleSource(SourceInfo info, MapWork work) - throws HiveException { - Set values = info.values; - // strip the column name of the targetId - String columnName = info.columnName.substring(info.columnName.indexOf(':') + 1); - - ObjectInspector oi = - PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory - .getPrimitiveTypeInfo(info.columnType)); - - ObjectInspectorConverters.Converter converter = - ObjectInspectorConverters.getConverter( - PrimitiveObjectInspectorFactory.javaStringObjectInspector, oi); - - StructObjectInspector soi = - ObjectInspectorFactory.getStandardStructObjectInspector( - Collections.singletonList(columnName), Collections.singletonList(oi)); - - @SuppressWarnings("rawtypes") - ExprNodeEvaluator eval = ExprNodeEvaluatorFactory.get(info.partKey); - eval.initialize(soi); - - applyFilterToPartitions(work, converter, eval, columnName, values); - } - - private void applyFilterToPartitions( - MapWork work, - ObjectInspectorConverters.Converter converter, - ExprNodeEvaluator eval, - String columnName, - Set values) throws HiveException { - - Object[] row = new Object[1]; - - Iterator it = work.getPathToPartitionInfo().keySet().iterator(); - while (it.hasNext()) { - Path p = it.next(); - PartitionDesc desc = work.getPathToPartitionInfo().get(p); - Map spec = desc.getPartSpec(); - if (spec == null) { - throw new AssertionException("No partition spec found in dynamic pruning"); - } - - String partValueString = spec.get(columnName); - if (partValueString == null) { - throw new AssertionException("Could not find partition value for column: " + columnName); - } - - Object partValue = converter.convert(partValueString); - if (LOG.isDebugEnabled()) { - LOG.debug("Converted partition value: " + partValue + " original (" + partValueString + ")"); - } - - row[0] = partValue; - partValue = eval.evaluate(row); - if (LOG.isDebugEnabled()) { - LOG.debug("part key expr applied: " + partValue); - } - - if (!values.contains(partValue)) { - LOG.info("Pruning path: " + p); - it.remove(); - work.removePathToAlias(p); - // HIVE-12244 call currently ineffective - work.getPartitionDescs().remove(desc); - } - } - } - - @SuppressWarnings("deprecation") - private static class SourceInfo { - final ExprNodeDesc partKey; - final Deserializer deserializer; - final StructObjectInspector soi; - final StructField field; - final ObjectInspector fieldInspector; - Set values = new HashSet(); - final String columnName; - final String columnType; - - SourceInfo(TableDesc table, ExprNodeDesc partKey, String columnName, String columnType, JobConf jobConf) - throws SerDeException { - this.partKey = partKey; - this.columnName = columnName; - this.columnType = columnType; - - deserializer = ReflectionUtils.newInstance(table.getDeserializerClass(), null); - deserializer.initialize(jobConf, table.getProperties()); - - ObjectInspector inspector = deserializer.getObjectInspector(); - if (LOG.isDebugEnabled()) { - LOG.debug("Type of obj insp: " + inspector.getTypeName()); - } - - soi = (StructObjectInspector) inspector; - List fields = soi.getAllStructFieldRefs(); - assert(fields.size() > 1) : "expecting single field in input"; - - field = fields.get(0); - fieldInspector = - ObjectInspectorUtils.getStandardObjectInspector(field.getFieldObjectInspector()); - } - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java deleted file mode 100644 index 88dd12c05ad..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java +++ /dev/null @@ -1,215 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; - -import org.apache.hadoop.hive.ql.plan.PartitionDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.AbstractMapOperator; -import org.apache.hadoop.hive.ql.exec.MapOperator; -import org.apache.hadoop.hive.ql.exec.MapredContext; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.ReportStats; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; -import org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; - - -/** - * Clone from ExecMapper. SparkMapRecordHandler is the bridge between the spark framework and - * the Hive operator pipeline at execution time. It's main responsibilities are: - * - * - Load and setup the operator pipeline from XML - * - Run the pipeline by transforming key value pairs to records and forwarding them to the operators - * - Stop execution when the "limit" is reached - * - Catch and handle errors during execution of the operators. - * - */ -public class SparkMapRecordHandler extends SparkRecordHandler { - private static final Logger LOG = LoggerFactory.getLogger(SparkMapRecordHandler.class); - private AbstractMapOperator mo; - private MapredLocalWork localWork = null; - private ExecMapperContext execContext; - - @Override - public void init(JobConf job, OutputCollector output, Reporter reporter) throws Exception { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); - super.init(job, output, reporter); - - try { - jc = job; - execContext = new ExecMapperContext(jc); - // create map and fetch operators - MapWork mrwork = Utilities.getMapWork(job); - for (PartitionDesc part : mrwork.getAliasToPartnInfo().values()) { - TableDesc tableDesc = part.getTableDesc(); - Utilities.copyJobSecretToTableProperties(tableDesc); - } - - CompilationOpContext runtimeCtx = new CompilationOpContext(); - if (mrwork.getVectorMode()) { - mo = new VectorMapOperator(runtimeCtx); - } else { - mo = new MapOperator(runtimeCtx); - } - mo.setConf(mrwork); - - // initialize map operator - mo.initialize(jc, null); - mo.setChildren(job); - LOG.info(mo.dump(0)); - // initialize map local work - localWork = mrwork.getMapRedLocalWork(); - execContext.setLocalWork(localWork); - - MapredContext.init(true, new JobConf(jc)); - MapredContext.get().setReporter(reporter); - - mo.passExecContext(execContext); - mo.initializeLocalWork(jc); - mo.initializeMapOperator(jc); - - mo.setReporter(rp); - - if (localWork == null) { - return; - } - - //The following code is for mapjoin - //initialize all the dummy ops - LOG.info("Initializing dummy operator"); - List> dummyOps = localWork.getDummyParentOp(); - for (Operator dummyOp : dummyOps) { - dummyOp.setExecContext(execContext); - dummyOp.initialize(jc, null); - } - } catch (Throwable e) { - abort = true; - if (e instanceof OutOfMemoryError) { - // will this be true here? - // Don't create a new object if we are already out of memory - throw (OutOfMemoryError) e; - } else { - throw new RuntimeException("Map operator initialization failed: " + e, e); - } - } - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); - } - - @Override - public void processRow(Object key, Object value) throws IOException { - if (!anyRow) { - OperatorUtils.setChildrenCollector(mo.getChildOperators(), oc); - anyRow = true; - } - // reset the execContext for each new row - execContext.resetRow(); - - try { - // Since there is no concept of a group, we don't invoke - // startGroup/endGroup for a mapper - mo.process((Writable) value); - if (LOG.isInfoEnabled()) { - logMemoryInfo(); - } - } catch (Throwable e) { - abort = true; - Utilities.setMapWork(jc, null); - if (e instanceof OutOfMemoryError) { - // Don't create a new object if we are already out of memory - throw (OutOfMemoryError) e; - } else { - String msg = "Error processing row: " + e; - LOG.error(msg, e); - throw new RuntimeException(msg, e); - } - } - } - - @Override - public void processRow(Object key, Iterator values) throws IOException { - throw new UnsupportedOperationException("Do not support this method in SparkMapRecordHandler."); - } - - @Override - public void close() { - // No row was processed - if (!anyRow) { - LOG.trace("Close called. no row processed by map."); - } - - // check if there are IOExceptions - if (!abort) { - abort = execContext.getIoCxt().getIOExceptions(); - } - - // detecting failed executions by exceptions thrown by the operator tree - // ideally hadoop should let us know whether map execution failed or not - try { - mo.close(abort); - - //for close the local work - if (localWork != null) { - List> dummyOps = localWork.getDummyParentOp(); - - for (Operator dummyOp : dummyOps) { - dummyOp.close(abort); - } - } - - if (LOG.isInfoEnabled()) { - logCloseInfo(); - } - - ReportStats rps = new ReportStats(rp, jc); - mo.preorderMap(rps); - return; - } catch (Exception e) { - if (!abort) { - // signal new failure to map-reduce - String msg = "Hit error while closing operators - failing tree: " + e; - LOG.error(msg, e); - throw new IllegalStateException(msg, e); - } - } finally { - MapredContext.close(); - Utilities.clearWorkMap(jc); - } - } - - @Override - public boolean getDone() { - return mo.getDone(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java deleted file mode 100644 index 8880bb604e0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMergeFileRecordHandler.java +++ /dev/null @@ -1,123 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.Iterator; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.FileMergeDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; - -import com.google.common.base.Preconditions; - -/** - * Copied from MergeFileMapper. - * - * As MergeFileMapper is very similar to ExecMapper, this class is - * very similar to SparkMapRecordHandler - */ -public class SparkMergeFileRecordHandler extends SparkRecordHandler { - - private static final String PLAN_KEY = "__MAP_PLAN__"; - private static final Logger LOG = LoggerFactory.getLogger(SparkMergeFileRecordHandler.class); - private Operator op; - private AbstractFileMergeOperator mergeOp; - private Object[] row; - - @SuppressWarnings("unchecked") - @Override - public void init(JobConf job, OutputCollector output, Reporter reporter) throws Exception { - super.init(job, output, reporter); - - try { - jc = job; - - MapWork mapWork = Utilities.getMapWork(job); - - if (mapWork instanceof MergeFileWork) { - MergeFileWork mergeFileWork = (MergeFileWork) mapWork; - String alias = mergeFileWork.getAliasToWork().keySet().iterator().next(); - op = mergeFileWork.getAliasToWork().get(alias); - if (op instanceof AbstractFileMergeOperator) { - mergeOp = (AbstractFileMergeOperator) op; - mergeOp.initializeOp(jc); - row = new Object[2]; - abort = false; - } else { - abort = true; - throw new IllegalStateException( - "Merge file work's top operator should be an" - + " instance of AbstractFileMergeOperator"); - } - } else { - abort = true; - throw new IllegalStateException("Map work should be a merge file work."); - } - - LOG.info(mergeOp.dump(0)); - } catch (HiveException e) { - abort = true; - throw new RuntimeException(e); - } - } - - @Override - public void processRow(Object key, Object value) throws IOException { - row[0] = key; - row[1] = value; - try { - mergeOp.process(row, 0); - } catch (HiveException e) { - abort = true; - throw new IOException(e); - } - } - - @Override - public void processRow(Object key, Iterator values) throws IOException { - throw new UnsupportedOperationException("Do not support this method in " - + this.getClass().getSimpleName()); - } - - @Override - public void close() { - LOG.info("Closing Merge Operator " + mergeOp.getName()); - try { - mergeOp.closeOp(abort); - } catch (HiveException e) { - throw new RuntimeException(e); - } - } - - @Override - public boolean getDone() { - return mergeOp.getDone(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java deleted file mode 100644 index b21e3865f68..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlan.java +++ /dev/null @@ -1,192 +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.hadoop.hive.ql.exec.spark; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.spark.SparkContext; -import org.apache.spark.util.CallSite; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; - -import com.google.common.base.Preconditions; - -@SuppressWarnings("rawtypes") -public class SparkPlan { - private static final String CLASS_NAME = SparkPlan.class.getName(); - private static final Logger LOG = LoggerFactory.getLogger(SparkPlan.class); - private final PerfLogger perfLogger = SessionState.getPerfLogger(); - - private final Set rootTrans = new HashSet(); - private final Set leafTrans = new HashSet(); - private final Map> transGraph = new HashMap>(); - private final Map> invertedTransGraph = new HashMap>(); - private final Set cachedRDDIds = new HashSet(); - - private final SparkContext sc; - - SparkPlan(SparkContext sc) { - this.sc = sc; - } - - @SuppressWarnings("unchecked") - public JavaPairRDD generateGraph() { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_BUILD_RDD_GRAPH); - Map> tranToOutputRDDMap - = new HashMap>(); - for (SparkTran tran : getAllTrans()) { - JavaPairRDD rdd = null; - List parents = getParents(tran); - if (parents.size() == 0) { - // Root tran, it must be MapInput - Preconditions.checkArgument(tran instanceof MapInput, - "AssertionError: tran must be an instance of MapInput"); - sc.setCallSite(CallSite.apply(tran.getName(), "")); - rdd = tran.transform(null); - } else { - for (SparkTran parent : parents) { - JavaPairRDD prevRDD = tranToOutputRDDMap.get(parent); - if (rdd == null) { - rdd = prevRDD; - } else { - sc.setCallSite(CallSite.apply("UnionRDD (" + rdd.name() + ", " + - prevRDD.name() + ")", "")); - rdd = rdd.union(prevRDD); - rdd.setName("UnionRDD (" + rdd.getNumPartitions() + ")"); - } - } - sc.setCallSite(CallSite.apply(tran.getName(), "")); - rdd = tran.transform(rdd); - } - - tranToOutputRDDMap.put(tran, rdd); - } - - JavaPairRDD finalRDD = null; - for (SparkTran leafTran : leafTrans) { - JavaPairRDD rdd = tranToOutputRDDMap.get(leafTran); - if (finalRDD == null) { - finalRDD = rdd; - } else { - sc.setCallSite(CallSite.apply("UnionRDD (" + rdd.name() + ", " + finalRDD.name() + ")", - "")); - finalRDD = finalRDD.union(rdd); - finalRDD.setName("UnionRDD (" + finalRDD.getNumPartitions() + ")"); - } - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_BUILD_RDD_GRAPH); - - LOG.info("\n\nSpark RDD Graph:\n\n" + finalRDD.toDebugString() + "\n"); - - return finalRDD; - } - - public void addTran(SparkTran tran) { - rootTrans.add(tran); - leafTrans.add(tran); - } - - public void addCachedRDDId(int rddId) { - cachedRDDIds.add(rddId); - } - - public Set getCachedRDDIds() { - return cachedRDDIds; - } - - /** - * This method returns a topologically sorted list of SparkTran. - */ - private List getAllTrans() { - List result = new LinkedList(); - Set seen = new HashSet(); - - for (SparkTran leaf: leafTrans) { - // make sure all leaves are visited at least once - visit(leaf, seen, result); - } - - return result; - } - - private void visit(SparkTran child, Set seen, List result) { - if (seen.contains(child)) { - // don't visit multiple times - return; - } - - seen.add(child); - - for (SparkTran parent: getParents(child)) { - if (!seen.contains(parent)) { - visit(parent, seen, result); - } - } - - result.add(child); - } - - /** - * Connects the two SparkTrans in the graph. Does not allow multiple connections - * between the same pair of SparkTrans. - * @param parent - * @param child - */ - public void connect(SparkTran parent, SparkTran child) { - rootTrans.remove(child); - leafTrans.remove(parent); - if (transGraph.get(parent) == null) { - transGraph.put(parent, new LinkedList()); - } - if (invertedTransGraph.get(child) == null) { - invertedTransGraph.put(child, new LinkedList()); - } - transGraph.get(parent).add(child); - invertedTransGraph.get(child).add(parent); - } - - public List getParents(SparkTran tran) { - if (!invertedTransGraph.containsKey(tran)) { - return new ArrayList(); - } - - return invertedTransGraph.get(tran); - } - - public List getChildren(SparkTran tran) { - if (!transGraph.containsKey(tran)) { - return new ArrayList(); - } - - return transGraph.get(tran); - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java deleted file mode 100644 index c9a3196126d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkPlanGenerator.java +++ /dev/null @@ -1,378 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -import org.apache.spark.util.CallSite; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.ql.io.merge.MergeFileMapper; -import org.apache.hadoop.hive.ql.io.merge.MergeFileOutputFormat; -import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.mapred.FileOutputFormat; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; -import org.apache.hadoop.hive.ql.exec.mr.ExecReducer; -import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.TableScanDesc; -import org.apache.hadoop.hive.ql.stats.StatsCollectionContext; -import org.apache.hadoop.hive.ql.stats.StatsFactory; -import org.apache.hadoop.hive.ql.stats.StatsPublisher; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaSparkContext; - -import com.google.common.base.Preconditions; - -@SuppressWarnings("rawtypes") -public class SparkPlanGenerator { - private static final String CLASS_NAME = SparkPlanGenerator.class.getName(); - private final PerfLogger perfLogger = SessionState.getPerfLogger(); - private static final Logger LOG = LoggerFactory.getLogger(SparkPlanGenerator.class); - - private final JavaSparkContext sc; - private final JobConf jobConf; - private final Context context; - private final Path scratchDir; - private final SparkReporter sparkReporter; - private Map cloneToWork; - private final Map workToTranMap; - private final Map workToParentWorkTranMap; - // a map from each BaseWork to its cloned JobConf - private final Map workToJobConf; - - public SparkPlanGenerator( - JavaSparkContext sc, - Context context, - JobConf jobConf, - Path scratchDir, - SparkReporter sparkReporter) { - - this.sc = sc; - this.context = context; - this.jobConf = jobConf; - this.scratchDir = scratchDir; - this.workToTranMap = new HashMap(); - this.workToParentWorkTranMap = new HashMap(); - this.sparkReporter = sparkReporter; - this.workToJobConf = new HashMap(); - } - - public SparkPlan generate(SparkWork sparkWork) throws Exception { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_BUILD_PLAN); - SparkPlan sparkPlan = new SparkPlan(this.sc.sc()); - cloneToWork = sparkWork.getCloneToWork(); - workToTranMap.clear(); - workToParentWorkTranMap.clear(); - - try { - for (BaseWork work : sparkWork.getAllWork()) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); - SparkTran tran = generate(work, sparkWork); - SparkTran parentTran = generateParentTran(sparkPlan, sparkWork, work); - sparkPlan.addTran(tran); - sparkPlan.connect(parentTran, tran); - workToTranMap.put(work, tran); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_CREATE_TRAN + work.getName()); - } - } finally { - // clear all ThreadLocal cached MapWork/ReduceWork after plan generation - // as this may executed in a pool thread. - Utilities.clearWorkMap(jobConf); - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_BUILD_PLAN); - return sparkPlan; - } - - // Generate (possibly get from a cached result) parent SparkTran - private SparkTran generateParentTran(SparkPlan sparkPlan, SparkWork sparkWork, - BaseWork work) throws Exception { - if (cloneToWork.containsKey(work)) { - BaseWork originalWork = cloneToWork.get(work); - if (workToParentWorkTranMap.containsKey(originalWork)) { - return workToParentWorkTranMap.get(originalWork); - } - } - - SparkTran result; - if (work instanceof MapWork) { - result = generateMapInput(sparkPlan, (MapWork)work); - sparkPlan.addTran(result); - } else if (work instanceof ReduceWork) { - boolean toCache = cloneToWork.containsKey(work); - List parentWorks = sparkWork.getParents(work); - SparkEdgeProperty sparkEdgeProperty = sparkWork.getEdgeProperty(parentWorks.get(0), work); - result = generate(sparkPlan, sparkEdgeProperty, toCache, work.getName()); - sparkPlan.addTran(result); - for (BaseWork parentWork : parentWorks) { - sparkPlan.connect(workToTranMap.get(parentWork), result); - } - } else { - throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " - + "but found " + work.getClass().getName()); - } - - if (cloneToWork.containsKey(work)) { - workToParentWorkTranMap.put(cloneToWork.get(work), result); - } - - return result; - } - - private Class getInputFormat(JobConf jobConf, MapWork mWork) throws HiveException { - // MergeFileWork is sub-class of MapWork, we don't need to distinguish here - if (mWork.getInputformat() != null) { - HiveConf.setVar(jobConf, HiveConf.ConfVars.HIVEINPUTFORMAT, - mWork.getInputformat()); - } - String inpFormat = HiveConf.getVar(jobConf, - HiveConf.ConfVars.HIVEINPUTFORMAT); - - if (mWork.isUseBucketizedHiveInputFormat()) { - inpFormat = BucketizedHiveInputFormat.class.getName(); - } - - Class inputFormatClass; - try { - inputFormatClass = JavaUtils.loadClass(inpFormat); - } catch (ClassNotFoundException e) { - String message = "Failed to load specified input format class:" - + inpFormat; - LOG.error(message, e); - throw new HiveException(message, e); - } - - return inputFormatClass; - } - - @SuppressWarnings("unchecked") - private MapInput generateMapInput(SparkPlan sparkPlan, MapWork mapWork) - throws Exception { - JobConf jobConf = cloneJobConf(mapWork); - Class ifClass = getInputFormat(jobConf, mapWork); - - sc.sc().setCallSite(CallSite.apply(mapWork.getName(), "")); - - JavaPairRDD hadoopRDD; - if (mapWork.getNumMapTasks() != null) { - jobConf.setNumMapTasks(mapWork.getNumMapTasks()); - hadoopRDD = sc.hadoopRDD(jobConf, ifClass, - WritableComparable.class, Writable.class, mapWork.getNumMapTasks()); - } else { - hadoopRDD = sc.hadoopRDD(jobConf, ifClass, WritableComparable.class, Writable.class); - } - - boolean toCache = false/*cloneToWork.containsKey(mapWork)*/; - - String tables = mapWork.getAllRootOperators().stream() - .filter(op -> op instanceof TableScanOperator) - .map(ts -> ((TableScanDesc) ts.getConf()).getAlias()) - .collect(Collectors.joining(", ")); - - String rddName = mapWork.getName() + " (" + tables + ", " + hadoopRDD.getNumPartitions() + - (toCache ? ", cached)" : ")"); - - // Caching is disabled for MapInput due to HIVE-8920 - MapInput result = new MapInput(sparkPlan, hadoopRDD, toCache, rddName); - return result; - } - - private ShuffleTran generate(SparkPlan sparkPlan, SparkEdgeProperty edge, boolean toCache, - String name) { - - Preconditions.checkArgument(!edge.isShuffleNone(), - "AssertionError: SHUFFLE_NONE should only be used for UnionWork."); - SparkShuffler shuffler; - if (edge.isMRShuffle()) { - shuffler = new SortByShuffler(false, sparkPlan); - } else if (edge.isShuffleSort()) { - shuffler = new SortByShuffler(true, sparkPlan); - } else { - shuffler = new GroupByShuffler(); - } - return new ShuffleTran(sparkPlan, shuffler, edge.getNumPartitions(), toCache, name, edge); - } - - private SparkTran generate(BaseWork work, SparkWork sparkWork) throws Exception { - initStatsPublisher(work); - JobConf newJobConf = cloneJobConf(work); - checkSpecs(work, newJobConf); - byte[] confBytes = KryoSerializer.serializeJobConf(newJobConf); - boolean caching = isCachingWork(work, sparkWork); - if (work instanceof MapWork) { - // Create tmp dir for MergeFileWork - if (work instanceof MergeFileWork) { - Path outputPath = ((MergeFileWork) work).getOutputDir(); - Path tempOutPath = Utilities.toTempPath(outputPath); - FileSystem fs = outputPath.getFileSystem(jobConf); - try { - if (!fs.exists(tempOutPath)) { - fs.mkdirs(tempOutPath); - } - } catch (IOException e) { - throw new RuntimeException( - "Can't make path " + outputPath + " : " + e.getMessage()); - } - } - MapTran mapTran = new MapTran(caching, work.getName()); - HiveMapFunction mapFunc = new HiveMapFunction(confBytes, sparkReporter); - mapTran.setMapFunction(mapFunc); - return mapTran; - } else if (work instanceof ReduceWork) { - ReduceTran reduceTran = new ReduceTran(caching, work.getName()); - HiveReduceFunction reduceFunc = new HiveReduceFunction(confBytes, sparkReporter); - reduceTran.setReduceFunction(reduceFunc); - return reduceTran; - } else { - throw new IllegalStateException("AssertionError: expected either MapWork or ReduceWork, " - + "but found " + work.getClass().getName()); - } - } - - private boolean isCachingWork(BaseWork work, SparkWork sparkWork) { - boolean caching = true; - List children = sparkWork.getChildren(work); - if (children.size() < 2) { - caching = false; - } else { - // do not cache this if its child RDD is intend to be cached. - for (BaseWork child : children) { - if (cloneToWork.containsKey(child)) { - caching = false; - } - } - } - return caching; - } - - private void checkSpecs(BaseWork work, JobConf jc) throws Exception { - Set> opList = work.getAllOperators(); - for (Operator op : opList) { - if (op instanceof FileSinkOperator) { - ((FileSinkOperator) op).checkOutputSpecs(null, jc); - } - } - } - - @SuppressWarnings({ "unchecked" }) - private JobConf cloneJobConf(BaseWork work) throws Exception { - if (workToJobConf.containsKey(work)) { - return workToJobConf.get(work); - } - JobConf cloned = new JobConf(jobConf); - // Make sure we'll use a different plan path from the original one - HiveConf.setVar(cloned, HiveConf.ConfVars.PLAN, ""); - try { - cloned.setPartitionerClass(JavaUtils.loadClass(HiveConf.getVar(cloned, HiveConf.ConfVars.HIVEPARTITIONER))); - } catch (ClassNotFoundException e) { - String msg = "Could not find partitioner class: " + e.getMessage() - + " which is specified by: " + HiveConf.ConfVars.HIVEPARTITIONER.varname; - throw new IllegalArgumentException(msg, e); - } - if (work instanceof MapWork) { - MapWork mapWork = (MapWork) work; - cloned.setBoolean("mapred.task.is.map", true); - List inputPaths = Utilities.getInputPaths(cloned, mapWork, - scratchDir, context, false); - Utilities.setInputPaths(cloned, inputPaths); - Utilities.setMapWork(cloned, mapWork, scratchDir, false); - Utilities.createTmpDirs(cloned, mapWork); - if (work instanceof MergeFileWork) { - MergeFileWork mergeFileWork = (MergeFileWork) work; - cloned.set(Utilities.MAPRED_MAPPER_CLASS, MergeFileMapper.class.getName()); - cloned.set("mapred.input.format.class", mergeFileWork.getInputformat()); - cloned.setClass("mapred.output.format.class", MergeFileOutputFormat.class, - FileOutputFormat.class); - } else { - cloned.set(Utilities.MAPRED_MAPPER_CLASS, ExecMapper.class.getName()); - } - if (mapWork.getMaxSplitSize() != null) { - HiveConf.setLongVar(cloned, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, - mapWork.getMaxSplitSize()); - } - if (mapWork.getMinSplitSize() != null) { - HiveConf.setLongVar(cloned, HiveConf.ConfVars.MAPREDMINSPLITSIZE, - mapWork.getMinSplitSize()); - } - if (mapWork.getMinSplitSizePerNode() != null) { - HiveConf.setLongVar(cloned, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, - mapWork.getMinSplitSizePerNode()); - } - if (mapWork.getMinSplitSizePerRack() != null) { - HiveConf.setLongVar(cloned, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, - mapWork.getMinSplitSizePerRack()); - } - // remember the JobConf cloned for each MapWork, so we won't clone for it again - workToJobConf.put(work, cloned); - } else if (work instanceof ReduceWork) { - cloned.setBoolean("mapred.task.is.map", false); - Utilities.setReduceWork(cloned, (ReduceWork) work, scratchDir, false); - Utilities.createTmpDirs(cloned, (ReduceWork) work); - cloned.set(Utilities.MAPRED_REDUCER_CLASS, ExecReducer.class.getName()); - } - return cloned; - } - - private void initStatsPublisher(BaseWork work) throws HiveException { - // initialize stats publisher if necessary - if (work.isGatheringStats()) { - StatsPublisher statsPublisher; - StatsFactory factory = StatsFactory.newFactory(jobConf); - if (factory != null) { - statsPublisher = factory.getStatsPublisher(); - StatsCollectionContext sc = new StatsCollectionContext(jobConf); - sc.setStatsTmpDirs(Utilities.getStatsTmpDirs(work, jobConf)); - if (!statsPublisher.init(sc)) { // creating stats table if not exists - if (HiveConf.getBoolVar(jobConf, HiveConf.ConfVars.HIVE_STATS_RELIABLE)) { - throw new HiveException( - ErrorMsg.STATSPUBLISHER_INITIALIZATION_ERROR.getErrorCodedMsg()); - } - } - } - } - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java deleted file mode 100644 index cb5bd7ada2d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkRecordHandler.java +++ /dev/null @@ -1,128 +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.hadoop.hive.ql.exec.spark; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.MapredContext; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; - -import java.io.IOException; -import java.lang.management.ManagementFactory; -import java.lang.management.MemoryMXBean; -import java.net.URLClassLoader; -import java.util.Arrays; -import java.util.Iterator; - -public abstract class SparkRecordHandler { - protected static final String CLASS_NAME = SparkRecordHandler.class.getName(); - protected final PerfLogger perfLogger = SessionState.getPerfLogger(); - private static final Logger LOG = LoggerFactory.getLogger(SparkRecordHandler.class); - - // used to log memory usage periodically - protected final MemoryMXBean memoryMXBean = ManagementFactory.getMemoryMXBean(); - - protected JobConf jc; - protected OutputCollector oc; - protected Reporter rp; - protected boolean abort = false; - private long rowNumber = 0; - private long nextLogThreshold = 1; - - protected boolean anyRow = false; - - public void init(JobConf job, OutputCollector output, Reporter reporter) throws Exception { - jc = job; - MapredContext.init(false, new JobConf(jc)); - MapredContext.get().setReporter(reporter); - - oc = output; - rp = reporter; - - LOG.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax()); - - try { - LOG.info("conf classpath = " - + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs())); - LOG.info("thread classpath = " - + Arrays.asList(((URLClassLoader) Thread.currentThread() - .getContextClassLoader()).getURLs())); - } catch (Exception e) { - LOG.info("cannot get classpath: " + e.getMessage()); - } - } - - /** - * Process row with key and single value. - */ - public abstract void processRow(Object key, Object value) throws IOException; - - /** - * Process row with key and value collection. - */ - public abstract void processRow(Object key, Iterator values) throws IOException; - - /** - * Logger processed row number and used memory info. - */ - protected void logMemoryInfo() { - rowNumber++; - if (rowNumber == nextLogThreshold) { - long usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); - LOG.info("processing " + rowNumber - + " rows: used memory = " + usedMemory); - nextLogThreshold = getNextLogThreshold(rowNumber); - } - } - - public abstract void close(); - public abstract boolean getDone(); - - /** - * Logger information to be logged at the end. - */ - protected void logCloseInfo() { - long usedMemory = memoryMXBean.getHeapMemoryUsage().getUsed(); - LOG.info("processed " + rowNumber + " rows: used memory = " - + usedMemory); - } - - private long getNextLogThreshold(long currentThreshold) { - // A very simple counter to keep track of number of rows processed by the - // reducer. It dumps - // every 1 million times, and quickly before that - if (currentThreshold >= 1000000) { - return currentThreshold + 1000000; - } - - return 10 * currentThreshold; - } - - public boolean isAbort() { - return abort; - } - - public void setAbort(boolean abort) { - this.abort = abort; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java deleted file mode 100644 index 20e7ea0f4e8..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java +++ /dev/null @@ -1,660 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.MapredContext; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.ReportStats; -import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorDeserializeRow; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe; -import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableDeserializeRead; -import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinaryDeserializeRead; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.StringUtils; - -import com.google.common.base.Preconditions; - -/** - * Clone from ExecReducer, it is the bridge between the spark framework and - * the Hive operator pipeline at execution time. It's main responsibilities are: - * - * - Load and setup the operator pipeline from XML - * - Run the pipeline by transforming key, value pairs to records and forwarding them to the operators - * - Sending start and end group messages to separate records with same key from one another - * - Catch and handle errors during execution of the operators. - * - */ -public class SparkReduceRecordHandler extends SparkRecordHandler { - - private static final Logger LOG = LoggerFactory.getLogger(SparkReduceRecordHandler.class); - - // Input value serde needs to be an array to support different SerDe - // for different tags - private final Deserializer[] inputValueDeserializer = new Deserializer[Byte.MAX_VALUE]; - private final Object[] valueObject = new Object[Byte.MAX_VALUE]; - private final List row = new ArrayList(Utilities.reduceFieldNameList.size()); - - // TODO: move to DynamicSerDe when it's ready - private Deserializer inputKeyDeserializer; - private Operator reducer; - private boolean isTagged = false; - private TableDesc keyTableDesc; - private TableDesc[] valueTableDesc; - private ObjectInspector[] rowObjectInspector; - private boolean vectorized = false; - - private VectorDeserializeRow keyBinarySortableDeserializeToRow; - - private VectorDeserializeRow valueLazyBinaryDeserializeToRow; - - private VectorizedRowBatch batch; - private long batchBytes = 0; - private boolean handleGroupKey = true; // For now. - - private DataOutputBuffer buffer; - - // number of columns pertaining to keys in a vectorized row batch - private int firstValueColumnOffset; - - private static final int BATCH_SIZE = VectorizedRowBatch.DEFAULT_SIZE; - private static final int BATCH_BYTES = VectorizedRowBatch.DEFAULT_BYTES; - - // runtime objects - private transient Object keyObject; - private transient BytesWritable groupKey; - - private StructObjectInspector keyStructInspector; - private StructObjectInspector valueStructInspector; - /* this is only used in the error code path */ - private MapredLocalWork localWork = null; - - @Override - @SuppressWarnings("unchecked") - public void init(JobConf job, OutputCollector output, Reporter reporter) throws Exception { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); - super.init(job, output, reporter); - - rowObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; - ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; - ObjectInspector keyObjectInspector; - - ReduceWork gWork = Utilities.getReduceWork(job); - - reducer = gWork.getReducer(); - vectorized = gWork.getVectorMode(); - reducer.setParentOperators(null); // clear out any parents as reducer is the - // root - isTagged = gWork.getNeedsTagging(); - try { - keyTableDesc = gWork.getKeyDesc(); - inputKeyDeserializer = ReflectionUtils.newInstance(keyTableDesc - .getDeserializerClass(), null); - SerDeUtils.initializeSerDe(inputKeyDeserializer, null, keyTableDesc.getProperties(), null); - keyObjectInspector = inputKeyDeserializer.getObjectInspector(); - valueTableDesc = new TableDesc[gWork.getTagToValueDesc().size()]; - - if (vectorized) { - final int maxTags = gWork.getTagToValueDesc().size(); - - // CONSIDER: Cleaning up this code and eliminating the arrays. Vectorization only handles - // one operator tree. - Preconditions.checkState(maxTags == 1); - - keyStructInspector = (StructObjectInspector) keyObjectInspector; - firstValueColumnOffset = keyStructInspector.getAllStructFieldRefs().size(); - - buffer = new DataOutputBuffer(); - } - - for (int tag = 0; tag < gWork.getTagToValueDesc().size(); tag++) { - // We should initialize the SerDe with the TypeInfo when available. - valueTableDesc[tag] = gWork.getTagToValueDesc().get(tag); - inputValueDeserializer[tag] = ReflectionUtils.newInstance( - valueTableDesc[tag].getDeserializerClass(), null); - SerDeUtils.initializeSerDe(inputValueDeserializer[tag], null, - valueTableDesc[tag].getProperties(), null); - valueObjectInspector[tag] = inputValueDeserializer[tag].getObjectInspector(); - - ArrayList ois = new ArrayList(); - - if (vectorized) { - /* vectorization only works with struct object inspectors */ - valueStructInspector = (StructObjectInspector) valueObjectInspector[tag]; - - final int totalColumns = firstValueColumnOffset - + valueStructInspector.getAllStructFieldRefs().size(); - - rowObjectInspector[tag] = Utilities.constructVectorizedReduceRowOI(keyStructInspector, - valueStructInspector); - batch = gWork.getVectorizedRowBatchCtx().createVectorizedRowBatch(); - - // Setup vectorized deserialization for the key and value. - BinarySortableSerDe binarySortableSerDe = (BinarySortableSerDe) inputKeyDeserializer; - - keyBinarySortableDeserializeToRow = - new VectorDeserializeRow( - new BinarySortableDeserializeRead( - VectorizedBatchUtil.typeInfosFromStructObjectInspector( - keyStructInspector), - /* useExternalBuffer */ true, - binarySortableSerDe.getSortOrders(), - binarySortableSerDe.getNullMarkers(), - binarySortableSerDe.getNotNullMarkers())); - keyBinarySortableDeserializeToRow.init(0); - - final int valuesSize = valueStructInspector.getAllStructFieldRefs().size(); - if (valuesSize > 0) { - valueLazyBinaryDeserializeToRow = - new VectorDeserializeRow( - new LazyBinaryDeserializeRead( - VectorizedBatchUtil.typeInfosFromStructObjectInspector( - valueStructInspector), - /* useExternalBuffer */ true)); - valueLazyBinaryDeserializeToRow.init(firstValueColumnOffset); - - // Create data buffers for value bytes column vectors. - for (int i = firstValueColumnOffset; i < batch.numCols; i++) { - ColumnVector colVector = batch.cols[i]; - if (colVector instanceof BytesColumnVector) { - BytesColumnVector bytesColumnVector = (BytesColumnVector) colVector; - bytesColumnVector.initBuffer(); - } - } - } - - } else { - ois.add(keyObjectInspector); - ois.add(valueObjectInspector[tag]); - //reducer.setGroupKeyObjectInspector(keyObjectInspector); - rowObjectInspector[tag] = ObjectInspectorFactory.getStandardStructObjectInspector( - Utilities.reduceFieldNameList, ois); - } - } - } catch (Exception e) { - throw new RuntimeException(e); - } - - ExecMapperContext execContext = new ExecMapperContext(job); - localWork = gWork.getMapRedLocalWork(); - execContext.setJc(jc); - execContext.setLocalWork(localWork); - reducer.passExecContext(execContext); - - reducer.setReporter(rp); - OperatorUtils.setChildrenCollector( - Arrays.>asList(reducer), output); - - // initialize reduce operator tree - try { - LOG.info(reducer.dump(0)); - reducer.initialize(jc, rowObjectInspector); - - if (localWork != null) { - for (Operator dummyOp : localWork.getDummyParentOp()) { - dummyOp.setExecContext(execContext); - dummyOp.initialize(jc, null); - } - } - - } catch (Throwable e) { - abort = true; - if (e instanceof OutOfMemoryError) { - // Don't create a new object if we are already out of memory - throw (OutOfMemoryError) e; - } else { - throw new RuntimeException("Reduce operator initialization failed", e); - } - } - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS); - } - - /** - * A reusable dummy iterator that has only one value. - * - */ - private static class DummyIterator implements Iterator { - private boolean done = false; - private Object value = null; - - public void setValue(Object v) { - this.value = v; - done = false; - } - - @Override - public boolean hasNext() { - return !done; - } - - @Override - public Object next() { - done = true; - return value; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("Iterator.remove() is not implemented/supported"); - } - } - - private DummyIterator dummyIterator = new DummyIterator(); - - /** - * Process one row using a dummy iterator. Or, add row to vector batch. - */ - @Override - public void processRow(Object key, final Object value) throws IOException { - if (!anyRow) { - anyRow = true; - } - if (vectorized) { - processVectorRow(key, value); - } else { - dummyIterator.setValue(value); - processRow(key, dummyIterator); - } - } - - - - @Override - public void processRow(Object key, Iterator values) throws IOException { - if (!anyRow) { - anyRow = true; - } - if (vectorized) { - processVectorRows(key, values); - return; - } - - if (reducer.getDone()) { - return; - } - - try { - BytesWritable keyWritable = (BytesWritable) key; - byte tag = 0; - if (isTagged) { - // remove the tag from key coming out of reducer - // and store it in separate variable. - // make a copy for multi-insert with join case as Spark re-uses input key from same parent - int size = keyWritable.getSize() - 1; - tag = keyWritable.get()[size]; - keyWritable = new BytesWritable(keyWritable.getBytes(), size); - keyWritable.setSize(size); - } - - if (!keyWritable.equals(groupKey)) { - // If a operator wants to do some work at the beginning of a group - if (groupKey == null) { // the first group - groupKey = new BytesWritable(); - } else { - // If a operator wants to do some work at the end of a group - LOG.trace("End Group"); - reducer.endGroup(); - } - - try { - keyObject = inputKeyDeserializer.deserialize(keyWritable); - } catch (Exception e) { - // Log the input key which caused exception so that it's available for debugging. But when - // exposed through an error message it can leak sensitive information, even to the client - // application. - LOG.trace("Hive Runtime Error: Unable to deserialize reduce input key from " - + Utilities.formatBinaryString(keyWritable.get(), 0, - keyWritable.getSize()) + " with properties " - + keyTableDesc.getProperties()); - throw new HiveException("Hive Runtime Error: Unable to deserialize reduce input key ", e); - } - - groupKey.set(keyWritable.get(), 0, keyWritable.getSize()); - LOG.trace("Start Group"); - reducer.setGroupKeyObject(keyObject); - reducer.startGroup(); - } - processKeyValues(values, tag); - - } catch (Throwable e) { - abort = true; - Utilities.setReduceWork(jc, null); - if (e instanceof OutOfMemoryError) { - // Don't create a new object if we are already out of memory - throw (OutOfMemoryError) e; - } else { - String msg = "Fatal error: " + e; - LOG.error(msg, e); - throw new RuntimeException(e); - } - } - } - - /** - * @param values - * @return true if it is not done and can take more inputs - */ - private boolean processKeyValues(Iterator values, byte tag) throws HiveException { - while (values.hasNext()) { - BytesWritable valueWritable = (BytesWritable) values.next(); - try { - valueObject[tag] = inputValueDeserializer[tag].deserialize(valueWritable); - } catch (SerDeException e) { - // Log the input value which caused exception so that it's available for debugging. But when - // exposed through an error message it can leak sensitive information, even to the client - // application. - LOG.trace("Hive Runtime Error: Unable to deserialize reduce input value (tag=" - + tag - + ") from " - + Utilities.formatBinaryString(valueWritable.get(), 0, - valueWritable.getSize()) + " with properties " - + valueTableDesc[tag].getProperties()); - throw new HiveException("Hive Runtime Error: Unable to deserialize reduce input value ", e); - } - row.clear(); - row.add(keyObject); - row.add(valueObject[tag]); - if (LOG.isInfoEnabled()) { - logMemoryInfo(); - } - try { - reducer.process(row, tag); - } catch (Exception e) { - String rowString = null; - try { - rowString = SerDeUtils.getJSONString(row, rowObjectInspector[tag]); - } catch (Exception e2) { - rowString = "[Error getting row data with exception " - + StringUtils.stringifyException(e2) + " ]"; - } - - // Log contents of the row which caused exception so that it's available for debugging. But - // when exposed through an error message it can leak sensitive information, even to the - // client application. - LOG.trace("Hive exception while processing row (tag=" + tag + ") " + rowString); - throw new HiveException("Error while processing row ", e); - } - } - return true; // give me more - } - - private void processVectorRows(Object key, Iterator values) throws IOException { - if (reducer.getDone()) { - return; - } - while (values.hasNext()) { - processVectorRow(key, values.next()); - } - } - - private void processVectorRow(Object key, final Object value) throws IOException { - BytesWritable keyWritable = (BytesWritable) key; - BytesWritable valueWritable = (BytesWritable) value; - - try { - - if (handleGroupKey) { - final boolean isKeyChange; - if (groupKey == null) { - - // The first group. - isKeyChange = true; - groupKey = new BytesWritable(); - } else { - isKeyChange = !keyWritable.equals(groupKey); - } - - if (isKeyChange) { - - // Flush current group batch as last batch of group. - if (batch.size > 0) { - - // Indicate last batch of current group. - reducer.setNextVectorBatchGroupStatus(/* isLastGroupBatch */ true); - - // Forward; reset key and value columns. - forwardBatch(/* resetValueColumnsOnly */ false); - } - - // Deserialize group key into vector row columns. - byte[] keyBytes = keyWritable.getBytes(); - int keyLength = keyWritable.getLength(); - - groupKey.set(keyBytes, 0, keyLength); - - keyBinarySortableDeserializeToRow.setBytes(keyBytes, 0, keyLength); - try { - keyBinarySortableDeserializeToRow.deserialize(batch, 0); - } catch (Exception e) { - throw new HiveException( - "\nDeserializeRead details: " + - keyBinarySortableDeserializeToRow.getDetailedReadPositionString(), - e); - } - - // And, mark group keys as repeating. - for(int i = 0; i < firstValueColumnOffset; i++) { - VectorizedBatchUtil.setRepeatingColumn(batch, i); - } - } - - // Can we add to current batch? - if (batch.size >= batch.getMaxSize() || - batch.size > 0 && batchBytes >= BATCH_BYTES) { - - // We have a row for current group, so we indicate not the last batch. - reducer.setNextVectorBatchGroupStatus(/* isLastGroupBatch */ false); - - // Batch is full or using too much space. - forwardBatch(/* resetValueColumnsOnly */ true); - } - - if (valueLazyBinaryDeserializeToRow != null) { - // Deserialize value into vector row columns. - byte[] valueBytes = valueWritable.getBytes(); - int valueLength = valueWritable.getLength(); - batchBytes += valueLength; - - valueLazyBinaryDeserializeToRow.setBytes(valueBytes, 0, valueLength); - valueLazyBinaryDeserializeToRow.deserialize(batch, batch.size); - } - batch.size++; - } else { - - // No group key. - - // Can we add to current batch? - if (batch.size >= batch.getMaxSize() || - batch.size > 0 && batchBytes >= BATCH_BYTES) { - - // Batch is full or using too much space. - forwardBatch(/* resetValueColumnsOnly */ false); - } - - // Deserialize key into vector row columns. - byte[] keyBytes = keyWritable.getBytes(); - int keyLength = keyWritable.getLength(); - - keyBinarySortableDeserializeToRow.setBytes(keyBytes, 0, keyLength); - try { - keyBinarySortableDeserializeToRow.deserialize(batch, 0); - } catch (Exception e) { - throw new HiveException( - "\nDeserializeRead details: " + - keyBinarySortableDeserializeToRow.getDetailedReadPositionString(), - e); - } - - if (valueLazyBinaryDeserializeToRow != null) { - // Deserialize value into vector row columns. - byte[] valueBytes = valueWritable.getBytes(); - int valueLength = valueWritable.getLength(); - - batchBytes += valueLength; - - valueLazyBinaryDeserializeToRow.setBytes(valueBytes, 0, valueLength); - valueLazyBinaryDeserializeToRow.deserialize(batch, batch.size); - } - batch.size++; - } - } catch (Throwable e) { - abort = true; - if (e instanceof OutOfMemoryError) { - // Don't create a new object if we are already out of memory - throw (OutOfMemoryError) e; - } else { - throw new RuntimeException(e); - } - } - } - - private void forwardBatch(boolean resetValueColumnsOnly) throws HiveException { - reducer.process(batch, 0); - - if (resetValueColumnsOnly) { - // Reset just the value columns and value buffer. - for (int i = firstValueColumnOffset; i < batch.numCols; i++) { - // Note that reset also resets the data buffer for bytes column vectors. - batch.cols[i].reset(); - } - batch.size = 0; - } else { - // Reset key and value columns; and batch.size - batch.reset(); - } - - batchBytes = 0; - if (LOG.isInfoEnabled()) { - logMemoryInfo(); - } - } - - private Object deserializeValue(BytesWritable valueWritable, byte tag) throws HiveException { - try { - return inputValueDeserializer[tag].deserialize(valueWritable); - } catch (SerDeException e) { - // Log the input value which caused exception so that it's available for debugging. But when - // exposed through an error message it can leak sensitive information, even to the client - // application. - LOG.trace("Error: Unable to deserialize reduce input value (tag=" + tag + ") from " + - Utilities.formatBinaryString(valueWritable.getBytes(), 0, - valueWritable.getLength()) + - " with properties " + valueTableDesc[tag].getProperties()); - throw new HiveException("Error: Unable to deserialize reduce input value ", e); - } - } - - @Override - public void close() { - - // No row was processed - if (!anyRow) { - LOG.trace("Close called without any rows processed"); - } - - try { - if (vectorized) { - if (batch.size > 0) { - - if (handleGroupKey) { - // Indicate last batch of current group. - reducer.setNextVectorBatchGroupStatus(/* isLastGroupBatch */ true); - } - - forwardBatch(/* resetValueColumnsOnly */ false); - } - } else { - if (groupKey != null) { - // If a operator wants to do some work at the end of a group - LOG.trace("End Group"); - reducer.endGroup(); - } - } - if (LOG.isInfoEnabled()) { - logCloseInfo(); - } - - reducer.close(abort); - - if (localWork != null) { - for (Operator dummyOp : localWork.getDummyParentOp()) { - dummyOp.close(abort); - } - } - - ReportStats rps = new ReportStats(rp, jc); - reducer.preorderMap(rps); - - } catch (Exception e) { - if (!abort) { - // signal new failure to map-reduce - LOG.error("Hit error while closing operators - failing tree"); - throw new RuntimeException("Hive Runtime Error while closing operators: " - + e.getMessage(), e); - } - } finally { - MapredContext.close(); - Utilities.clearWorkMap(jc); - } - } - - @Override - public boolean getDone() { - return reducer.getDone(); - } - - public static String displayBytes(byte[] bytes, int start, int length) { - StringBuilder sb = new StringBuilder(); - for (int i = start; i < start + length; i++) { - sb.append(String.format("\\%03d", (int) (bytes[i] & 0xff))); - } - return sb.toString(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java deleted file mode 100644 index eca6b3396e7..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReporter.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.exec.spark; - -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.hadoop.mapred.Counters.Counter; -import org.apache.hadoop.mapred.InputSplit; -import org.apache.hadoop.mapred.Reporter; - -import java.io.Serializable; - -/** - * Implement SparkReporter for Hive operator level statistics collection, and throw - * UnsupportedOperationException for other unrelated methods, so if any Hive feature - * depends on these unimplemented methods, we could go back here quickly and enable it. - */ -public class SparkReporter implements Reporter, Serializable { - - private SparkCounters sparkCounters; - private String status; - public SparkReporter(SparkCounters sparkCounters) { - this.sparkCounters = sparkCounters; - } - - @Override - public void setStatus(String status) { - this.status = status; - } - - public String getStatus() { - return this.status; - } - - @Override - public Counter getCounter(Enum name) { - throw new UnsupportedOperationException("do not support this method now."); - } - - @Override - public Counter getCounter(String group, String name) { - throw new UnsupportedOperationException("do not support this method now."); - } - - @Override - public void incrCounter(Enum key, long amount) { - sparkCounters.increment(key.getDeclaringClass().getName(), key.name(), amount); - } - - @Override - public void incrCounter(String group, String counter, long amount) { - sparkCounters.increment(group, counter, amount); - } - - @Override - public InputSplit getInputSplit() { - throw new UnsupportedOperationException("do not support this method now."); - } - - @Override - public float getProgress() { - throw new UnsupportedOperationException("do not support this method now."); - } - - @Override - public void progress() { - //do not support task level progress, do nothing here. - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkShuffler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkShuffler.java deleted file mode 100644 index c29a93d06d3..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkShuffler.java +++ /dev/null @@ -1,32 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.apache.spark.api.java.JavaPairRDD; - -public interface SparkShuffler { - - JavaPairRDD shuffle( - JavaPairRDD input, int numPartitions); - - public String getName(); - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java deleted file mode 100644 index 3083e30ee3d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java +++ /dev/null @@ -1,520 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Strings; -import com.google.common.base.Throwables; -import org.apache.hadoop.hive.common.metrics.common.Metrics; -import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.QueryState; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.ScriptOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistic; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticGroup; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkStageProgress; -import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.api.StageType; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; -import org.apache.hadoop.util.StringUtils; - -import com.google.common.collect.Lists; -import org.apache.spark.SparkException; - -public class SparkTask extends Task { - private static final String CLASS_NAME = SparkTask.class.getName(); - private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); - private static final LogHelper console = new LogHelper(LOG); - private PerfLogger perfLogger; - private static final long serialVersionUID = 1L; - // The id of the actual Spark job - private transient int sparkJobID; - // The id of the JobHandle used to track the actual Spark job - private transient String sparkJobHandleId; - private transient SparkStatistics sparkStatistics; - private transient long submitTime; - private transient long startTime; - private transient long finishTime; - private transient int succeededTaskCount; - private transient int totalTaskCount; - private transient int failedTaskCount; - private transient List stageIds; - private transient SparkJobRef jobRef = null; - private transient boolean isShutdown = false; - private transient boolean jobKilled = false; - - @Override - public void initialize(QueryState queryState, QueryPlan queryPlan, DriverContext driverContext, - CompilationOpContext opContext) { - super.initialize(queryState, queryPlan, driverContext, opContext); - } - - @Override - public int execute(DriverContext driverContext) { - - int rc = 0; - perfLogger = SessionState.getPerfLogger(); - SparkSession sparkSession = null; - SparkSessionManager sparkSessionManager = null; - try { - printConfigInfo(); - sparkSessionManager = SparkSessionManagerImpl.getInstance(); - sparkSession = SparkUtilities.getSparkSession(conf, sparkSessionManager); - - SparkWork sparkWork = getWork(); - sparkWork.setRequiredCounterPrefix(getOperatorCounters()); - - // Submit the Spark job - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); - submitTime = perfLogger.getStartTime(PerfLogger.SPARK_SUBMIT_JOB); - jobRef = sparkSession.submit(driverContext, sparkWork); - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_JOB); - - // If the driver context has been shutdown (due to query cancellation) kill the Spark job - if (driverContext.isShutdown()) { - LOG.warn("Killing Spark job"); - killJob(); - throw new HiveException("Operation is cancelled."); - } - - // Get the Job Handle id associated with the Spark job - sparkJobHandleId = jobRef.getJobId(); - - // Add Spark job handle id to the Hive History - addToHistory(Keys.SPARK_JOB_HANDLE_ID, jobRef.getJobId()); - - LOG.debug("Starting Spark job with job handle id " + sparkJobHandleId); - - // Get the application id of the Spark app - jobID = jobRef.getSparkJobStatus().getAppID(); - - // Start monitoring the Spark job, returns when the Spark job has completed / failed, or if - // a timeout occurs - rc = jobRef.monitorJob(); - - // Get the id the Spark job that was launched, returns -1 if no Spark job was launched - sparkJobID = jobRef.getSparkJobStatus().getJobId(); - - // Add Spark job id to the Hive History - addToHistory(Keys.SPARK_JOB_ID, Integer.toString(sparkJobID)); - - // Get the final state of the Spark job and parses its job info - SparkJobStatus sparkJobStatus = jobRef.getSparkJobStatus(); - getSparkJobInfo(sparkJobStatus, rc); - - if (rc == 0) { - sparkStatistics = sparkJobStatus.getSparkStatistics(); - printExcessiveGCWarning(); - if (LOG.isInfoEnabled() && sparkStatistics != null) { - LOG.info(sparkStatisticsToString(sparkStatistics, sparkJobID)); - } - LOG.info("Successfully completed Spark job[" + sparkJobID + "] with application ID " + - jobID + " and task ID " + getId()); - } else if (rc == 2) { // Cancel job if the monitor found job submission timeout. - // TODO: If the timeout is because of lack of resources in the cluster, we should - // ideally also cancel the app request here. But w/o facilities from Spark or YARN, - // it's difficult to do it on hive side alone. See HIVE-12650. - LOG.debug("Failed to submit Spark job with job handle id " + sparkJobHandleId); - LOG.info("Failed to submit Spark job for application id " + (Strings.isNullOrEmpty(jobID) - ? "UNKNOWN" : jobID)); - killJob(); - } else if (rc == 4) { - LOG.info("The spark job or one stage of it has too many tasks" + - ". Cancelling Spark job " + sparkJobID + " with application ID " + jobID ); - killJob(); - } - - if (this.jobID == null) { - this.jobID = sparkJobStatus.getAppID(); - } - sparkJobStatus.cleanup(); - } catch (Exception e) { - String msg = "Failed to execute spark task, with exception '" + Utilities.getNameMessage(e) + "'"; - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - LOG.error(msg, e); - setException(e); - if (e instanceof HiveException) { - HiveException he = (HiveException) e; - rc = he.getCanonicalErrorMsg().getErrorCode(); - } else { - rc = 1; - } - } finally { - startTime = perfLogger.getEndTime(PerfLogger.SPARK_SUBMIT_TO_RUNNING); - // The startTime may not be set if the sparkTask finished too fast, - // because SparkJobMonitor will sleep for 1 second then check the state, - // right after sleep, the spark job may be already completed. - // In this case, set startTime the same as submitTime. - if (startTime < submitTime) { - startTime = submitTime; - } - finishTime = perfLogger.getEndTime(PerfLogger.SPARK_RUN_JOB); - Utilities.clearWork(conf); - if (sparkSession != null && sparkSessionManager != null) { - rc = close(rc); - try { - sparkSessionManager.returnSession(sparkSession); - } catch (HiveException ex) { - LOG.error("Failed to return the session to SessionManager", ex); - } - } - } - return rc; - } - - /** - * Use the Spark metrics and calculate how much task executione time was spent performing GC - * operations. If more than a defined threshold of time is spent, print out a warning on the - * console. - */ - private void printExcessiveGCWarning() { - SparkStatisticGroup sparkStatisticGroup = sparkStatistics.getStatisticGroup( - SparkStatisticsNames.SPARK_GROUP_NAME); - if (sparkStatisticGroup != null) { - long taskDurationTime = Long.parseLong(sparkStatisticGroup.getSparkStatistic( - SparkStatisticsNames.TASK_DURATION_TIME).getValue()); - long jvmGCTime = Long.parseLong(sparkStatisticGroup.getSparkStatistic( - SparkStatisticsNames.JVM_GC_TIME).getValue()); - - // Threshold percentage to trigger the GC warning - double threshold = 0.1; - - if (jvmGCTime > taskDurationTime * threshold) { - long percentGcTime = Math.round((double) jvmGCTime / taskDurationTime * 100); - String gcWarning = String.format("WARNING: Spark Job[%s] Spent %s%% (%s ms / %s ms) of " + - "task time in GC", sparkJobID, percentGcTime, jvmGCTime, taskDurationTime); - console.printInfo(gcWarning); - } - } - } - - private void addToHistory(Keys key, String value) { - if (SessionState.get() != null) { - SessionState.get().getHiveHistory().setQueryProperty(queryState.getQueryId(), key, value); - } - } - - @VisibleForTesting - static String sparkStatisticsToString(SparkStatistics sparkStatistic, int sparkJobID) { - StringBuilder sparkStatsString = new StringBuilder(); - sparkStatsString.append("\n\n"); - sparkStatsString.append(String.format("=====Spark Job[%d] Statistics=====", sparkJobID)); - sparkStatsString.append("\n\n"); - - Iterator groupIterator = sparkStatistic.getStatisticGroups(); - while (groupIterator.hasNext()) { - SparkStatisticGroup group = groupIterator.next(); - sparkStatsString.append(group.getGroupName()).append("\n"); - Iterator statisticIterator = group.getStatistics(); - while (statisticIterator.hasNext()) { - SparkStatistic statistic = statisticIterator.next(); - sparkStatsString.append("\t").append(statistic.getName()).append(": ").append( - statistic.getValue()).append("\n"); - } - } - return sparkStatsString.toString(); - } - - /** - * Close will move the temp files into the right place for the fetch - * task. If the job has failed it will clean up the files. - */ - private int close(int rc) { - try { - List ws = work.getAllWork(); - for (BaseWork w: ws) { - for (Operator op: w.getAllOperators()) { - op.jobClose(conf, rc == 0); - } - } - } catch (Exception e) { - // jobClose needs to execute successfully otherwise fail task - if (rc == 0) { - rc = 3; - String mesg = "Job Commit failed with exception '" - + Utilities.getNameMessage(e) + "'"; - console.printError(mesg, "\n" + StringUtils.stringifyException(e)); - setException(e); - } - } - return rc; - } - - @Override - public void updateTaskMetrics(Metrics metrics) { - metrics.incrementCounter(MetricsConstant.HIVE_SPARK_TASKS); - } - - @Override - public boolean isMapRedTask() { - return true; - } - - @Override - public StageType getType() { - return StageType.MAPRED; - } - - @Override - public String getName() { - return "SPARK"; - } - - @Override - public Collection getMapWork() { - List result = Lists.newArrayList(); - for (BaseWork w : getWork().getRoots()) { - result.add((MapWork) w); - } - return result; - } - - @Override - public Operator getReducer(MapWork mapWork) { - List children = getWork().getChildren(mapWork); - if (children.size() != 1) { - return null; - } - - if (!(children.get(0) instanceof ReduceWork)) { - return null; - } - - return ((ReduceWork) children.get(0)).getReducer(); - } - - public int getSparkJobID() { - return sparkJobID; - } - - public SparkStatistics getSparkStatistics() { - return sparkStatistics; - } - - public int getSucceededTaskCount() { - return succeededTaskCount; - } - - public int getTotalTaskCount() { - return totalTaskCount; - } - - public int getFailedTaskCount() { - return failedTaskCount; - } - - public List getStageIds() { - return stageIds; - } - - public long getStartTime() { - return startTime; - } - - public long getSubmitTime() { - return submitTime; - } - - public long getFinishTime() { - return finishTime; - } - - public boolean isTaskShutdown() { - return isShutdown; - } - - @Override - public void shutdown() { - super.shutdown(); - killJob(); - isShutdown = true; - } - - private void killJob() { - LOG.debug("Killing Spark job with job handle id " + sparkJobHandleId); - boolean needToKillJob = false; - if (jobRef != null && !jobKilled) { - synchronized (this) { - if (!jobKilled) { - jobKilled = true; - needToKillJob = true; - } - } - } - if (needToKillJob) { - try { - jobRef.cancelJob(); - } catch (Exception e) { - LOG.warn("Failed to kill Spark job", e); - } - } - } - - /** - * Set the number of reducers for the spark work. - */ - private void printConfigInfo() throws IOException { - - console.printInfo("In order to change the average load for a reducer (in bytes):"); - console.printInfo(" set " + HiveConf.ConfVars.BYTESPERREDUCER.varname + "="); - console.printInfo("In order to limit the maximum number of reducers:"); - console.printInfo(" set " + HiveConf.ConfVars.MAXREDUCERS.varname + "="); - console.printInfo("In order to set a constant number of reducers:"); - console.printInfo(" set " + HiveConf.ConfVars.HADOOPNUMREDUCERS + "="); - } - - private Map> getOperatorCounters() { - String groupName = HiveConf.getVar(conf, HiveConf.ConfVars.HIVECOUNTERGROUP); - Map> counters = new HashMap>(); - List hiveCounters = new LinkedList(); - counters.put(groupName, hiveCounters); - hiveCounters.add(Operator.HIVE_COUNTER_CREATED_FILES); - // MapOperator is out of SparkWork, SparkMapRecordHandler use it to bridge - // Spark transformation and Hive operators in SparkWork. - for (MapOperator.Counter counter : MapOperator.Counter.values()) { - hiveCounters.add(counter.toString()); - } - SparkWork sparkWork = this.getWork(); - for (BaseWork work : sparkWork.getAllWork()) { - for (Operator operator : work.getAllOperators()) { - if (operator instanceof FileSinkOperator) { - for (FileSinkOperator.Counter counter : FileSinkOperator.Counter.values()) { - hiveCounters.add(((FileSinkOperator) operator).getCounterName(counter)); - } - } else if (operator instanceof ReduceSinkOperator) { - final String contextName = conf.get(Operator.CONTEXT_NAME_KEY, ""); - for (ReduceSinkOperator.Counter counter : ReduceSinkOperator.Counter.values()) { - hiveCounters.add(Utilities.getVertexCounterName(counter.name(), contextName)); - } - } else if (operator instanceof ScriptOperator) { - for (ScriptOperator.Counter counter : ScriptOperator.Counter.values()) { - hiveCounters.add(counter.toString()); - } - } else if (operator instanceof JoinOperator) { - for (JoinOperator.SkewkeyTableCounter counter : JoinOperator.SkewkeyTableCounter.values()) { - hiveCounters.add(counter.toString()); - } - } - } - } - - return counters; - } - - private void getSparkJobInfo(SparkJobStatus sparkJobStatus, int rc) { - try { - stageIds = new ArrayList(); - int[] ids = sparkJobStatus.getStageIds(); - if (ids != null) { - for (int stageId : ids) { - stageIds.add(stageId); - } - } - Map progressMap = sparkJobStatus.getSparkStageProgress(); - int sumTotal = 0; - int sumComplete = 0; - int sumFailed = 0; - for (String s : progressMap.keySet()) { - SparkStageProgress progress = progressMap.get(s); - final int complete = progress.getSucceededTaskCount(); - final int total = progress.getTotalTaskCount(); - final int failed = progress.getFailedTaskCount(); - sumTotal += total; - sumComplete += complete; - sumFailed += failed; - } - succeededTaskCount = sumComplete; - totalTaskCount = sumTotal; - failedTaskCount = sumFailed; - if (rc != 0) { - Throwable error = sparkJobStatus.getError(); - if (error != null) { - if ((error instanceof InterruptedException) || - (error instanceof HiveException && - error.getCause() instanceof InterruptedException)) { - LOG.info("Killing Spark job since query was interrupted"); - killJob(); - } - HiveException he; - if (isOOMError(error)) { - he = new HiveException(error, ErrorMsg.SPARK_RUNTIME_OOM); - } else { - he = new HiveException(error, ErrorMsg.SPARK_JOB_RUNTIME_ERROR); - } - setException(he); - } - } - } catch (Exception e) { - LOG.error("Failed to get Spark job information", e); - } - } - - private boolean isOOMError(Throwable error) { - while (error != null) { - if (error instanceof OutOfMemoryError) { - return true; - } else if (error instanceof SparkException) { - String sts = Throwables.getStackTraceAsString(error); - return sts.contains("Container killed by YARN for exceeding memory limits"); - } - error = error.getCause(); - } - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java deleted file mode 100644 index f9057b9254f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTran.java +++ /dev/null @@ -1,32 +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.hadoop.hive.ql.exec.spark; - -import org.apache.hadoop.io.WritableComparable; -import org.apache.spark.api.java.JavaPairRDD; - -@SuppressWarnings("rawtypes") -public interface SparkTran { - JavaPairRDD transform( - JavaPairRDD input); - - public String getName(); - - public Boolean isCacheEnable(); -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java deleted file mode 100644 index fdc53619893..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ /dev/null @@ -1,342 +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.hadoop.hive.ql.exec.spark; - -import java.io.IOException; -import java.net.InetAddress; -import java.net.URI; -import java.net.UnknownHostException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Deque; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import com.google.common.base.Preconditions; -import org.apache.commons.io.FilenameUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.GroupByOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.SelectOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hive.spark.client.SparkClientUtilities; -import org.apache.spark.SparkConf; - - -/** - * Contains utilities methods used as part of Spark tasks. - */ -public class SparkUtilities { - - public static HiveKey copyHiveKey(HiveKey key) { - HiveKey copy = new HiveKey(); - copy.setDistKeyLength(key.getDistKeyLength()); - copy.setHashCode(key.hashCode()); - copy.set(key); - return copy; - } - - public static BytesWritable copyBytesWritable(BytesWritable bw) { - BytesWritable copy = new BytesWritable(); - copy.set(bw); - return copy; - } - - /** - * Uploads a local file to HDFS - * - * @param source - * @param conf - * @return - * @throws IOException - */ - public static URI uploadToHDFS(URI source, HiveConf conf) throws IOException { - Path localFile = new Path(source.getPath()); - Path remoteFile = new Path(SessionState.get().getSparkSession().getHDFSSessionDir(), - getFileName(source)); - FileSystem fileSystem = FileSystem.get(remoteFile.toUri(), conf); - // Overwrite if the remote file already exists. Whether the file can be added - // on executor is up to spark, i.e. spark.files.overwrite - fileSystem.copyFromLocalFile(false, true, localFile, remoteFile); - Path fullPath = fileSystem.getFileStatus(remoteFile).getPath(); - return fullPath.toUri(); - } - - // checks if a resource has to be uploaded to HDFS for yarn-cluster mode - public static boolean needUploadToHDFS(URI source, SparkConf sparkConf) { - String master = sparkConf.get("spark.master"); - String deployMode = sparkConf.contains("spark.submit.deployMode") ? - sparkConf.get("spark.submit.deployMode") : null; - return SparkClientUtilities.isYarnClusterMode(master, deployMode) && - !(source.getScheme().equals("hdfs") || source.getScheme().equals("viewfs")); - } - - private static String getFileName(URI uri) { - if (uri == null) { - return null; - } - - String name = FilenameUtils.getName(uri.getPath()); - return name; - } - - public static boolean isDedicatedCluster(Configuration conf) { - String master = conf.get("spark.master"); - return SparkClientUtilities.isYarnMaster(master) || SparkClientUtilities.isLocalMaster(master); - } - - public static SparkSession getSparkSession(HiveConf conf, - SparkSessionManager sparkSessionManager) throws HiveException { - SparkSession sparkSession = SessionState.get().getSparkSession(); - HiveConf sessionConf = SessionState.get().getConf(); - - // Spark configurations are updated close the existing session - // In case of async queries or confOverlay is not empty, - // sessionConf and conf are different objects - if (sessionConf.getSparkConfigUpdated() || conf.getSparkConfigUpdated()) { - sparkSessionManager.closeSession(sparkSession); - sparkSession = null; - conf.setSparkConfigUpdated(false); - sessionConf.setSparkConfigUpdated(false); - } - sparkSession = sparkSessionManager.getSession(sparkSession, conf, true); - SessionState.get().setSparkSession(sparkSession); - return sparkSession; - } - - /** - * Generate a temporary path for dynamic partition pruning in Spark branch - * TODO: no longer need this if we use accumulator! - * @param basePath - * @param id - * @return - */ - public static Path generateTmpPathForPartitionPruning(Path basePath, String id) { - return new Path(basePath, id); - } - - /** - * Return the ID for this BaseWork, in String form. - * @param work the input BaseWork - * @return the unique ID for this BaseWork - */ - public static String getWorkId(BaseWork work) { - String workName = work.getName(); - return workName.substring(workName.indexOf(" ") + 1); - } - - public static SparkTask createSparkTask(HiveConf conf) { - return (SparkTask) TaskFactory.get( - new SparkWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID))); - } - - public static SparkTask createSparkTask(SparkWork work, HiveConf conf) { - return (SparkTask) TaskFactory.get(work); - } - - /** - * Recursively find all operators under root, that are of class clazz or are the sub-class of clazz, and - * put them in result. - * @param result all operators under root that are of class clazz - * @param root the root operator under which all operators will be examined - * @param clazz clas to collect. Must NOT be null. - */ - public static void collectOp(Collection> result, Operator root, Class clazz) { - Preconditions.checkArgument(clazz != null, "AssertionError: clazz should not be null"); - if (root == null) { - return; - } - if (clazz.isAssignableFrom(root.getClass())) { - result.add(root); - } - for (Operator child : root.getChildOperators()) { - collectOp(result, child, clazz); - } - } - - /** - * Collect operators of type T starting from root. Matching operators will be put into result. - * Set seen can be used to skip search in certain branches. - */ - public static > void collectOp(Operator root, Class cls, - Collection result, Set> seen) { - if (seen.contains(root)) { - return; - } - Deque> deque = new ArrayDeque<>(); - deque.add(root); - while (!deque.isEmpty()) { - Operator op = deque.remove(); - seen.add(op); - if (cls.isInstance(op)) { - result.add((T) op); - } - if (op.getChildOperators() != null) { - for (Operator child : op.getChildOperators()) { - if (!seen.contains(child)) { - deque.add(child); - } - } - } - } - } - - /** - * remove currTask from the children of its parentTask - * remove currTask from the parent of its childrenTask - * @param currTask - */ - public static void removeEmptySparkTask(SparkTask currTask) { - //remove currTask from parentTasks - ArrayList parTasks = new ArrayList(); - parTasks.addAll(currTask.getParentTasks()); - - Object[] parTaskArr = parTasks.toArray(); - for (Object parTask : parTaskArr) { - ((Task) parTask).removeDependentTask(currTask); - } - //remove currTask from childTasks - currTask.removeFromChildrenTasks(); - } - - // Find if there's any DPP sink branch of the branchingOP that is equivalent - // to the branch represented by the list. - public static SparkPartitionPruningSinkOperator findReusableDPPSink( - Operator branchingOP, List> list) { - for (Operator other : branchingOP.getChildOperators()) { - int i; - for (i = 0; i < list.size(); i++) { - if (other == list.get(i) || !other.logicalEquals(list.get(i))) { - break; - } - if (i != list.size() - 1) { - if (other.getChildOperators() == null || other.getChildOperators().size() != 1) { - break; - } - other = other.getChildOperators().get(0); - } - } - if (i == list.size()) { - return (SparkPartitionPruningSinkOperator) other; - } - } - return null; - } - - /** - * For DPP sinks w/ common join, we'll split the tree and what's above the branching - * operator is computed multiple times. Therefore it may not be good for performance to support - * nested DPP sinks, i.e. one DPP sink depends on other DPP sinks. - * The following is an example: - * - * TS TS - * | | - * ... FIL - * | | \ - * RS RS SEL - * \ / | - * TS JOIN GBY - * | / \ | - * RS RS SEL DPP2 - * \ / | - * JOIN GBY - * | - * DPP1 - * - * where DPP1 depends on DPP2. - * - * To avoid such case, we'll visit all the branching operators. If a branching operator has any - * further away DPP branches w/ common join in its sub-tree, such branches will be removed. - * In the above example, the branch of DPP1 will be removed. - */ - public static void removeNestedDPP(OptimizeSparkProcContext procContext) { - Set allDPPs = new HashSet<>(); - Set> seen = new HashSet<>(); - // collect all DPP sinks - for (TableScanOperator root : procContext.getParseContext().getTopOps().values()) { - SparkUtilities.collectOp(root, SparkPartitionPruningSinkOperator.class, allDPPs, seen); - } - // collect all branching operators - Set> branchingOps = new HashSet<>(); - for (SparkPartitionPruningSinkOperator dpp : allDPPs) { - branchingOps.add(dpp.getBranchingOp()); - } - // remember the branching ops we have visited - Set> visited = new HashSet<>(); - for (Operator branchingOp : branchingOps) { - if (!visited.contains(branchingOp)) { - visited.add(branchingOp); - seen.clear(); - Set nestedDPPs = new HashSet<>(); - for (Operator branch : branchingOp.getChildOperators()) { - if (!isDirectDPPBranch(branch)) { - SparkUtilities.collectOp(branch, SparkPartitionPruningSinkOperator.class, nestedDPPs, - seen); - } - } - for (SparkPartitionPruningSinkOperator nestedDPP : nestedDPPs) { - visited.add(nestedDPP.getBranchingOp()); - // if a DPP is with MJ, the tree won't be split and so we don't have to remove it - if (!nestedDPP.isWithMapjoin()) { - OperatorUtils.removeBranch(nestedDPP); - } - } - } - } - } - - // whether of pattern "SEL - GBY - DPP" - public static boolean isDirectDPPBranch(Operator op) { - if (op instanceof SelectOperator && op.getChildOperators() != null - && op.getChildOperators().size() == 1) { - op = op.getChildOperators().get(0); - if (op instanceof GroupByOperator && op.getChildOperators() != null - && op.getChildOperators().size() == 1) { - op = op.getChildOperators().get(0); - return op instanceof SparkPartitionPruningSinkOperator; - } - } - return false; - } - - public static String reverseDNSLookupURL(String url) throws UnknownHostException { - // Run a reverse DNS lookup on the URL - URI uri = URI.create(url); - InetAddress address = InetAddress.getByName(uri.getHost()); - return uri.getScheme() + "://" + address.getCanonicalHostName() + ":" + uri.getPort(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistic.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistic.java deleted file mode 100644 index a69773034ae..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistic.java +++ /dev/null @@ -1,36 +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.hadoop.hive.ql.exec.spark.Statistic; - -public class SparkStatistic { - private final String name; - private final String value; - - SparkStatistic(String name, String value) { - this.name = name; - this.value = value; - } - - public String getValue() { - return value; - } - - public String getName() { - return name; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticGroup.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticGroup.java deleted file mode 100644 index e1006e383e9..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticGroup.java +++ /dev/null @@ -1,50 +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.hadoop.hive.ql.exec.spark.Statistic; - -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -public class SparkStatisticGroup { - private final String groupName; - private final Map statistics = new LinkedHashMap<>(); - - SparkStatisticGroup(String groupName, List statisticList) { - this.groupName = groupName; - for (SparkStatistic sparkStatistic : statisticList) { - this.statistics.put(sparkStatistic.getName(), sparkStatistic); - } - } - - public String getGroupName() { - return groupName; - } - - public Iterator getStatistics() { - return this.statistics.values().iterator(); - } - - /** - * Get a {@link SparkStatistic} by its given name - */ - public SparkStatistic getSparkStatistic(String name) { - return this.statistics.get(name); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistics.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistics.java deleted file mode 100644 index 946cadc0ab1..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatistics.java +++ /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. - */ -package org.apache.hadoop.hive.ql.exec.spark.Statistic; - -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -public class SparkStatistics { - - private final Map statisticGroups = new LinkedHashMap<>(); - - SparkStatistics(List statisticGroupsList) { - for (SparkStatisticGroup group : statisticGroupsList) { - statisticGroups.put(group.getGroupName(), group); - } - } - - public Iterator getStatisticGroups() { - return this.statisticGroups.values().iterator(); - } - - public SparkStatisticGroup getStatisticGroup(String groupName) { - return this.statisticGroups.get(groupName); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsBuilder.java deleted file mode 100644 index d31d60a1f46..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsBuilder.java +++ /dev/null @@ -1,73 +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.hadoop.hive.ql.exec.spark.Statistic; - -import org.apache.hive.spark.counter.SparkCounter; -import org.apache.hive.spark.counter.SparkCounterGroup; -import org.apache.hive.spark.counter.SparkCounters; - -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -public class SparkStatisticsBuilder { - - private Map> statisticMap; - - public SparkStatisticsBuilder() { - statisticMap = new LinkedHashMap<>(); - } - - public SparkStatistics build() { - List statisticGroups = new LinkedList(); - for (Map.Entry> entry : statisticMap.entrySet()) { - String groupName = entry.getKey(); - List statisticList = entry.getValue(); - statisticGroups.add(new SparkStatisticGroup(groupName, statisticList)); - } - - return new SparkStatistics(statisticGroups); - } - - public SparkStatisticsBuilder add(SparkCounters sparkCounters) { - for (SparkCounterGroup counterGroup : sparkCounters.getSparkCounterGroups().values()) { - String groupDisplayName = counterGroup.getGroupDisplayName(); - List statisticList = statisticMap.get(groupDisplayName); - if (statisticList == null) { - statisticList = new LinkedList(); - statisticMap.put(groupDisplayName, statisticList); - } - for (SparkCounter counter : counterGroup.getSparkCounters().values()) { - String displayName = counter.getDisplayName(); - statisticList.add(new SparkStatistic(displayName, Long.toString(counter.getValue()))); - } - } - return this; - } - - public SparkStatisticsBuilder add(String groupName, String name, String value) { - List statisticList = statisticMap.get(groupName); - if (statisticList == null) { - statisticList = new LinkedList(); - statisticMap.put(groupName, statisticList); - } - statisticList.add(new SparkStatistic(name, value)); - return this; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsNames.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsNames.java deleted file mode 100644 index 68e4f9e4560..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/Statistic/SparkStatisticsNames.java +++ /dev/null @@ -1,45 +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.hadoop.hive.ql.exec.spark.Statistic; - -/** - * A collection of names that define different {@link SparkStatistic} objects. - */ -public class SparkStatisticsNames { - - public static final String EXECUTOR_DESERIALIZE_TIME = "ExecutorDeserializeTime"; - public static final String EXECUTOR_DESERIALIZE_CPU_TIME = "ExecutorDeserializeCpuTime"; - public static final String EXECUTOR_RUN_TIME = "ExecutorRunTime"; - public static final String EXECUTOR_CPU_TIME = "ExecutorCpuTime"; - public static final String RESULT_SIZE = "ResultSize"; - public static final String JVM_GC_TIME = "JvmGCTime"; - public static final String RESULT_SERIALIZATION_TIME = "ResultSerializationTime"; - public static final String MEMORY_BYTES_SPILLED = "MemoryBytesSpilled"; - public static final String DISK_BYTES_SPILLED = "DiskBytesSpilled"; - public static final String BYTES_READ = "BytesRead"; - public static final String REMOTE_BLOCKS_FETCHED = "RemoteBlocksFetched"; - public static final String LOCAL_BLOCKS_FETCHED = "LocalBlocksFetched"; - public static final String TOTAL_BLOCKS_FETCHED = "TotalBlocksFetched"; - public static final String FETCH_WAIT_TIME = "FetchWaitTime"; - public static final String REMOTE_BYTES_READ = "RemoteBytesRead"; - public static final String SHUFFLE_BYTES_WRITTEN = "ShuffleBytesWritten"; - public static final String SHUFFLE_WRITE_TIME = "ShuffleWriteTime"; - public static final String TASK_DURATION_TIME = "TaskDurationTime"; - - public static final String SPARK_GROUP_NAME = "SPARK"; -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java deleted file mode 100644 index f96a8f77ce0..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSession.java +++ /dev/null @@ -1,78 +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.hadoop.hive.ql.exec.spark.session; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.SparkWork; - -import java.io.IOException; - -public interface SparkSession { - /** - * Initializes a Spark session for DAG execution. - * @param conf Hive configuration. - */ - void open(HiveConf conf) throws HiveException; - - /** - * Submit given sparkWork to SparkClient. - * @param driverContext - * @param sparkWork - * @return SparkJobRef - */ - SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception; - - /** - * Get Spark shuffle memory per task, and total number of cores. This - * information can be used to estimate how many reducers a task can have. - * - * @return an object pair, the first element is the shuffle memory per task in bytes, - * the second element is the number of total cores usable by the client - */ - ObjectPair getMemoryAndCores() throws Exception; - - /** - * @return true if the session is open and ready to submit jobs. - */ - boolean isOpen(); - - /** - * @return configuration. - */ - HiveConf getConf(); - - /** - * @return session id. - */ - String getSessionId(); - - /** - * Close session and release resources. - */ - void close(); - - /** - * Get an HDFS dir specific to the SparkSession - * */ - Path getHDFSSessionDir() throws IOException; -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java deleted file mode 100644 index 2d5d03ee71b..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionImpl.java +++ /dev/null @@ -1,271 +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.hadoop.hive.ql.exec.spark.session; - -import java.io.IOException; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.TimeoutException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClient; -import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClientFactory; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.spark.SparkConf; -import org.apache.spark.util.Utils; - -import com.google.common.base.Preconditions; - -public class SparkSessionImpl implements SparkSession { - private static final Logger LOG = LoggerFactory.getLogger(SparkSession.class); - private static final String SPARK_DIR = "_spark_session_dir"; - - /** Regex for different Spark session error messages */ - private static final String AM_TIMEOUT_ERR = ".*ApplicationMaster for attempt.*timed out.*"; - private static final String UNKNOWN_QUEUE_ERR = "(submitted by user.*to unknown queue:.*)\n"; - private static final String STOPPED_QUEUE_ERR = "(Queue.*is STOPPED)"; - private static final String FULL_QUEUE_ERR = "(Queue.*already has.*applications)"; - private static final String INVALILD_MEM_ERR = - "(Required executor memory.*is above the max threshold.*) of this"; - private static final String INVALID_CORE_ERR = - "(initial executor number.*must between min executor.*and max executor number.*)\n"; - - /** Pre-compiled error patterns. Shared between all Spark sessions */ - private static Map errorPatterns; - - private HiveConf conf; - private boolean isOpen; - private final String sessionId; - private HiveSparkClient hiveSparkClient; - private Path scratchDir; - private final Object dirLock = new Object(); - private String matchedString = null; - - public SparkSessionImpl() { - sessionId = makeSessionId(); - initErrorPatterns(); - } - - @Override - public void open(HiveConf conf) throws HiveException { - LOG.info("Trying to open Spark session {}", sessionId); - this.conf = conf; - isOpen = true; - try { - hiveSparkClient = HiveSparkClientFactory.createHiveSparkClient(conf, sessionId); - } catch (Throwable e) { - // It's possible that user session is closed while creating Spark client. - HiveException he; - if (isOpen) { - he = getHiveException(e); - } else { - he = new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_CLOSED_SESSION, sessionId); - } - throw he; - } - LOG.info("Spark session {} is successfully opened", sessionId); - } - - @Override - public SparkJobRef submit(DriverContext driverContext, SparkWork sparkWork) throws Exception { - Preconditions.checkState(isOpen, "Session is not open. Can't submit jobs."); - return hiveSparkClient.execute(driverContext, sparkWork); - } - - @Override - public ObjectPair getMemoryAndCores() throws Exception { - SparkConf sparkConf = hiveSparkClient.getSparkConf(); - int numExecutors = hiveSparkClient.getExecutorCount(); - // at start-up, we may be unable to get number of executors - if (numExecutors <= 0) { - return new ObjectPair(-1L, -1); - } - int executorMemoryInMB = Utils.memoryStringToMb( - sparkConf.get("spark.executor.memory", "512m")); - double memoryFraction = 1.0 - sparkConf.getDouble("spark.storage.memoryFraction", 0.6); - long totalMemory = (long) (numExecutors * executorMemoryInMB * memoryFraction * 1024 * 1024); - int totalCores; - String masterURL = sparkConf.get("spark.master"); - if (masterURL.startsWith("spark") || masterURL.startsWith("local")) { - totalCores = sparkConf.contains("spark.default.parallelism") ? - sparkConf.getInt("spark.default.parallelism", 1) : - hiveSparkClient.getDefaultParallelism(); - totalCores = Math.max(totalCores, numExecutors); - } else { - int coresPerExecutor = sparkConf.getInt("spark.executor.cores", 1); - totalCores = numExecutors * coresPerExecutor; - } - totalCores = totalCores / sparkConf.getInt("spark.task.cpus", 1); - - long memoryPerTaskInBytes = totalMemory / totalCores; - LOG.info("Spark cluster current has executors: " + numExecutors - + ", total cores: " + totalCores + ", memory per executor: " - + executorMemoryInMB + "M, memoryFraction: " + memoryFraction); - return new ObjectPair(Long.valueOf(memoryPerTaskInBytes), - Integer.valueOf(totalCores)); - } - - @Override - public boolean isOpen() { - return isOpen; - } - - @Override - public HiveConf getConf() { - return conf; - } - - @Override - public String getSessionId() { - return sessionId; - } - - @Override - public void close() { - LOG.info("Trying to close Spark session {}", sessionId); - isOpen = false; - if (hiveSparkClient != null) { - try { - hiveSparkClient.close(); - LOG.info("Spark session {} is successfully closed", sessionId); - cleanScratchDir(); - } catch (IOException e) { - LOG.error("Failed to close spark session (" + sessionId + ").", e); - } - } - hiveSparkClient = null; - } - - private Path createScratchDir() throws IOException { - Path parent = new Path(SessionState.get().getHdfsScratchDirURIString(), SPARK_DIR); - Path sparkDir = new Path(parent, sessionId); - FileSystem fs = sparkDir.getFileSystem(conf); - FsPermission fsPermission = new FsPermission(HiveConf.getVar( - conf, HiveConf.ConfVars.SCRATCHDIRPERMISSION)); - fs.mkdirs(sparkDir, fsPermission); - fs.deleteOnExit(sparkDir); - return sparkDir; - } - - private static void initErrorPatterns() { - errorPatterns = Maps.newHashMap( - new ImmutableMap.Builder() - .put(AM_TIMEOUT_ERR, Pattern.compile(AM_TIMEOUT_ERR)) - .put(UNKNOWN_QUEUE_ERR, Pattern.compile(UNKNOWN_QUEUE_ERR)) - .put(STOPPED_QUEUE_ERR, Pattern.compile(STOPPED_QUEUE_ERR)) - .put(FULL_QUEUE_ERR, Pattern.compile(FULL_QUEUE_ERR)) - .put(INVALILD_MEM_ERR, Pattern.compile(INVALILD_MEM_ERR)) - .put(INVALID_CORE_ERR, Pattern.compile(INVALID_CORE_ERR)) - .build() - ); - } - - @VisibleForTesting - HiveException getHiveException(Throwable e) { - Throwable oe = e; - while (e != null) { - if (e instanceof TimeoutException) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_TIMEOUT); - } else if (e instanceof InterruptedException) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_INTERRUPTED, sessionId); - } else if (e instanceof RuntimeException) { - String sts = Throwables.getStackTraceAsString(e); - if (matches(sts, AM_TIMEOUT_ERR)) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_TIMEOUT); - } else if (matches(sts, UNKNOWN_QUEUE_ERR) || matches(sts, STOPPED_QUEUE_ERR)) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_QUEUE, matchedString); - } else if (matches(sts, FULL_QUEUE_ERR)) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_QUEUE_FULL, matchedString); - } else if (matches(sts, INVALILD_MEM_ERR) || matches(sts, INVALID_CORE_ERR)) { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST, - matchedString); - } else { - return new HiveException(e, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId); - } - } - e = e.getCause(); - } - - return new HiveException(oe, ErrorMsg.SPARK_CREATE_CLIENT_ERROR, sessionId); - } - - @VisibleForTesting - String getMatchedString() { - return matchedString; - } - - private boolean matches(String input, String regex) { - if (!errorPatterns.containsKey(regex)) { - LOG.warn("No error pattern found for regex: {}", regex); - return false; - } - Pattern p = errorPatterns.get(regex); - Matcher m = p.matcher(input); - boolean result = m.find(); - if (result && m.groupCount() == 1) { - this.matchedString = m.group(1); - } - return result; - } - - private void cleanScratchDir() throws IOException { - if (scratchDir != null) { - FileSystem fs = scratchDir.getFileSystem(conf); - fs.delete(scratchDir, true); - scratchDir = null; - } - } - - @Override - public Path getHDFSSessionDir() throws IOException { - if (scratchDir == null) { - synchronized (dirLock) { - if (scratchDir == null) { - scratchDir = createScratchDir(); - } - } - } - return scratchDir; - } - - public static String makeSessionId() { - return UUID.randomUUID().toString(); - } - - @VisibleForTesting - HiveSparkClient getHiveSparkClient() { - return hiveSparkClient; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java deleted file mode 100644 index 227477ae9dc..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManager.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hive.ql.exec.spark.session; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.metadata.HiveException; - -/** - * Defines interface for managing multiple SparkSessions in Hive when multiple users - * are executing queries simultaneously on Spark execution engine. - */ -public interface SparkSessionManager { - /** - * Initialize based on given configuration. - * - * @param hiveConf - */ - void setup(HiveConf hiveConf) throws HiveException; - - /** - * Get a valid SparkSession. First try to check if existing session is reusable - * based on the given conf. If not release existingSession and return - * a new session based on session manager criteria and conf. - * - * @param existingSession Existing session (can be null) - * @param conf - * @param doOpen Should the session be opened before returning? - * @return SparkSession - */ - SparkSession getSession(SparkSession existingSession, HiveConf conf, - boolean doOpen) throws HiveException; - - /** - * Return the given sparkSession to pool. This is used when the client - * still holds references to session and may want to reuse it in future. - * When client wants to reuse the session, it should pass the it getSession method. - */ - void returnSession(SparkSession sparkSession) throws HiveException; - - /** - * Close the given session and return it to pool. This is used when the client - * no longer needs a SparkSession. - */ - void closeSession(SparkSession sparkSession) throws HiveException; - - /** - * Shutdown the session manager. Also closing up SparkSessions in pool. - */ - void shutdown(); -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java deleted file mode 100644 index 46cee0d903d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/session/SparkSessionManagerImpl.java +++ /dev/null @@ -1,158 +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.hadoop.hive.ql.exec.spark.session; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; - -import org.apache.hive.common.util.ShutdownHookManager; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClientFactory; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hive.spark.client.SparkClientFactory; - -/** - * Simple implementation of SparkSessionManager - * - returns SparkSession when requested through getSession and keeps track of - * created sessions. Currently no limit on the number sessions. - * - SparkSession is reused if the userName in new conf and user name in session conf match. - */ -public class SparkSessionManagerImpl implements SparkSessionManager { - private static final Logger LOG = LoggerFactory.getLogger(SparkSessionManagerImpl.class); - - private Set createdSessions = Collections.synchronizedSet(new HashSet()); - private volatile boolean inited = false; - - private static SparkSessionManagerImpl instance; - - static { - ShutdownHookManager.addShutdownHook(new Runnable() { - @Override - public void run() { - try { - if (instance != null) { - instance.shutdown(); - } - } catch (Exception e) { - // ignore - } - } - }); - } - - public static synchronized SparkSessionManagerImpl getInstance() - throws HiveException { - if (instance == null) { - instance = new SparkSessionManagerImpl(); - } - - return instance; - } - - private SparkSessionManagerImpl() { - } - - @Override - public void setup(HiveConf hiveConf) throws HiveException { - if (!inited) { - synchronized (this) { - if (!inited) { - LOG.info("Setting up the session manager."); - Map conf = HiveSparkClientFactory.initiateSparkConf(hiveConf, null); - try { - SparkClientFactory.initialize(conf); - inited = true; - } catch (IOException e) { - throw new HiveException("Error initializing SparkClientFactory", e); - } - } - } - } - } - - /** - * If the existingSession can be reused return it. - * Otherwise - * - close it and remove it from the list. - * - create a new session and add it to the list. - */ - @Override - public SparkSession getSession(SparkSession existingSession, HiveConf conf, boolean doOpen) - throws HiveException { - setup(conf); - - if (existingSession != null) { - // Open the session if it is closed. - if (!existingSession.isOpen() && doOpen) { - existingSession.open(conf); - } - return existingSession; - } - - SparkSession sparkSession = new SparkSessionImpl(); - if (doOpen) { - sparkSession.open(conf); - } - - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("New session (%s) is created.", sparkSession.getSessionId())); - } - createdSessions.add(sparkSession); - return sparkSession; - } - - @Override - public void returnSession(SparkSession sparkSession) throws HiveException { - // In this particular SparkSessionManager implementation, we don't recycle - // returned sessions. References to session are still valid. - } - - @Override - public void closeSession(SparkSession sparkSession) throws HiveException { - if (sparkSession == null) { - return; - } - - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Closing session (%s).", sparkSession.getSessionId())); - } - sparkSession.close(); - createdSessions.remove(sparkSession); - } - - @Override - public void shutdown() { - LOG.info("Closing the session manager."); - synchronized (createdSessions) { - Iterator it = createdSessions.iterator(); - while (it.hasNext()) { - SparkSession session = it.next(); - session.close(); - } - createdSessions.clear(); - } - inited = false; - SparkClientFactory.stop(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java deleted file mode 100644 index 05253157d05..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/LocalSparkJobMonitor.java +++ /dev/null @@ -1,142 +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.hadoop.hive.ql.exec.spark.status; - -import java.util.Map; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.spark.JobExecutionStatus; - -/** - * LocalSparkJobMonitor monitor a single Spark job status in a loop until job finished/failed/killed. - * It print current job status to console and sleep current thread between monitor interval. - */ -public class LocalSparkJobMonitor extends SparkJobMonitor { - - private SparkJobStatus sparkJobStatus; - - public LocalSparkJobMonitor(HiveConf hiveConf, SparkJobStatus sparkJobStatus) { - super(hiveConf); - this.sparkJobStatus = sparkJobStatus; - } - - public int startMonitor() { - boolean running = false; - boolean done = false; - int rc = 0; - JobExecutionStatus lastState = null; - Map lastProgressMap = null; - - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - - startTime = System.currentTimeMillis(); - - while (true) { - try { - JobExecutionStatus state = sparkJobStatus.getState(); - if (LOG.isDebugEnabled()) { - console.printInfo("state = " + state); - } - - if (state == null) { - long timeCount = (System.currentTimeMillis() - startTime)/1000; - if (timeCount > monitorTimeoutInterval) { - console.printError("Job hasn't been submitted after " + timeCount + "s. Aborting it."); - console.printError("Status: " + state); - running = false; - done = true; - rc = 2; - break; - } - } else if (state != lastState || state == JobExecutionStatus.RUNNING) { - lastState = state; - Map progressMap = sparkJobStatus.getSparkStageProgress(); - - switch (state) { - case RUNNING: - if (!running) { - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - // print job stages. - console.printInfo("\nQuery Hive on Spark job[" - + sparkJobStatus.getJobId() + "] stages:"); - for (int stageId : sparkJobStatus.getStageIds()) { - console.printInfo(Integer.toString(stageId)); - } - - console.printInfo("\nStatus: Running (Hive on Spark job[" - + sparkJobStatus.getJobId() + "])"); - running = true; - - console.printInfo("Job Progress Format\nCurrentTime StageId_StageAttemptId: " - + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount [StageCost]"); - } - - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - break; - case SUCCEEDED: - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Status: Finished successfully in " - + String.format("%.2f seconds", duration)); - running = false; - done = true; - break; - case FAILED: - console.printError("Status: Failed"); - running = false; - done = true; - rc = 3; - break; - case UNKNOWN: - console.printError("Status: Unknown"); - running = false; - done = true; - rc = 4; - break; - } - } - if (!done) { - Thread.sleep(checkInterval); - } - } catch (Exception e) { - String msg = " with exception '" + Utilities.getNameMessage(e) + "'"; - msg = "Failed to monitor Job[ " + sparkJobStatus.getJobId() + "]" + msg; - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - LOG.error(msg, e); - console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - rc = 1; - done = true; - sparkJobStatus.setError(e); - } finally { - if (done) { - break; - } - } - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - return rc; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java deleted file mode 100644 index a132f74f9e8..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/RemoteSparkJobMonitor.java +++ /dev/null @@ -1,228 +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.hadoop.hive.ql.exec.spark.status; - -import java.util.Arrays; -import java.util.Map; - -import com.google.common.base.Preconditions; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hive.spark.client.JobHandle; -import org.apache.spark.JobExecutionStatus; - -/** - * RemoteSparkJobMonitor monitor a RSC remote job status in a loop until job finished/failed/killed. - * It print current job status to console and sleep current thread between monitor interval. - */ -public class RemoteSparkJobMonitor extends SparkJobMonitor { - private int sparkJobMaxTaskCount = -1; - private int sparkStageMaxTaskCount = -1; - private int totalTaskCount = 0; - private int stageMaxTaskCount = 0; - private RemoteSparkJobStatus sparkJobStatus; - private final HiveConf hiveConf; - - public RemoteSparkJobMonitor(HiveConf hiveConf, RemoteSparkJobStatus sparkJobStatus) { - super(hiveConf); - this.sparkJobStatus = sparkJobStatus; - this.hiveConf = hiveConf; - sparkJobMaxTaskCount = hiveConf.getIntVar(HiveConf.ConfVars.SPARK_JOB_MAX_TASKS); - sparkStageMaxTaskCount = hiveConf.getIntVar(HiveConf.ConfVars.SPARK_STAGE_MAX_TASKS); - } - - @Override - public int startMonitor() { - boolean running = false; - boolean done = false; - int rc = 0; - Map lastProgressMap = null; - - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - - startTime = System.currentTimeMillis(); - JobHandle.State state = null; - - while (true) { - try { - state = sparkJobStatus.getRemoteJobState(); - Preconditions.checkState(sparkJobStatus.isRemoteActive(), "Connection to remote Spark driver was lost"); - - switch (state) { - case SENT: - case QUEUED: - long timeCount = (System.currentTimeMillis() - startTime) / 1000; - if ((timeCount > monitorTimeoutInterval)) { - HiveException he = new HiveException(ErrorMsg.SPARK_JOB_MONITOR_TIMEOUT, - Long.toString(timeCount)); - console.printError(he.getMessage()); - sparkJobStatus.setError(he); - running = false; - done = true; - rc = 2; - } - if (LOG.isDebugEnabled()) { - console.printInfo("Spark job state = " + state ); - } - break; - case STARTED: - JobExecutionStatus sparkJobState = sparkJobStatus.getState(); - if (sparkJobState == JobExecutionStatus.RUNNING) { - Map progressMap = sparkJobStatus.getSparkStageProgress(); - if (!running) { - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_SUBMIT_TO_RUNNING); - printAppInfo(); - console.printInfo("Hive on Spark Session Web UI URL: " + sparkJobStatus.getWebUIURL()); - // print job stages. - console.printInfo("\nQuery Hive on Spark job[" + sparkJobStatus.getJobId() + - "] stages: " + Arrays.toString(sparkJobStatus.getStageIds())); - - console.printInfo("Spark job[" + sparkJobStatus.getJobId() + "] status = RUNNING"); - running = true; - - String format = "Job Progress Format\nCurrentTime StageId_StageAttemptId: " - + "SucceededTasksCount(+RunningTasksCount-FailedTasksCount)/TotalTasksCount"; - if (!inPlaceUpdate) { - console.printInfo(format); - } else { - console.logInfo(format); - } - } else { - // Get the maximum of the number of tasks in the stages of the job and cancel the job if it goes beyond the limit. - if (sparkStageMaxTaskCount != -1 && stageMaxTaskCount == 0) { - stageMaxTaskCount = getStageMaxTaskCount(progressMap); - if (stageMaxTaskCount > sparkStageMaxTaskCount) { - rc = 4; - done = true; - console.printInfo("\nThe number of task in one stage of the Spark job [" + stageMaxTaskCount + "] is greater than the limit [" + - sparkStageMaxTaskCount + "]. The Spark job will be cancelled."); - } - } - - // Count the number of tasks, and kill application if it goes beyond the limit. - if (sparkJobMaxTaskCount != -1 && totalTaskCount == 0) { - totalTaskCount = getTotalTaskCount(progressMap); - if (totalTaskCount > sparkJobMaxTaskCount) { - rc = 4; - done = true; - console.printInfo("\nThe total number of task in the Spark job [" + totalTaskCount + "] is greater than the limit [" + - sparkJobMaxTaskCount + "]. The Spark job will be cancelled."); - } - } - } - - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - } - break; - case SUCCEEDED: - Map progressMap = sparkJobStatus.getSparkStageProgress(); - printStatus(progressMap, lastProgressMap); - lastProgressMap = progressMap; - double duration = (System.currentTimeMillis() - startTime) / 1000.0; - console.printInfo("Spark job[" + sparkJobStatus.getJobId() + "] finished successfully in " - + String.format("%.2f second(s)", duration)); - running = false; - done = true; - break; - case FAILED: - String detail = sparkJobStatus.getError().getMessage(); - StringBuilder errBuilder = new StringBuilder(); - errBuilder.append("Job failed with "); - if (detail == null) { - errBuilder.append("UNKNOWN reason"); - } else { - // We SerDe the Throwable as String, parse it for the root cause - final String CAUSE_CAPTION = "Caused by: "; - int index = detail.lastIndexOf(CAUSE_CAPTION); - if (index != -1) { - String rootCause = detail.substring(index + CAUSE_CAPTION.length()); - index = rootCause.indexOf(System.getProperty("line.separator")); - if (index != -1) { - errBuilder.append(rootCause.substring(0, index)); - } else { - errBuilder.append(rootCause); - } - } else { - errBuilder.append(detail); - } - detail = System.getProperty("line.separator") + detail; - } - console.printError(errBuilder.toString(), detail); - running = false; - done = true; - rc = 3; - break; - case CANCELLED: - console.printInfo("Spark job[" + sparkJobStatus.getJobId() + " was cancelled"); - running = false; - done = true; - rc = 3; - break; - } - - if (!done) { - Thread.sleep(checkInterval); - } - } catch (Exception e) { - Exception finalException = e; - if (e instanceof InterruptedException || - (e instanceof HiveException && e.getCause() instanceof InterruptedException)) { - finalException = new HiveException(e, ErrorMsg.SPARK_JOB_INTERRUPTED); - LOG.warn("Interrupted while monitoring the Hive on Spark application, exiting"); - } else { - String msg = " with exception '" + Utilities.getNameMessage(e) + "' Last known state = " + - (state != null ? state.name() : "UNKNOWN"); - msg = "Failed to monitor Job[" + sparkJobStatus.getJobId() + "]" + msg; - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - console.printError(msg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - rc = 1; - done = true; - sparkJobStatus.setError(finalException); - } finally { - if (done) { - break; - } - } - } - - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_JOB); - return rc; - } - - private void printAppInfo() { - String sparkMaster = hiveConf.get("spark.master"); - if (sparkMaster != null && sparkMaster.startsWith("yarn")) { - String appID = sparkJobStatus.getAppID(); - if (appID != null) { - console.printInfo("Running with YARN Application = " + appID); - console.printInfo("Kill Command = " + - HiveConf.getVar(hiveConf, HiveConf.ConfVars.YARNBIN) + " application -kill " + appID); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java deleted file mode 100644 index 7afd8864075..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobMonitor.java +++ /dev/null @@ -1,376 +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.hadoop.hive.ql.exec.spark.status; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.common.log.InPlaceUpdate; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.fusesource.jansi.Ansi; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.PrintStream; -import java.text.DecimalFormat; -import java.text.NumberFormat; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; - -import static org.fusesource.jansi.Ansi.ansi; - -abstract class SparkJobMonitor { - - protected static final String CLASS_NAME = SparkJobMonitor.class.getName(); - protected static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); - protected transient final SessionState.LogHelper console; - protected final PerfLogger perfLogger = SessionState.getPerfLogger(); - protected final int checkInterval = 1000; - protected final long monitorTimeoutInterval; - - private final Set completed = new HashSet(); - private final int printInterval = 3000; - private long lastPrintTime; - - protected long startTime; - - protected enum StageState { - PENDING, - RUNNING, - FINISHED - } - - // in-place progress update related variables - protected final boolean inPlaceUpdate; - private int lines = 0; - private final PrintStream out; - - private static final int COLUMN_1_WIDTH = 16; - private static final String HEADER_FORMAT = "%16s%10s %13s %5s %9s %7s %7s %6s "; - private static final String STAGE_FORMAT = "%-16s%10s %13s %5s %9s %7s %7s %6s "; - private static final String HEADER = String.format(HEADER_FORMAT, - "STAGES", "ATTEMPT", "STATUS", "TOTAL", "COMPLETED", "RUNNING", "PENDING", "FAILED"); - private static final int SEPARATOR_WIDTH = 86; - private static final String SEPARATOR = new String(new char[SEPARATOR_WIDTH]).replace("\0", "-"); - private static final String FOOTER_FORMAT = "%-15s %-30s %-4s %-25s"; - private static final int progressBarChars = 30; - - private final NumberFormat secondsFormat = new DecimalFormat("#0.00"); - - protected SparkJobMonitor(HiveConf hiveConf) { - monitorTimeoutInterval = hiveConf.getTimeVar( - HiveConf.ConfVars.SPARK_JOB_MONITOR_TIMEOUT, TimeUnit.SECONDS); - inPlaceUpdate = InPlaceUpdate.canRenderInPlace(hiveConf) && !SessionState.getConsole().getIsSilent(); - console = new SessionState.LogHelper(LOG); - out = SessionState.LogHelper.getInfoStream(); - } - - public abstract int startMonitor(); - - private void printStatusInPlace(Map progressMap) { - - StringBuilder reportBuffer = new StringBuilder(); - - // Num of total and completed tasks - int sumTotal = 0; - int sumComplete = 0; - - // position the cursor to line 0 - repositionCursor(); - - // header - reprintLine(SEPARATOR); - reprintLineWithColorAsBold(HEADER, Ansi.Color.CYAN); - reprintLine(SEPARATOR); - - SortedSet keys = new TreeSet(progressMap.keySet()); - int idx = 0; - final int numKey = keys.size(); - for (String s : keys) { - SparkStageProgress progress = progressMap.get(s); - final int complete = progress.getSucceededTaskCount(); - final int total = progress.getTotalTaskCount(); - final int running = progress.getRunningTaskCount(); - final int failed = progress.getFailedTaskCount(); - sumTotal += total; - sumComplete += complete; - - StageState state = total > 0 ? StageState.PENDING : StageState.FINISHED; - if (complete > 0 || running > 0 || failed > 0) { - if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s); - } - if (complete < total) { - state = StageState.RUNNING; - } else { - state = StageState.FINISHED; - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s); - completed.add(s); - } - } - - int div = s.indexOf('_'); - String attempt = div > 0 ? s.substring(div + 1) : "-"; - String stageName = "Stage-" + (div > 0 ? s.substring(0, div) : s); - String nameWithProgress = getNameWithProgress(stageName, complete, total); - - final int pending = total - complete - running; - String stageStr = String.format(STAGE_FORMAT, - nameWithProgress, attempt, state, total, complete, running, pending, failed); - reportBuffer.append(stageStr); - if (idx++ != numKey - 1) { - reportBuffer.append("\n"); - } - } - reprintMultiLine(reportBuffer.toString()); - reprintLine(SEPARATOR); - final float progress = (sumTotal == 0) ? 1.0f : (float) sumComplete / (float) sumTotal; - String footer = getFooter(numKey, completed.size(), progress, startTime); - reprintLineWithColorAsBold(footer, Ansi.Color.RED); - reprintLine(SEPARATOR); - } - - protected void printStatus(Map progressMap, - Map lastProgressMap) { - - // do not print duplicate status while still in middle of print interval. - boolean isDuplicateState = isSameAsPreviousProgress(progressMap, lastProgressMap); - boolean withinInterval = System.currentTimeMillis() <= lastPrintTime + printInterval; - if (isDuplicateState && withinInterval) { - return; - } - - String report = getReport(progressMap); - if (inPlaceUpdate) { - printStatusInPlace(progressMap); - console.logInfo(report); - } else { - console.printInfo(report); - } - - lastPrintTime = System.currentTimeMillis(); - } - - protected int getTotalTaskCount(Map progressMap) { - int totalTasks = 0; - for (SparkStageProgress progress: progressMap.values() ) { - totalTasks += progress.getTotalTaskCount(); - } - - return totalTasks; - } - - protected int getStageMaxTaskCount(Map progressMap) { - int stageMaxTasks = 0; - for (SparkStageProgress progress: progressMap.values() ) { - int tasks = progress.getTotalTaskCount(); - if (tasks > stageMaxTasks) { - stageMaxTasks = tasks; - } - } - - return stageMaxTasks; - } - - private String getReport(Map progressMap) { - StringBuilder reportBuffer = new StringBuilder(); - SimpleDateFormat dt = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); - String currentDate = dt.format(new Date()); - reportBuffer.append(currentDate + "\t"); - - // Num of total and completed tasks - int sumTotal = 0; - int sumComplete = 0; - - SortedSet keys = new TreeSet(progressMap.keySet()); - for (String s : keys) { - SparkStageProgress progress = progressMap.get(s); - final int complete = progress.getSucceededTaskCount(); - final int total = progress.getTotalTaskCount(); - final int running = progress.getRunningTaskCount(); - final int failed = progress.getFailedTaskCount(); - sumTotal += total; - sumComplete += complete; - - String stageName = "Stage-" + s; - if (total <= 0) { - reportBuffer.append(String.format("%s: -/-\t", stageName)); - } else { - if (complete == total && !completed.contains(s)) { - completed.add(s); - - if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s); - } - perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s); - } - if (complete < total && (complete > 0 || running > 0 || failed > 0)) { - /* stage is started, but not complete */ - if (!perfLogger.startTimeHasMethod(PerfLogger.SPARK_RUN_STAGE + s)) { - perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_RUN_STAGE + s); - } - if (failed > 0) { - reportBuffer.append( - String.format( - "%s: %d(+%d,-%d)/%d\t", stageName, complete, running, failed, total)); - } else { - reportBuffer.append( - String.format("%s: %d(+%d)/%d\t", stageName, complete, running, total)); - } - } else { - /* stage is waiting for input/slots or complete */ - if (failed > 0) { - /* tasks finished but some failed */ - reportBuffer.append( - String.format( - "%s: %d(-%d)/%d Finished with failed tasks\t", - stageName, complete, failed, total)); - } else { - if (complete == total) { - reportBuffer.append( - String.format("%s: %d/%d Finished\t", stageName, complete, total)); - } else { - reportBuffer.append(String.format("%s: %d/%d\t", stageName, complete, total)); - } - } - } - } - } - - if (SessionState.get() != null) { - final float progress = (sumTotal == 0) ? 1.0f : (float) sumComplete / (float) sumTotal; - SessionState.get().updateProgressedPercentage(progress); - } - return reportBuffer.toString(); - } - - private boolean isSameAsPreviousProgress( - Map progressMap, - Map lastProgressMap) { - - if (lastProgressMap == null) { - return false; - } - - if (progressMap.isEmpty()) { - return lastProgressMap.isEmpty(); - } else { - if (lastProgressMap.isEmpty()) { - return false; - } else { - if (progressMap.size() != lastProgressMap.size()) { - return false; - } - for (String key : progressMap.keySet()) { - if (!lastProgressMap.containsKey(key) - || !progressMap.get(key).equals(lastProgressMap.get(key))) { - return false; - } - } - } - } - return true; - } - - private void repositionCursor() { - if (lines > 0) { - out.print(ansi().cursorUp(lines).toString()); - out.flush(); - lines = 0; - } - } - - private void reprintLine(String line) { - InPlaceUpdate.reprintLine(out, line); - lines++; - } - - private void reprintLineWithColorAsBold(String line, Ansi.Color color) { - out.print(ansi().eraseLine(Ansi.Erase.ALL).fg(color).bold().a(line).a('\n').boldOff().reset() - .toString()); - out.flush(); - lines++; - } - - private String getNameWithProgress(String s, int complete, int total) { - String result = ""; - if (s != null) { - float percent = total == 0 ? 1.0f : (float) complete / (float) total; - // lets use the remaining space in column 1 as progress bar - int spaceRemaining = COLUMN_1_WIDTH - s.length() - 1; - String trimmedVName = s; - - // if the vertex name is longer than column 1 width, trim it down - if (s.length() > COLUMN_1_WIDTH) { - trimmedVName = s.substring(0, COLUMN_1_WIDTH - 2); - result = trimmedVName + ".."; - } else { - result = trimmedVName + " "; - } - - int toFill = (int) (spaceRemaining * percent); - for (int i = 0; i < toFill; i++) { - result += "."; - } - } - return result; - } - - // STAGES: 03/04 [==================>>-----] 86% ELAPSED TIME: 1.71 s - private String getFooter(int keySize, int completedSize, float progress, long startTime) { - String verticesSummary = String.format("STAGES: %02d/%02d", completedSize, keySize); - String progressBar = getInPlaceProgressBar(progress); - final int progressPercent = (int) (progress * 100); - String progressStr = "" + progressPercent + "%"; - float et = (float) (System.currentTimeMillis() - startTime) / (float) 1000; - String elapsedTime = "ELAPSED TIME: " + secondsFormat.format(et) + " s"; - String footer = String.format(FOOTER_FORMAT, - verticesSummary, progressBar, progressStr, elapsedTime); - return footer; - } - - // [==================>>-----] - private String getInPlaceProgressBar(float percent) { - StringBuilder bar = new StringBuilder("["); - int remainingChars = progressBarChars - 4; - int completed = (int) (remainingChars * percent); - int pending = remainingChars - completed; - for (int i = 0; i < completed; i++) { - bar.append("="); - } - bar.append(">>"); - for (int i = 0; i < pending; i++) { - bar.append("-"); - } - bar.append("]"); - return bar.toString(); - } - - private void reprintMultiLine(String line) { - int numLines = line.split("\r\n|\r|\n").length; - out.print(ansi().eraseLine(Ansi.Erase.ALL).a(line).a('\n').toString()); - out.flush(); - lines += numLines; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java deleted file mode 100644 index e112882db43..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobRef.java +++ /dev/null @@ -1,29 +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.hadoop.hive.ql.exec.spark.status; - -public interface SparkJobRef { - - public String getJobId(); - - public SparkJobStatus getSparkJobStatus(); - - public boolean cancelJob(); - - public int monitorJob(); -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobState.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobState.java deleted file mode 100644 index 2f9963e242f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobState.java +++ /dev/null @@ -1,29 +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.hadoop.hive.ql.exec.spark.status; - -public enum SparkJobState { - - SUBMITTED, - INITING, - RUNNING, - SUCCEEDED, - KILLED, - FAILED, - ERROR, -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java deleted file mode 100644 index 8474afcc2ac..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkJobStatus.java +++ /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.hadoop.hive.ql.exec.spark.status; - -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.spark.JobExecutionStatus; - -import java.util.Map; - -/** - * SparkJobStatus identify what Hive want to know about the status of a Spark job. - */ -public interface SparkJobStatus { - - String getAppID(); - - int getJobId(); - - JobExecutionStatus getState() throws HiveException; - - int[] getStageIds() throws HiveException; - - Map getSparkStageProgress() throws HiveException; - - SparkCounters getCounter(); - - SparkStatistics getSparkStatistics(); - - String getWebUIURL(); - - void cleanup(); - - Throwable getError(); - - void setError(Throwable e); -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java deleted file mode 100644 index e8d63d6cc49..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/SparkStageProgress.java +++ /dev/null @@ -1,80 +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.hadoop.hive.ql.exec.spark.status; - -public class SparkStageProgress { - - private int totalTaskCount; - private int succeededTaskCount; - private int runningTaskCount; - private int failedTaskCount; - - public SparkStageProgress( - int totalTaskCount, - int succeededTaskCount, - int runningTaskCount, - int failedTaskCount) { - - this.totalTaskCount = totalTaskCount; - this.succeededTaskCount = succeededTaskCount; - this.runningTaskCount = runningTaskCount; - this.failedTaskCount = failedTaskCount; - } - - public int getTotalTaskCount() { - return totalTaskCount; - } - - public int getSucceededTaskCount() { - return succeededTaskCount; - } - - public int getRunningTaskCount() { - return runningTaskCount; - } - - public int getFailedTaskCount() { - return failedTaskCount; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof SparkStageProgress) { - SparkStageProgress other = (SparkStageProgress) obj; - return getTotalTaskCount() == other.getTotalTaskCount() - && getSucceededTaskCount() == other.getSucceededTaskCount() - && getRunningTaskCount() == other.getRunningTaskCount() - && getFailedTaskCount() == other.getFailedTaskCount(); - } - return false; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("TotalTasks: "); - sb.append(getTotalTaskCount()); - sb.append(" Succeeded: "); - sb.append(getSucceededTaskCount()); - sb.append(" Running: "); - sb.append(getRunningTaskCount()); - sb.append(" Failed: "); - sb.append(getFailedTaskCount()); - return sb.toString(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java deleted file mode 100644 index 773fe9700e6..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/JobMetricsListener.java +++ /dev/null @@ -1,90 +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.hadoop.hive.ql.exec.spark.status.impl; - -import java.util.AbstractMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.scheduler.SparkListener; -import org.apache.spark.scheduler.SparkListenerJobStart; -import org.apache.spark.scheduler.SparkListenerTaskEnd; -import org.apache.spark.scheduler.TaskInfo; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -public class JobMetricsListener extends SparkListener { - - private static final Logger LOG = LoggerFactory.getLogger(JobMetricsListener.class); - - private final Map jobIdToStageId = Maps.newHashMap(); - private final Map stageIdToJobId = Maps.newHashMap(); - private final Map>>> allJobMetrics = - Maps.newHashMap(); - - @Override - public synchronized void onTaskEnd(SparkListenerTaskEnd taskEnd) { - int stageId = taskEnd.stageId(); - Integer jobId = stageIdToJobId.get(stageId); - if (jobId == null) { - LOG.warn("Can not find job id for stage[" + stageId + "]."); - } else { - Map>> jobMetrics = allJobMetrics.computeIfAbsent( - jobId, k -> Maps.newHashMap()); - List> stageMetrics = jobMetrics.computeIfAbsent(stageId, - k -> Lists.newLinkedList()); - - stageMetrics.add(new AbstractMap.SimpleEntry<>(taskEnd.taskMetrics(), taskEnd.taskInfo())); - } - } - - @Override - public synchronized void onJobStart(SparkListenerJobStart jobStart) { - int jobId = jobStart.jobId(); - int size = jobStart.stageIds().size(); - int[] intStageIds = new int[size]; - for (int i = 0; i < size; i++) { - Integer stageId = (Integer) jobStart.stageIds().apply(i); - intStageIds[i] = stageId; - stageIdToJobId.put(stageId, jobId); - } - jobIdToStageId.put(jobId, intStageIds); - } - - public synchronized Map>> getJobMetric(int jobId) { - return allJobMetrics.get(jobId); - } - - public synchronized void cleanup(int jobId) { - allJobMetrics.remove(jobId); - jobIdToStageId.remove(jobId); - Iterator> iterator = stageIdToJobId.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - if (entry.getValue() == jobId) { - iterator.remove(); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.java deleted file mode 100644 index 18b66c6594d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobRef.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.hadoop.hive.ql.exec.spark.status.impl; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.spark.status.LocalSparkJobMonitor; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; -import org.apache.spark.api.java.JavaSparkContext; - -public class LocalSparkJobRef implements SparkJobRef { - - private final String jobId; - private final HiveConf hiveConf; - private final LocalSparkJobStatus sparkJobStatus; - private final JavaSparkContext javaSparkContext; - - public LocalSparkJobRef( - String jobId, - HiveConf hiveConf, - LocalSparkJobStatus sparkJobStatus, - JavaSparkContext javaSparkContext) { - - this.jobId = jobId; - this.hiveConf = hiveConf; - this.sparkJobStatus = sparkJobStatus; - this.javaSparkContext = javaSparkContext; - } - - @Override - public String getJobId() { - return jobId; - } - - @Override - public SparkJobStatus getSparkJobStatus() { - return sparkJobStatus; - } - - @Override - public boolean cancelJob() { - int id = Integer.parseInt(jobId); - javaSparkContext.sc().cancelJob(id); - return true; - } - - @Override - public int monitorJob() { - LocalSparkJobMonitor localSparkJobMonitor = new LocalSparkJobMonitor(hiveConf, sparkJobStatus); - return localSparkJobMonitor.startMonitor(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java deleted file mode 100644 index 03f8a0b680c..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/LocalSparkJobStatus.java +++ /dev/null @@ -1,210 +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.hadoop.hive.ql.exec.spark.status.impl; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsBuilder; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkStageProgress; -import org.apache.hive.spark.client.MetricsCollection; -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.counter.SparkCounters; - -import org.apache.spark.JobExecutionStatus; -import org.apache.spark.SparkJobInfo; -import org.apache.spark.SparkStageInfo; -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.scheduler.TaskInfo; - -public class LocalSparkJobStatus implements SparkJobStatus { - - private final JavaSparkContext sparkContext; - private static final Logger LOG = LoggerFactory.getLogger(LocalSparkJobStatus.class.getName()); - private int jobId; - // After SPARK-2321, we only use JobMetricsListener to get job metrics - // TODO: remove it when the new API provides equivalent functionality - private JobMetricsListener jobMetricsListener; - private SparkCounters sparkCounters; - private JavaFutureAction future; - private Set cachedRDDIds; - private Throwable error; - - public LocalSparkJobStatus(JavaSparkContext sparkContext, int jobId, - JobMetricsListener jobMetricsListener, SparkCounters sparkCounters, - Set cachedRDDIds, JavaFutureAction future) { - this.sparkContext = sparkContext; - this.jobId = jobId; - this.jobMetricsListener = jobMetricsListener; - this.sparkCounters = sparkCounters; - this.cachedRDDIds = cachedRDDIds; - this.future = future; - this.error = null; - } - - @Override - public String getAppID() { - return sparkContext.sc().applicationId(); - } - - @Override - public int getJobId() { - return jobId; - } - - @Override - public JobExecutionStatus getState() { - SparkJobInfo sparkJobInfo = getJobInfo(); - // For spark job with empty source data, it's not submitted actually, so we would never - // receive JobStart/JobEnd event in JobStateListener, use JavaFutureAction to get current - // job state. - if (sparkJobInfo == null && future.isDone()) { - try { - future.get(); - } catch (Exception e) { - LOG.error("Failed to run job " + jobId, e); - return JobExecutionStatus.FAILED; - } - return JobExecutionStatus.SUCCEEDED; - } - return sparkJobInfo == null ? null : sparkJobInfo.status(); - } - - @Override - public int[] getStageIds() { - SparkJobInfo sparkJobInfo = getJobInfo(); - return sparkJobInfo == null ? new int[0] : sparkJobInfo.stageIds(); - } - - @Override - public Map getSparkStageProgress() { - Map stageProgresses = new HashMap(); - for (int stageId : getStageIds()) { - SparkStageInfo sparkStageInfo = getStageInfo(stageId); - if (sparkStageInfo != null) { - int runningTaskCount = sparkStageInfo.numActiveTasks(); - int completedTaskCount = sparkStageInfo.numCompletedTasks(); - int failedTaskCount = sparkStageInfo.numFailedTasks(); - int totalTaskCount = sparkStageInfo.numTasks(); - SparkStageProgress sparkStageProgress = new SparkStageProgress( - totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); - stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" - + sparkStageInfo.currentAttemptId(), sparkStageProgress); - } - } - return stageProgresses; - } - - @Override - public SparkCounters getCounter() { - return sparkCounters; - } - - @Override - public SparkStatistics getSparkStatistics() { - SparkStatisticsBuilder sparkStatisticsBuilder = new SparkStatisticsBuilder(); - // add Hive operator level statistics. - sparkStatisticsBuilder.add(sparkCounters); - // add spark job metrics. - Map>> jobMetric = jobMetricsListener.getJobMetric(jobId); - if (jobMetric == null) { - return null; - } - - MetricsCollection metricsCollection = new MetricsCollection(); - Set stageIds = jobMetric.keySet(); - for (int stageId : stageIds) { - List> taskMetrics = jobMetric.get(stageId); - for (Map.Entry taskMetric : taskMetrics) { - Metrics metrics = new Metrics(taskMetric.getKey(), taskMetric.getValue()); - metricsCollection.addMetrics(jobId, stageId, 0, metrics); - } - } - Map flatJobMetric = SparkMetricsUtils.collectMetrics(metricsCollection - .getAllMetrics()); - for (Map.Entry entry : flatJobMetric.entrySet()) { - sparkStatisticsBuilder.add(SparkStatisticsNames.SPARK_GROUP_NAME, entry.getKey(), - Long.toString(entry.getValue())); - } - - return sparkStatisticsBuilder.build(); - } - - @Override - public String getWebUIURL() { - try { - if (sparkContext.sc().uiWebUrl().isDefined()) { - return SparkUtilities.reverseDNSLookupURL(sparkContext.sc().uiWebUrl().get()); - } else { - return "UNDEFINED"; - } - } catch (Exception e) { - LOG.warn("Failed to get web UI URL.", e); - } - return "UNKNOWN"; - } - - @Override - public void cleanup() { - jobMetricsListener.cleanup(jobId); - if (cachedRDDIds != null) { - for (Integer cachedRDDId: cachedRDDIds) { - sparkContext.sc().unpersistRDD(cachedRDDId, false); - } - } - } - - @Override - public Throwable getError() { - if (error != null) { - return error; - } - if (future.isDone()) { - try { - future.get(); - } catch (Throwable e) { - return e; - } - } - return null; - } - - @Override - public void setError(Throwable e) { - this.error = e; - } - - private SparkJobInfo getJobInfo() { - return sparkContext.statusTracker().getJobInfo(jobId); - } - - private SparkStageInfo getStageInfo(int stageId) { - return sparkContext.statusTracker().getStageInfo(stageId); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java deleted file mode 100644 index e130eb6d310..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobRef.java +++ /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.hadoop.hive.ql.exec.spark.status.impl; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.spark.status.RemoteSparkJobMonitor; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobRef; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; -import org.apache.hive.spark.client.JobHandle; - -import java.io.Serializable; - -public class RemoteSparkJobRef implements SparkJobRef { - - private final String jobId; - private final HiveConf hiveConf; - private final RemoteSparkJobStatus sparkJobStatus; - private final JobHandle jobHandler; - - public RemoteSparkJobRef(HiveConf hiveConf, JobHandle jobHandler, RemoteSparkJobStatus sparkJobStatus) { - this.jobHandler = jobHandler; - this.jobId = jobHandler.getClientJobId(); - this.hiveConf = hiveConf; - this.sparkJobStatus = sparkJobStatus; - } - - @Override - public String getJobId() { - return jobId; - } - - @Override - public SparkJobStatus getSparkJobStatus() { - return sparkJobStatus; - } - - @Override - public boolean cancelJob() { - return jobHandler.cancel(true); - } - - @Override - public int monitorJob() { - RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, sparkJobStatus); - return remoteSparkJobMonitor.startMonitor(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java deleted file mode 100644 index ff969e048f6..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/RemoteSparkJobStatus.java +++ /dev/null @@ -1,326 +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.hadoop.hive.ql.exec.spark.status.impl; - -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatistics; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsBuilder; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkJobStatus; -import org.apache.hadoop.hive.ql.exec.spark.status.SparkStageProgress; -import org.apache.hive.spark.client.MetricsCollection; -import org.apache.hive.spark.client.Job; -import org.apache.hive.spark.client.JobContext; -import org.apache.hive.spark.client.JobHandle; -import org.apache.hive.spark.client.SparkClient; -import org.apache.hive.spark.counter.SparkCounters; - -import org.apache.spark.JobExecutionStatus; -import org.apache.spark.SparkJobInfo; -import org.apache.spark.SparkStageInfo; -import org.apache.spark.api.java.JavaFutureAction; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -/** - * Used with remove spark client. - */ -public class RemoteSparkJobStatus implements SparkJobStatus { - private static final Logger LOG = LoggerFactory.getLogger(RemoteSparkJobStatus.class.getName()); - private final SparkClient sparkClient; - private final JobHandle jobHandle; - private Throwable error; - private final transient long sparkClientTimeoutInSeconds; - - public RemoteSparkJobStatus(SparkClient sparkClient, JobHandle jobHandle, long timeoutInSeconds) { - this.sparkClient = sparkClient; - this.jobHandle = jobHandle; - this.error = null; - this.sparkClientTimeoutInSeconds = timeoutInSeconds; - } - - @Override - public String getAppID() { - Future getAppID = sparkClient.run(new GetAppIDJob()); - try { - return getAppID.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); - } catch (Exception e) { - LOG.warn("Failed to get APP ID.", e); - if (Thread.interrupted()) { - error = e; - } - return null; - } - } - - @Override - public int getJobId() { - return jobHandle.getSparkJobIds().size() == 1 ? jobHandle.getSparkJobIds().get(0) : -1; - } - - @Override - public JobExecutionStatus getState() throws HiveException { - SparkJobInfo sparkJobInfo = getSparkJobInfo(); - return sparkJobInfo != null ? sparkJobInfo.status() : null; - } - - @Override - public int[] getStageIds() throws HiveException { - SparkJobInfo sparkJobInfo = getSparkJobInfo(); - return sparkJobInfo != null ? sparkJobInfo.stageIds() : new int[0]; - } - - @Override - public Map getSparkStageProgress() throws HiveException { - Map stageProgresses = new HashMap(); - for (int stageId : getStageIds()) { - SparkStageInfo sparkStageInfo = getSparkStageInfo(stageId); - if (sparkStageInfo != null && sparkStageInfo.name() != null) { - int runningTaskCount = sparkStageInfo.numActiveTasks(); - int completedTaskCount = sparkStageInfo.numCompletedTasks(); - int failedTaskCount = sparkStageInfo.numFailedTasks(); - int totalTaskCount = sparkStageInfo.numTasks(); - SparkStageProgress sparkStageProgress = new SparkStageProgress( - totalTaskCount, completedTaskCount, runningTaskCount, failedTaskCount); - stageProgresses.put(String.valueOf(sparkStageInfo.stageId()) + "_" - + sparkStageInfo.currentAttemptId(), sparkStageProgress); - } - } - return stageProgresses; - } - - @Override - public SparkCounters getCounter() { - return jobHandle.getSparkCounters(); - } - - @Override - public SparkStatistics getSparkStatistics() { - MetricsCollection metricsCollection = jobHandle.getMetrics(); - if (metricsCollection == null || getCounter() == null) { - return null; - } - - SparkStatisticsBuilder sparkStatisticsBuilder = new SparkStatisticsBuilder(); - - // add Hive operator level statistics. - e.g. RECORDS_IN, RECORDS_OUT - sparkStatisticsBuilder.add(getCounter()); - - // add spark job metrics. - e.g. metrics collected by Spark itself (JvmGCTime, - // ExecutorRunTime, etc.) - Map flatJobMetric = SparkMetricsUtils.collectMetrics( - metricsCollection.getAllMetrics()); - for (Map.Entry entry : flatJobMetric.entrySet()) { - sparkStatisticsBuilder.add(SparkStatisticsNames.SPARK_GROUP_NAME, entry.getKey(), - Long.toString(entry.getValue())); - } - - return sparkStatisticsBuilder.build(); - } - - @Override - public String getWebUIURL() { - Future getWebUIURL = sparkClient.run(new GetWebUIURLJob()); - try { - return getWebUIURL.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); - } catch (Exception e) { - LOG.warn("Failed to get web UI URL.", e); - if (Thread.interrupted()) { - error = e; - } - return "UNKNOWN"; - } - } - - @Override - public void cleanup() { - - } - - @Override - public Throwable getError() { - if (error != null) { - return error; - } - return jobHandle.getError(); - } - - @Override - public void setError(Throwable e) { - this.error = e; - } - - /** - * Indicates whether the remote context is active. SparkJobMonitor can use this to decide whether - * to stop monitoring. - */ - public boolean isRemoteActive() { - return sparkClient.isActive(); - } - - private SparkJobInfo getSparkJobInfo() throws HiveException { - Integer sparkJobId = jobHandle.getSparkJobIds().size() == 1 - ? jobHandle.getSparkJobIds().get(0) : null; - if (sparkJobId == null) { - return null; - } - Future getJobInfo = sparkClient.run( - new GetJobInfoJob(jobHandle.getClientJobId(), sparkJobId)); - try { - return getJobInfo.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); - } catch (Exception e) { - LOG.warn("Failed to get job info.", e); - throw new HiveException(e, ErrorMsg.SPARK_GET_JOB_INFO_TIMEOUT, - Long.toString(sparkClientTimeoutInSeconds)); - } - } - - private SparkStageInfo getSparkStageInfo(int stageId) { - Future getStageInfo = sparkClient.run(new GetStageInfoJob(stageId)); - try { - return getStageInfo.get(sparkClientTimeoutInSeconds, TimeUnit.SECONDS); - } catch (Throwable t) { - LOG.warn("Error getting stage info", t); - return null; - } - } - - public JobHandle.State getRemoteJobState() { - if (error != null) { - return JobHandle.State.FAILED; - } - return jobHandle.getState(); - } - - private static class GetJobInfoJob implements Job { - private final String clientJobId; - private final int sparkJobId; - - private GetJobInfoJob() { - // For serialization. - this(null, -1); - } - - GetJobInfoJob(String clientJobId, int sparkJobId) { - this.clientJobId = clientJobId; - this.sparkJobId = sparkJobId; - } - - @Override - public SparkJobInfo call(JobContext jc) throws Exception { - SparkJobInfo jobInfo = jc.sc().statusTracker().getJobInfo(sparkJobId); - if (jobInfo == null) { - List> list = jc.getMonitoredJobs().get(clientJobId); - if (list != null && list.size() == 1) { - JavaFutureAction futureAction = list.get(0); - if (futureAction.isDone()) { - boolean futureSucceed = true; - try { - futureAction.get(); - } catch (Exception e) { - LOG.error("Failed to run job " + sparkJobId, e); - futureSucceed = false; - } - jobInfo = getDefaultJobInfo(sparkJobId, - futureSucceed ? JobExecutionStatus.SUCCEEDED : JobExecutionStatus.FAILED); - } - } - } - if (jobInfo == null) { - jobInfo = getDefaultJobInfo(sparkJobId, JobExecutionStatus.UNKNOWN); - } - return jobInfo; - } - } - - private static class GetStageInfoJob implements Job { - private final int stageId; - - private GetStageInfoJob() { - // For serialization. - this(-1); - } - - GetStageInfoJob(int stageId) { - this.stageId = stageId; - } - - @Override - public SparkStageInfo call(JobContext jc) throws Exception { - return jc.sc().statusTracker().getStageInfo(stageId); - } - } - - private static SparkJobInfo getDefaultJobInfo(final Integer jobId, - final JobExecutionStatus status) { - return new SparkJobInfo() { - - @Override - public int jobId() { - return jobId == null ? -1 : jobId; - } - - @Override - public int[] stageIds() { - return new int[0]; - } - - @Override - public JobExecutionStatus status() { - return status; - } - }; - } - - private static class GetAppIDJob implements Job { - - public GetAppIDJob() { - } - - @Override - public String call(JobContext jc) throws Exception { - return jc.sc().sc().applicationId(); - } - } - - private static class GetWebUIURLJob implements Job { - - public GetWebUIURLJob() { - } - - @Override - public String call(JobContext jc) throws Exception { - if (jc.sc().sc().uiWebUrl().isDefined()) { - return SparkUtilities.reverseDNSLookupURL(jc.sc().sc().uiWebUrl().get()); - } - return "UNDEFINED"; - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SparkMetricsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SparkMetricsUtils.java deleted file mode 100644 index fab5422f1fc..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/status/impl/SparkMetricsUtils.java +++ /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.hadoop.hive.ql.exec.spark.status.impl; - -import java.util.LinkedHashMap; -import java.util.Map; - -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsNames; -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; - -final class SparkMetricsUtils { - - private SparkMetricsUtils(){} - - static Map collectMetrics(Metrics allMetrics) { - Map results = new LinkedHashMap(); - results.put(SparkStatisticsNames.EXECUTOR_DESERIALIZE_TIME, allMetrics.executorDeserializeTime); - results.put(SparkStatisticsNames.EXECUTOR_DESERIALIZE_CPU_TIME, - allMetrics.executorDeserializeCpuTime); - results.put(SparkStatisticsNames.EXECUTOR_RUN_TIME, allMetrics.executorRunTime); - results.put(SparkStatisticsNames.EXECUTOR_CPU_TIME, allMetrics.executorCpuTime); - results.put(SparkStatisticsNames.RESULT_SIZE, allMetrics.resultSize); - results.put(SparkStatisticsNames.JVM_GC_TIME, allMetrics.jvmGCTime); - results.put(SparkStatisticsNames.RESULT_SERIALIZATION_TIME, allMetrics.resultSerializationTime); - results.put(SparkStatisticsNames.MEMORY_BYTES_SPILLED, allMetrics.memoryBytesSpilled); - results.put(SparkStatisticsNames.DISK_BYTES_SPILLED, allMetrics.diskBytesSpilled); - results.put(SparkStatisticsNames.TASK_DURATION_TIME, allMetrics.taskDurationTime); - if (allMetrics.inputMetrics != null) { - results.put(SparkStatisticsNames.BYTES_READ, allMetrics.inputMetrics.bytesRead); - } - if (allMetrics.shuffleReadMetrics != null) { - ShuffleReadMetrics shuffleReadMetrics = allMetrics.shuffleReadMetrics; - long rbf = shuffleReadMetrics.remoteBlocksFetched; - long lbf = shuffleReadMetrics.localBlocksFetched; - results.put(SparkStatisticsNames.REMOTE_BLOCKS_FETCHED, rbf); - results.put(SparkStatisticsNames.LOCAL_BLOCKS_FETCHED, lbf); - results.put(SparkStatisticsNames.TOTAL_BLOCKS_FETCHED, rbf + lbf); - results.put(SparkStatisticsNames.FETCH_WAIT_TIME, shuffleReadMetrics.fetchWaitTime); - results.put(SparkStatisticsNames.REMOTE_BYTES_READ, shuffleReadMetrics.remoteBytesRead); - } - if (allMetrics.shuffleWriteMetrics != null) { - results.put(SparkStatisticsNames.SHUFFLE_BYTES_WRITTEN, allMetrics.shuffleWriteMetrics.shuffleBytesWritten); - results.put(SparkStatisticsNames.SHUFFLE_WRITE_TIME, allMetrics.shuffleWriteMetrics.shuffleWriteTime); - } - return results; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java deleted file mode 100644 index 70bdab98ac7..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java +++ /dev/null @@ -1,123 +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.hadoop.hive.ql.exec.vector; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.VectorDesc; -import org.apache.hadoop.hive.ql.plan.VectorSparkHashTableSinkDesc; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; - -import com.google.common.annotations.VisibleForTesting; - -/** - * Vectorized version of SparkHashTableSinkOperator - * Currently the implementation just delegates all the work to super class - * - * Copied from VectorFileSinkOperator - */ -public class VectorSparkHashTableSinkOperator extends SparkHashTableSinkOperator - implements VectorizationOperator { - - private static final long serialVersionUID = 1L; - - private VectorizationContext vContext; - private VectorSparkHashTableSinkDesc vectorDesc; - - // The above members are initialized by the constructor and must not be - // transient. - //--------------------------------------------------------------------------- - - private transient boolean firstBatch; - - private transient VectorExtractRow vectorExtractRow; - - protected transient Object[] singleRow; - - /** Kryo ctor. */ - @VisibleForTesting - public VectorSparkHashTableSinkOperator() { - super(); - } - - public VectorSparkHashTableSinkOperator(CompilationOpContext ctx) { - super(ctx); - } - - public VectorSparkHashTableSinkOperator( - CompilationOpContext ctx, OperatorDesc conf, - VectorizationContext vContext, VectorDesc vectorDesc) { - this(ctx); - this.conf = (SparkHashTableSinkDesc) conf; - this.vContext = vContext; - this.vectorDesc = (VectorSparkHashTableSinkDesc) vectorDesc; - } - - @Override - public VectorizationContext getInputVectorizationContext() { - return vContext; - } - - @Override - protected void initializeOp(Configuration hconf) throws HiveException { - inputObjInspectors[0] = - VectorizedBatchUtil.convertToStandardStructObjectInspector((StructObjectInspector) inputObjInspectors[0]); - - super.initializeOp(hconf); - - firstBatch = true; - } - - @Override - public void process(Object row, int tag) throws HiveException { - VectorizedRowBatch batch = (VectorizedRowBatch) row; - - if (firstBatch) { - vectorExtractRow = new VectorExtractRow(); - vectorExtractRow.init((StructObjectInspector) inputObjInspectors[0], vContext.getProjectedColumns()); - - singleRow = new Object[vectorExtractRow.getCount()]; - - firstBatch = false; - } - - if (batch.selectedInUse) { - int selected[] = batch.selected; - for (int logical = 0 ; logical < batch.size; logical++) { - int batchIndex = selected[logical]; - vectorExtractRow.extractRow(batch, batchIndex, singleRow); - super.process(singleRow, tag); - } - } else { - for (int batchIndex = 0 ; batchIndex < batch.size; batchIndex++) { - vectorExtractRow.extractRow(batch, batchIndex, singleRow); - super.process(singleRow, tag); - } - } - } - - @Override - public VectorDesc getVectorDesc() { - return vectorDesc; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java deleted file mode 100644 index edd33d7f322..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java +++ /dev/null @@ -1,115 +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.hadoop.hive.ql.exec.vector; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.VectorDesc; -import org.apache.hadoop.hive.ql.plan.VectorSparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.Writable; - -import com.google.common.annotations.VisibleForTesting; - -/** - * Vectorized version for SparkPartitionPruningSinkOperator. - * Forked from VectorAppMasterEventOperator. - **/ -public class VectorSparkPartitionPruningSinkOperator extends SparkPartitionPruningSinkOperator - implements VectorizationOperator { - - private static final long serialVersionUID = 1L; - - private VectorizationContext vContext; - private VectorSparkPartitionPruningSinkDesc vectorDesc; - - protected transient boolean firstBatch; - - protected transient VectorExtractRow vectorExtractRow; - - protected transient Object[] singleRow; - - public VectorSparkPartitionPruningSinkOperator(CompilationOpContext ctx, OperatorDesc conf, - VectorizationContext context, VectorDesc vectorDesc) { - this(ctx); - this.conf = (SparkPartitionPruningSinkDesc) conf; - this.vContext = context; - this.vectorDesc = (VectorSparkPartitionPruningSinkDesc) vectorDesc; - } - - /** Kryo ctor. */ - @VisibleForTesting - public VectorSparkPartitionPruningSinkOperator() { - super(); - } - - public VectorSparkPartitionPruningSinkOperator(CompilationOpContext ctx) { - super(ctx); - } - - @Override - public VectorizationContext getInputVectorizationContext() { - return vContext; - } - - @Override - public void initializeOp(Configuration hconf) throws HiveException { - inputObjInspectors[0] = - VectorizedBatchUtil.convertToStandardStructObjectInspector( - (StructObjectInspector) inputObjInspectors[0]); - super.initializeOp(hconf); - - firstBatch = true; - } - - @Override - public void process(Object data, int tag) throws HiveException { - VectorizedRowBatch batch = (VectorizedRowBatch) data; - if (firstBatch) { - vectorExtractRow = new VectorExtractRow(); - vectorExtractRow.init((StructObjectInspector) inputObjInspectors[0], - vContext.getProjectedColumns()); - singleRow = new Object[vectorExtractRow.getCount()]; - firstBatch = false; - } - - ObjectInspector rowInspector = inputObjInspectors[0]; - try { - Writable writableRow; - for (int logical = 0; logical < batch.size; logical++) { - int batchIndex = batch.selectedInUse ? batch.selected[logical] : logical; - vectorExtractRow.extractRow(batch, batchIndex, singleRow); - writableRow = serializer.serialize(singleRow, rowInspector); - writableRow.write(buffer); - } - } catch (Exception e) { - throw new HiveException(e); - } - } - - @Override - public VectorDesc getVectorDesc() { - return vectorDesc; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java index 327628f8a09..d4b48d5e2dd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/history/HiveHistory.java @@ -68,9 +68,7 @@ public static enum Keys { TASK_COUNTERS, TASK_NUM_MAPPERS, TASK_NUM_REDUCERS, - ROWS_INSERTED, - SPARK_JOB_HANDLE_ID, - SPARK_JOB_ID + ROWS_INSERTED }; /** diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java index 76b36b3f475..ae9eab117eb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java @@ -85,7 +85,7 @@ enum OtherInfoTypes { CLIENT_IP_ADDRESS, HIVE_ADDRESS, HIVE_INSTANCE_TYPE, CONF, PERF, LLAP_APP_ID }; private enum ExecutionMode { - MR, TEZ, LLAP, SPARK, NONE + MR, TEZ, LLAP, NONE }; private enum PrimaryFilterTypes { user, requestuser, operationid, executionmode, tablesread, tableswritten, queue @@ -324,14 +324,11 @@ protected List getTablesFromEntitySet(Set entities) { protected ExecutionMode getExecutionMode(QueryPlan plan) { int numMRJobs = Utilities.getMRTasks(plan.getRootTasks()).size(); - int numSparkJobs = Utilities.getSparkTasks(plan.getRootTasks()).size(); int numTezJobs = Utilities.getTezTasks(plan.getRootTasks()).size(); ExecutionMode mode = ExecutionMode.MR; - if (0 == (numMRJobs + numSparkJobs + numTezJobs)) { + if (0 == (numMRJobs + numTezJobs)) { mode = ExecutionMode.NONE; - } else if (numSparkJobs > 0) { - return ExecutionMode.SPARK; } else if (numTezJobs > 0) { mode = ExecutionMode.TEZ; // Need to go in and check if any of the tasks is running in LLAP mode. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java index 262f57611f6..99113270623 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/HiveProtoLoggingHook.java @@ -174,7 +174,7 @@ public enum OtherInfoType { } public enum ExecutionMode { - MR, TEZ, LLAP, SPARK, NONE + MR, TEZ, LLAP, NONE } static class EventLogger { @@ -461,8 +461,6 @@ private ExecutionMode getExecutionMode(QueryPlan plan, List mrTasks, return ExecutionMode.TEZ; } else if (mrTasks.size() > 0) { return ExecutionMode.MR; - } else if (Utilities.getSparkTasks(plan.getRootTasks()).size() > 0) { - return ExecutionMode.SPARK; } else { return ExecutionMode.NONE; } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index 4bcb4d79d6c..f2ca614ecad 100755 --- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hive.ql.exec.SerializationUtilities; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.metadata.HiveException; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/log/LogDivertAppender.java b/ql/src/java/org/apache/hadoop/hive/ql/log/LogDivertAppender.java index dd25f622c73..ac654e88199 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/log/LogDivertAppender.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/log/LogDivertAppender.java @@ -82,8 +82,8 @@ private static class NameFilter extends AbstractFilter { private static final Pattern executionIncludeNamePattern = Pattern.compile(Joiner.on("|"). join(new String[]{"org.apache.hadoop.mapreduce.JobSubmitter", "org.apache.hadoop.mapreduce.Job", "SessionState", "ReplState", Task.class.getName(), - TezTask.class.getName(), Driver.class.getName(), - "org.apache.hadoop.hive.ql.exec.spark.status.SparkJobMonitor"})); + TezTask.class.getName(), Driver.class.getName() + })); /* Patterns that are included in performance logging level. * In performance mode, show execution and performance logger messages. diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 5ebc282d763..7551c8a658c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -63,7 +63,6 @@ import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.io.RCFileInputFormat; import org.apache.hadoop.hive.ql.io.merge.MergeFileWork; @@ -107,7 +106,6 @@ import org.apache.hadoop.hive.ql.plan.RCFileMergeDesc; import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.StatsWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; @@ -875,13 +873,6 @@ public static void setKeyAndValueDescForTaskTree(Task ta ((MapWork)w).deriveExplainAttributes(); } } - } else if (task instanceof SparkTask) { - SparkWork work = (SparkWork) task.getWork(); - for (BaseWork w : work.getAllWorkUnsorted()) { - if (w instanceof MapWork) { - ((MapWork) w).deriveExplainAttributes(); - } - } } if (task.getChildTasks() == null) { @@ -914,13 +905,6 @@ public static void deriveFinalExplainAttributes( ((MapWork)w).deriveLlap(conf, false); } } - } else if (task instanceof SparkTask) { - SparkWork work = (SparkWork) task.getWork(); - for (BaseWork w : work.getAllWorkUnsorted()) { - if (w instanceof MapWork) { - ((MapWork) w).deriveLlap(conf, false); - } - } } if (task.getChildTasks() == null) { @@ -1521,11 +1505,6 @@ public static void addStatsTask(FileSinkOperator nd, MoveTask mvTask, if (mrWork.getReduceWork() != null) { mrWork.getReduceWork().setGatheringStats(true); } - } else if (currTask.getWork() instanceof SparkWork) { - SparkWork work = (SparkWork) currTask.getWork(); - for (BaseWork w: work.getAllWork()) { - w.setGatheringStats(true); - } } else { // must be TezWork TezWork work = (TezWork) currTask.getWork(); for (BaseWork w: work.getAllWork()) { @@ -1918,9 +1897,6 @@ public static boolean isMergeRequired(List> mvTasks, HiveConf hco if (currTask.getWork() instanceof TezWork) { // tez blurs the boundary between map and reduce, thus it has it's own config return hconf.getBoolVar(ConfVars.HIVEMERGETEZFILES); - } else if (currTask.getWork() instanceof SparkWork) { - // spark has its own config for merging - return hconf.getBoolVar(ConfVars.HIVEMERGESPARKFILES); } return isMergeRequiredForMr(hconf, fsOp, currTask); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java index cf54aa37092..c098fcc38c0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/QueryPlanPostProcessor.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.parse.GenTezProcContext; import org.apache.hadoop.hive.ql.parse.GenTezWork; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkWork; import org.apache.hadoop.hive.ql.plan.ArchiveWork; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.BasicStatsNoJobWork; @@ -44,7 +43,6 @@ import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.StatsWork; import org.apache.hadoop.hive.ql.plan.TezWork; import org.slf4j.Logger; @@ -84,11 +82,6 @@ else if(work instanceof MapredWork) { collectFileSinkDescs(w.getReduceWork().getAllLeafOperators(), acidSinks); } } - else if(work instanceof SparkWork) { - for(BaseWork bw : ((SparkWork)work).getRoots()) { - collectFileSinkDescs(bw.getAllLeafOperators(), acidSinks); - } - } else if(work instanceof MapredLocalWork) { //I don't think this can have any FileSinkOperatorS - more future proofing Set fileSinkOperatorSet = OperatorUtils.findOperators(((MapredLocalWork) work).getAliasToWork().values(), FileSinkOperator.class); @@ -102,7 +95,6 @@ else if(work instanceof ExplainWork) { else if(work instanceof ReplLoadWork || work instanceof ReplStateLogWork || work instanceof GenTezWork || - work instanceof GenSparkWork || work instanceof ArchiveWork || work instanceof ColumnStatsUpdateWork || work instanceof BasicStatsWork || diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkMapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkMapJoinProcessor.java deleted file mode 100644 index 16d088ab0b3..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkMapJoinProcessor.java +++ /dev/null @@ -1,86 +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.hadoop.hive.ql.optimizer; - -import java.util.List; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.JoinCondDesc; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; - -import com.google.common.base.Preconditions; - -public class SparkMapJoinProcessor extends MapJoinProcessor { - - /** - * convert a regular join to a a map-side join. - * - * @param conf - * @param opParseCtxMap - * @param op join operator - * @param joinTree qb join tree - * @param bigTablePos position of the source to be read as part of - * map-reduce framework. All other sources are cached in memory - * @param noCheckOuterJoin - * @param validateMapJoinTree - */ - @Override - public MapJoinOperator convertMapJoin(HiveConf conf, - JoinOperator op, boolean leftSrc, String[] baseSrc, List mapAliases, - int bigTablePos, boolean noCheckOuterJoin, boolean validateMapJoinTree) - throws SemanticException { - - // outer join cannot be performed on a table which is being cached - JoinCondDesc[] condns = op.getConf().getConds(); - - if (!noCheckOuterJoin) { - if (checkMapJoin(bigTablePos, condns) < 0) { - throw new SemanticException(ErrorMsg.NO_OUTER_MAPJOIN.getMsg()); - } - } - - // create the map-join operator - MapJoinOperator mapJoinOp = convertJoinOpMapJoinOp(conf, - op, op.getConf().isLeftInputJoin(), op.getConf().getBaseSrc(), - op.getConf().getMapAliases(), bigTablePos, noCheckOuterJoin); - - // 1. remove RS as parent for the big table branch - // 2. remove old join op from child set of all the RSs - List> parentOps = mapJoinOp.getParentOperators(); - for (int i = 0; i < parentOps.size(); i++) { - Operator parentOp = parentOps.get(i); - parentOp.getChildOperators().remove(op); - if (i == bigTablePos) { - List> grandParentOps = parentOp.getParentOperators(); - Preconditions.checkArgument(grandParentOps.size() == 1, - "AssertionError: expect number of parents to be 1, but was " + grandParentOps.size()); - Operator grandParentOp = grandParentOps.get(0); - grandParentOp.replaceChild(parentOp, mapJoinOp); - mapJoinOp.replaceParent(parentOp, grandParentOp); - } - } - - return mapJoinOp; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java deleted file mode 100644 index a117a6573c4..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SparkRemoveDynamicPruning.java +++ /dev/null @@ -1,80 +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.hadoop.hive.ql.optimizer; - -import java.util.Stack; - -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; - -/** - * Check if dynamic partition pruning should be disabled. Currently the following 2 cases - * checked. - * 1. The expected number of keys for dynamic pruning is too large - * 2. If DPP enabled only for mapjoin and join is not a map join. - * - * Cloned from RemoveDynamicPruningBySize - */ -public class SparkRemoveDynamicPruning implements NodeProcessor { - - static final private Logger LOG = - LoggerFactory.getLogger(SparkRemoveDynamicPruning.class.getName()); - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procContext, - Object... nodeOutputs) - throws SemanticException { - - OptimizeSparkProcContext context = (OptimizeSparkProcContext) procContext; - boolean remove = false; - - SparkPartitionPruningSinkOperator op = (SparkPartitionPruningSinkOperator) nd; - SparkPartitionPruningSinkDesc desc = op.getConf(); - - if (context.getConf().isSparkDPPOnlyMapjoin() && - !op.isWithMapjoin()) { - LOG.info("Disabling dynamic partition pruning based on: " + desc.getTableScanNames() - + ". This is not part of a map join."); - remove = true; - } - else if (desc.getStatistics().getDataSize() > context.getConf() - .getLongVar(ConfVars.SPARK_DYNAMIC_PARTITION_PRUNING_MAX_DATA_SIZE)) { - LOG.info("Disabling dynamic partition pruning based on: " - + desc.getTableScanNames() - + ". Expected data size is too big: " + desc.getStatistics().getDataSize()); - remove = true; - } - - if (remove) { - // at this point we've found the fork in the op pipeline that has the pruning as a child plan. - OperatorUtils.removeBranch(op); - } - - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java index c04b4717e7c..07d7b96dde2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AnnotateRunTimeStatsOptimizer.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; import org.apache.hadoop.hive.ql.lib.Dispatcher; @@ -47,7 +46,6 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.BaseWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TezWork; import org.apache.hadoop.hive.ql.stats.StatsCollectionContext; import org.apache.hadoop.hive.ql.stats.StatsPublisher; @@ -79,11 +77,6 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) for (BaseWork w : work.getAllWork()) { ops.addAll(w.getAllOperators()); } - } else if (currTask instanceof SparkTask) { - SparkWork sparkWork = (SparkWork) currTask.getWork(); - for (BaseWork w : sparkWork.getAllWork()) { - ops.addAll(w.getAllOperators()); - } } setOrAnnotateStats(ops, physicalContext.getParseContext()); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java deleted file mode 100644 index 7f7f49ba8ca..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java +++ /dev/null @@ -1,423 +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.hadoop.hive.ql.optimizer.physical; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorFactory; -import org.apache.hadoop.hive.ql.exec.RowSchema; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; -import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.io.HiveInputFormat; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.ConditionalResolverSkewJoin; -import org.apache.hadoop.hive.ql.plan.ConditionalWork; -import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc; -import org.apache.hadoop.hive.ql.plan.JoinDesc; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.PartitionDesc; -import org.apache.hadoop.hive.ql.plan.PlanUtils; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; - -/** - * Copied from GenMRSkewJoinProcessor. It's used for spark task - * - */ -public class GenSparkSkewJoinProcessor { - private static final Logger LOG = LoggerFactory.getLogger(GenSparkSkewJoinProcessor.class.getName()); - - private GenSparkSkewJoinProcessor() { - // prevent instantiation - } - - @SuppressWarnings("unchecked") - public static void processSkewJoin(JoinOperator joinOp, Task currTask, - ReduceWork reduceWork, ParseContext parseCtx) throws SemanticException { - - SparkWork currentWork = ((SparkTask) currTask).getWork(); - if (currentWork.getChildren(reduceWork).size() > 0) { - LOG.warn("Skip runtime skew join as the ReduceWork has child work and hasn't been split."); - return; - } - - List> children = currTask.getChildTasks(); - - Path baseTmpDir = parseCtx.getContext().getMRTmpPath(); - - JoinDesc joinDescriptor = joinOp.getConf(); - Map> joinValues = joinDescriptor.getExprs(); - int numAliases = joinValues.size(); - - Map bigKeysDirMap = new HashMap(); - Map> smallKeysDirMap = new HashMap>(); - Map skewJoinJobResultsDir = new HashMap(); - Byte[] tags = joinDescriptor.getTagOrder(); - // for each joining table, set dir for big key and small keys properly - for (int i = 0; i < numAliases; i++) { - Byte alias = tags[i]; - bigKeysDirMap.put(alias, GenMRSkewJoinProcessor.getBigKeysDir(baseTmpDir, alias)); - Map smallKeysMap = new HashMap(); - smallKeysDirMap.put(alias, smallKeysMap); - for (Byte src2 : tags) { - if (!src2.equals(alias)) { - smallKeysMap.put(src2, GenMRSkewJoinProcessor.getSmallKeysDir(baseTmpDir, alias, src2)); - } - } - skewJoinJobResultsDir.put(alias, - GenMRSkewJoinProcessor.getBigKeysSkewJoinResultDir(baseTmpDir, alias)); - } - - joinDescriptor.setHandleSkewJoin(true); - joinDescriptor.setBigKeysDirMap(bigKeysDirMap); - joinDescriptor.setSmallKeysDirMap(smallKeysDirMap); - joinDescriptor.setSkewKeyDefinition(HiveConf.getIntVar(parseCtx.getConf(), - HiveConf.ConfVars.HIVESKEWJOINKEY)); - - // create proper table/column desc for spilled tables - TableDesc keyTblDesc = (TableDesc) reduceWork.getKeyDesc().clone(); - List joinKeys = Utilities - .getColumnNames(keyTblDesc.getProperties()); - List joinKeyTypes = Utilities.getColumnTypes(keyTblDesc - .getProperties()); - - Map tableDescList = new HashMap(); - Map rowSchemaList = new HashMap(); - Map> newJoinValues = new HashMap>(); - Map> newJoinKeys = new HashMap>(); - // used for create mapJoinDesc, should be in order - List newJoinValueTblDesc = new ArrayList(); - - for (int i = 0; i < tags.length; i++) { - newJoinValueTblDesc.add(null); - } - - for (int i = 0; i < numAliases; i++) { - Byte alias = tags[i]; - List valueCols = joinValues.get(alias); - String colNames = ""; - String colTypes = ""; - int columnSize = valueCols.size(); - List newValueExpr = new ArrayList(); - List newKeyExpr = new ArrayList(); - ArrayList columnInfos = new ArrayList(); - - boolean first = true; - for (int k = 0; k < columnSize; k++) { - TypeInfo type = valueCols.get(k).getTypeInfo(); - String newColName = i + "_VALUE_" + k; // any name, it does not matter. - ColumnInfo columnInfo = new ColumnInfo(newColName, type, alias.toString(), false); - columnInfos.add(columnInfo); - newValueExpr.add(new ExprNodeColumnDesc( - columnInfo.getType(), columnInfo.getInternalName(), - columnInfo.getTabAlias(), false)); - if (!first) { - colNames = colNames + ","; - colTypes = colTypes + ","; - } - first = false; - colNames = colNames + newColName; - colTypes = colTypes + valueCols.get(k).getTypeString(); - } - - // we are putting join keys at last part of the spilled table - for (int k = 0; k < joinKeys.size(); k++) { - if (!first) { - colNames = colNames + ","; - colTypes = colTypes + ","; - } - first = false; - colNames = colNames + joinKeys.get(k); - colTypes = colTypes + joinKeyTypes.get(k); - ColumnInfo columnInfo = new ColumnInfo(joinKeys.get(k), TypeInfoFactory - .getPrimitiveTypeInfo(joinKeyTypes.get(k)), alias.toString(), false); - columnInfos.add(columnInfo); - newKeyExpr.add(new ExprNodeColumnDesc( - columnInfo.getType(), columnInfo.getInternalName(), - columnInfo.getTabAlias(), false)); - } - - newJoinValues.put(alias, newValueExpr); - newJoinKeys.put(alias, newKeyExpr); - tableDescList.put(alias, Utilities.getTableDesc(colNames, colTypes)); - rowSchemaList.put(alias, new RowSchema(columnInfos)); - - // construct value table Desc - String valueColNames = ""; - String valueColTypes = ""; - first = true; - for (int k = 0; k < columnSize; k++) { - String newColName = i + "_VALUE_" + k; // any name, it does not matter. - if (!first) { - valueColNames = valueColNames + ","; - valueColTypes = valueColTypes + ","; - } - valueColNames = valueColNames + newColName; - valueColTypes = valueColTypes + valueCols.get(k).getTypeString(); - first = false; - } - newJoinValueTblDesc.set((byte) i, Utilities.getTableDesc( - valueColNames, valueColTypes)); - } - - joinDescriptor.setSkewKeysValuesTables(tableDescList); - joinDescriptor.setKeyTableDesc(keyTblDesc); - - // create N-1 map join tasks - HashMap> bigKeysDirToTaskMap = - new HashMap>(); - List listWorks = new ArrayList(); - List> listTasks = new ArrayList>(); - for (int i = 0; i < numAliases - 1; i++) { - Byte src = tags[i]; - HiveConf hiveConf = new HiveConf(parseCtx.getConf(), - GenSparkSkewJoinProcessor.class); - SparkWork sparkWork = new SparkWork(parseCtx.getConf().getVar(HiveConf.ConfVars.HIVEQUERYID)); - Task skewJoinMapJoinTask = TaskFactory.get(sparkWork); - skewJoinMapJoinTask.setFetchSource(currTask.isFetchSource()); - - // create N TableScans - Operator[] parentOps = new TableScanOperator[tags.length]; - for (int k = 0; k < tags.length; k++) { - Operator ts = GenMapRedUtils.createTemporaryTableScanOperator( - joinOp.getCompilationOpContext(), rowSchemaList.get((byte) k)); - ((TableScanOperator) ts).setTableDescSkewJoin(tableDescList.get((byte) k)); - parentOps[k] = ts; - } - - // create the MapJoinOperator - String dumpFilePrefix = "mapfile" + PlanUtils.getCountForMapJoinDumpFilePrefix(); - MapJoinDesc mapJoinDescriptor = new MapJoinDesc(newJoinKeys, keyTblDesc, - newJoinValues, newJoinValueTblDesc, newJoinValueTblDesc, joinDescriptor - .getOutputColumnNames(), i, joinDescriptor.getConds(), - joinDescriptor.getFilters(), joinDescriptor.getNoOuterJoin(), dumpFilePrefix, - joinDescriptor.getMemoryMonitorInfo(), joinDescriptor.getInMemoryDataSize()); - mapJoinDescriptor.setTagOrder(tags); - mapJoinDescriptor.setHandleSkewJoin(false); - mapJoinDescriptor.setNullSafes(joinDescriptor.getNullSafes()); - mapJoinDescriptor.setColumnExprMap(joinDescriptor.getColumnExprMap()); - // temporarily, mark it as child of all the TS - MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory - .getAndMakeChild(joinOp.getCompilationOpContext(), mapJoinDescriptor, null, parentOps); - - // clone the original join operator, and replace it with the MJ - // this makes sure MJ has the same downstream operator plan as the original join - List> reducerList = new ArrayList>(); - reducerList.add(reduceWork.getReducer()); - Operator reducer = SerializationUtilities.cloneOperatorTree( - reducerList).get(0); - Preconditions.checkArgument(reducer instanceof JoinOperator, - "Reducer should be join operator, but actually is " + reducer.getName()); - JoinOperator cloneJoinOp = (JoinOperator) reducer; - List> childOps = cloneJoinOp - .getChildOperators(); - for (Operator childOp : childOps) { - childOp.replaceParent(cloneJoinOp, mapJoinOp); - } - mapJoinOp.setChildOperators(childOps); - - // set memory usage for the MJ operator - setMemUsage(mapJoinOp, skewJoinMapJoinTask, parseCtx); - - // create N MapWorks and add them to the SparkWork - MapWork bigMapWork = null; - Map smallTblDirs = smallKeysDirMap.get(src); - for (int j = 0; j < tags.length; j++) { - MapWork mapWork = PlanUtils.getMapRedWork().getMapWork(); - sparkWork.add(mapWork); - // This code has been only added for testing - boolean mapperCannotSpanPartns = - parseCtx.getConf().getBoolVar( - HiveConf.ConfVars.HIVE_MAPPER_CANNOT_SPAN_MULTIPLE_PARTITIONS); - mapWork.setMapperCannotSpanPartns(mapperCannotSpanPartns); - Operator tableScan = parentOps[j]; - String alias = tags[j].toString(); - ArrayList aliases = new ArrayList(); - aliases.add(alias); - Path path; - if (j == i) { - path = bigKeysDirMap.get(tags[j]); - bigKeysDirToTaskMap.put(path, skewJoinMapJoinTask); - bigMapWork = mapWork; - } else { - path = smallTblDirs.get(tags[j]); - } - mapWork.addPathToAlias(path, aliases); - mapWork.getAliasToWork().put(alias, tableScan); - PartitionDesc partitionDesc = new PartitionDesc(tableDescList.get(tags[j]), null); - mapWork.addPathToPartitionInfo(path, partitionDesc); - mapWork.getAliasToPartnInfo().put(alias, partitionDesc); - mapWork.setName("Map " + GenSparkUtils.getUtils().getNextSeqNumber()); - } - // connect all small dir map work to the big dir map work - Preconditions.checkArgument(bigMapWork != null, "Haven't identified big dir MapWork"); - // these 2 flags are intended only for the big-key map work - bigMapWork.setNumMapTasks(HiveConf.getIntVar(hiveConf, - HiveConf.ConfVars.HIVESKEWJOINMAPJOINNUMMAPTASK)); - bigMapWork.setMinSplitSize(HiveConf.getLongVar(hiveConf, - HiveConf.ConfVars.HIVESKEWJOINMAPJOINMINSPLIT)); - // use HiveInputFormat so that we can control the number of map tasks - bigMapWork.setInputformat(HiveInputFormat.class.getName()); - for (BaseWork work : sparkWork.getRoots()) { - Preconditions.checkArgument(work instanceof MapWork, - "All root work should be MapWork, but got " + work.getClass().getSimpleName()); - if (work != bigMapWork) { - sparkWork.connect(work, bigMapWork, - new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE)); - } - } - - // insert SparkHashTableSink and Dummy operators - for (int j = 0; j < tags.length; j++) { - if (j != i) { - insertSHTS(tags[j], (TableScanOperator) parentOps[j], bigMapWork); - } - } - - listWorks.add(skewJoinMapJoinTask.getWork()); - listTasks.add(skewJoinMapJoinTask); - } - if (children != null) { - for (Task tsk : listTasks) { - for (Task oldChild : children) { - tsk.addDependentTask(oldChild); - } - } - currTask.setChildTasks(new ArrayList>()); - for (Task oldChild : children) { - oldChild.getParentTasks().remove(currTask); - } - listTasks.addAll(children); - for (Task oldChild : children) { - listWorks.add(oldChild.getWork()); - } - } - ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx context = - new ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx(bigKeysDirToTaskMap, children); - - ConditionalWork cndWork = new ConditionalWork(listWorks); - ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork); - cndTsk.setListTasks(listTasks); - cndTsk.setResolver(new ConditionalResolverSkewJoin()); - cndTsk.setResolverCtx(context); - currTask.setChildTasks(new ArrayList>()); - currTask.addDependentTask(cndTsk); - } - - /** - * Insert SparkHashTableSink and HashTableDummy between small dir TS and MJ. - */ - @SuppressWarnings("unchecked") - private static void insertSHTS(byte tag, TableScanOperator tableScan, MapWork bigMapWork) { - Preconditions.checkArgument(tableScan.getChildOperators().size() == 1 - && tableScan.getChildOperators().get(0) instanceof MapJoinOperator); - HashTableDummyDesc desc = new HashTableDummyDesc(); - HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get( - tableScan.getCompilationOpContext(), desc); - dummyOp.getConf().setTbl(tableScan.getTableDescSkewJoin()); - MapJoinOperator mapJoinOp = (MapJoinOperator) tableScan.getChildOperators().get(0); - mapJoinOp.replaceParent(tableScan, dummyOp); - List> mapJoinChildren = - new ArrayList>(); - mapJoinChildren.add(mapJoinOp); - dummyOp.setChildOperators(mapJoinChildren); - bigMapWork.addDummyOp(dummyOp); - MapJoinDesc mjDesc = mapJoinOp.getConf(); - // mapjoin should not be affected by join reordering - mjDesc.resetOrder(); - SparkHashTableSinkDesc hashTableSinkDesc = new SparkHashTableSinkDesc(mjDesc); - SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator)OperatorFactory.get( - tableScan.getCompilationOpContext(), hashTableSinkDesc); - int[] valueIndex = mjDesc.getValueIndex(tag); - if (valueIndex != null) { - List newValues = new ArrayList(); - List values = hashTableSinkDesc.getExprs().get(tag); - for (int index = 0; index < values.size(); index++) { - if (valueIndex[index] < 0) { - newValues.add(values.get(index)); - } - } - hashTableSinkDesc.getExprs().put(tag, newValues); - } - tableScan.replaceChild(mapJoinOp, hashTableSinkOp); - List> tableScanParents = - new ArrayList>(); - tableScanParents.add(tableScan); - hashTableSinkOp.setParentOperators(tableScanParents); - hashTableSinkOp.getConf().setTag(tag); - } - - private static void setMemUsage(MapJoinOperator mapJoinOp, Task task, - ParseContext parseContext) { - MapJoinResolver.LocalMapJoinProcCtx context = - new MapJoinResolver.LocalMapJoinProcCtx(task, parseContext); - try { - new LocalMapJoinProcFactory.LocalMapJoinProcessor().hasGroupBy(mapJoinOp, - context); - } catch (Exception e) { - LOG.warn("Error setting memory usage.", e); - return; - } - MapJoinDesc mapJoinDesc = mapJoinOp.getConf(); - HiveConf conf = context.getParseCtx().getConf(); - float hashtableMemoryUsage; - if (context.isFollowedByGroupBy()) { - hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE); - } else { - hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEMAXMEMORYUSAGE); - } - mapJoinDesc.setHashTableMemoryUsage(hashtableMemoryUsage); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java deleted file mode 100644 index 50245963a0f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkCrossProductCheck.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.optimizer.physical; - -import org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.TaskGraphWalker; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Stack; -import java.util.TreeMap; - -/** - * Check each MapJoin and ShuffleJoin Operator to see if they are performing a cross product. - * If yes, output a warning to the Session's console. - * The Checks made are the following: - * 1. Shuffle Join: - * Check the parent ReduceSinkOp of the JoinOp. If its keys list is size = 0, then - * this is a cross product. - * 2. Map Join: - * If the keys expr list on the mapJoin Desc is an empty list for any input, - * this implies a cross product. - */ -public class SparkCrossProductCheck implements PhysicalPlanResolver, Dispatcher { - - @Override - public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) - throws SemanticException { - @SuppressWarnings("unchecked") - Task currTask = (Task) nd; - if (currTask instanceof SparkTask) { - SparkWork sparkWork = ((SparkTask) currTask).getWork(); - checkShuffleJoin(sparkWork); - checkMapJoin((SparkTask) currTask); - } else if (currTask instanceof ConditionalTask) { - List> taskList = ((ConditionalTask) currTask).getListTasks(); - for (Task task : taskList) { - dispatch(task, stack, nodeOutputs); - } - } - return null; - } - - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - TaskGraphWalker ogw = new TaskGraphWalker(this); - - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pctx.getRootTasks()); - - ogw.startWalking(topNodes, null); - return pctx; - } - - private void warn(String msg) { - SessionState.getConsole().printInfo("Warning: " + msg, false); - } - - private void checkShuffleJoin(SparkWork sparkWork) throws SemanticException { - for (ReduceWork reduceWork : sparkWork.getAllReduceWork()) { - Operator reducer = reduceWork.getReducer(); - if (reducer instanceof JoinOperator || reducer instanceof CommonMergeJoinOperator) { - Map rsInfo = new TreeMap(); - for (BaseWork parent : sparkWork.getParents(reduceWork)) { - rsInfo.putAll(new CrossProductHandler.ExtractReduceSinkInfo(null).analyze(parent)); - } - checkForCrossProduct(reduceWork.getName(), reducer, rsInfo); - } - } - } - - private void checkMapJoin(SparkTask sparkTask) throws SemanticException { - SparkWork sparkWork = sparkTask.getWork(); - for (BaseWork baseWork : sparkWork.getAllWork()) { - List warnings = - new CrossProductHandler.MapJoinCheck(sparkTask.toString()).analyze(baseWork); - for (String w : warnings) { - warn(w); - } - } - } - - private void checkForCrossProduct(String workName, - Operator reducer, - Map rsInfo) { - if (rsInfo.isEmpty()) { - return; - } - Iterator it = rsInfo.values().iterator(); - CrossProductHandler.ExtractReduceSinkInfo.Info info = it.next(); - if (info.keyCols.size() == 0) { - List iAliases = new ArrayList(); - iAliases.addAll(info.inputAliases); - while (it.hasNext()) { - info = it.next(); - iAliases.addAll(info.inputAliases); - } - String warning = String.format( - "Shuffle Join %s[tables = %s] in Work '%s' is a cross product", - reducer.toString(), - iAliases, - workName); - warn(warning); - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java deleted file mode 100644 index 124138361b6..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java +++ /dev/null @@ -1,141 +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.hadoop.hive.ql.optimizer.physical; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.TaskGraphWalker; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc.DPPTargetInfo; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; - - -/** - * A physical optimization that disables DPP if the source {@link MapWork} and target {@link MapWork} aren't in - * dependent {@link SparkTask}s. - * - *

- * When DPP is run, the source {@link MapWork} produces a temp file that is read by the target {@link MapWork}. The - * source {@link MapWork} must be run before the target {@link MapWork} is run, otherwise the target {@link MapWork} - * will throw a {@link java.io.FileNotFoundException}. In order to guarantee this, the source {@link MapWork} must be - * inside a {@link SparkTask} that runs before the {@link SparkTask} containing the target {@link MapWork}. - *

- * - *

- * This {@link PhysicalPlanResolver} works by walking through the {@link Task} DAG and iterating over all the - * {@link SparkPartitionPruningSinkOperator}s inside the {@link SparkTask}. For each sink operator, it takes the - * target {@link MapWork} and checks if it exists in any of the child {@link SparkTask}s. If the target {@link MapWork} - * is not in any child {@link SparkTask} then it removes the operator subtree that contains the - * {@link SparkPartitionPruningSinkOperator}. - *

- */ -public class SparkDynamicPartitionPruningResolver implements PhysicalPlanResolver { - - private static final Logger LOG = LoggerFactory.getLogger(SparkDynamicPartitionPruningResolver.class.getName()); - - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - // Walk through the Task Graph and invoke SparkDynamicPartitionPruningDispatcher - TaskGraphWalker graphWalker = new TaskGraphWalker(new SparkDynamicPartitionPruningDispatcher()); - - ArrayList rootTasks = new ArrayList<>(); - rootTasks.addAll(pctx.getRootTasks()); - graphWalker.startWalking(rootTasks, null); - return pctx; - } - - private class SparkDynamicPartitionPruningDispatcher implements Dispatcher { - - @Override - public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws SemanticException { - Task task = (Task) nd; - - // If the given Task is a SparkTask then search its Work DAG for SparkPartitionPruningSinkOperator - if (task instanceof SparkTask) { - - // Search for any SparkPartitionPruningSinkOperator in the SparkTask - for (BaseWork baseWork : ((SparkTask) task).getWork().getAllWork()) { - Set> pruningSinkOps = OperatorUtils.getOp(baseWork, SparkPartitionPruningSinkOperator.class); - - // For each SparkPartitionPruningSinkOperator, take the target MapWork and see if it is in a dependent SparkTask - for (Operator op : pruningSinkOps) { - SparkPartitionPruningSinkOperator pruningSinkOp = - (SparkPartitionPruningSinkOperator) op; - SparkPartitionPruningSinkDesc desc = pruningSinkOp.getConf(); - List toRemove = new ArrayList<>(); - for (DPPTargetInfo targetInfo : desc.getTargetInfos()) { - MapWork targetMapWork = targetInfo.work; - // Check if the given SparkTask has a child SparkTask that contains the target MapWork - // If it does not, then remove the target from DPP op - if (!taskContainsDependentMapWork(task, targetMapWork)) { - toRemove.add(targetInfo); - pruningSinkOp.removeFromSourceEvent(targetMapWork, targetInfo.partKey, - targetInfo.columnName, targetInfo.columnType); - LOG.info("Removing target map work " + targetMapWork.getName() + " from " + baseWork - .getName() + " as no dependency exists between the two works."); - } - } - desc.getTargetInfos().removeAll(toRemove); - if (desc.getTargetInfos().isEmpty()) { - // The DPP sink has no target, remove the subtree. - OperatorUtils.removeBranch(pruningSinkOp); - } - } - } - } - return null; - } - } - - /** - * Recursively go through the children of the given {@link Task} and check if any child {@link SparkTask} contains - * the specified {@link MapWork} object. - */ - private boolean taskContainsDependentMapWork(Task task, - MapWork work) throws SemanticException { - if (task == null || task.getChildTasks() == null) { - return false; - } - for (Task childTask : task.getChildTasks()) { - if (childTask != null && childTask instanceof SparkTask && childTask.getMapWork().contains(work)) { - return true; - } else if (taskContainsDependentMapWork(childTask, work)) { - return true; - } - } - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java deleted file mode 100644 index 55b46e50294..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java +++ /dev/null @@ -1,394 +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.hadoop.hive.ql.optimizer.physical; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.TaskGraphWalker; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.ConditionalResolver; -import org.apache.hadoop.hive.ql.plan.ConditionalResolverSkewJoin; -import org.apache.hadoop.hive.ql.plan.ConditionalWork; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkBucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; - -public class SparkMapJoinResolver implements PhysicalPlanResolver { - - // prevents a task from being processed multiple times - private final Set> visitedTasks = new HashSet<>(); - - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - - Dispatcher dispatcher = new SparkMapJoinTaskDispatcher(pctx); - TaskGraphWalker graphWalker = new TaskGraphWalker(dispatcher); - - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pctx.getRootTasks()); - graphWalker.startWalking(topNodes, null); - return pctx; - } - - // Check whether the specified BaseWork's operator tree contains a operator - // of the specified operator class - private boolean containsOp(BaseWork work, Class clazz) { - Set> matchingOps = OperatorUtils.getOp(work, clazz); - return matchingOps != null && !matchingOps.isEmpty(); - } - - private boolean containsOp(SparkWork sparkWork, Class clazz) { - for (BaseWork work : sparkWork.getAllWorkUnsorted()) { - if (containsOp(work, clazz)) { - return true; - } - } - return false; - } - - @SuppressWarnings("unchecked") - class SparkMapJoinTaskDispatcher implements Dispatcher { - - private final PhysicalContext physicalContext; - - // For each BaseWork with MJ operator, we build a SparkWork for its small table BaseWorks - // This map records such information - private final Map sparkWorkMap; - - // SparkWork dependency graph - from a SparkWork with MJ operators to all - // of its parent SparkWorks for the small tables - private final Map> dependencyGraph; - - public SparkMapJoinTaskDispatcher(PhysicalContext pc) { - super(); - physicalContext = pc; - sparkWorkMap = new LinkedHashMap(); - dependencyGraph = new LinkedHashMap>(); - } - - // Move the specified work from the sparkWork to the targetWork - // Note that, in order not to break the graph (since we need it for the edges), - // we don't remove the work from the sparkWork here. The removal is done later. - private void moveWork(SparkWork sparkWork, BaseWork work, SparkWork targetWork) { - List parentWorks = sparkWork.getParents(work); - if (sparkWork != targetWork) { - targetWork.add(work); - - // If any child work for this work is already added to the targetWork earlier, - // we should connect this work with it - for (BaseWork childWork : sparkWork.getChildren(work)) { - if (targetWork.contains(childWork)) { - targetWork.connect(work, childWork, sparkWork.getEdgeProperty(work, childWork)); - } - } - } - - if (!containsOp(work, MapJoinOperator.class)) { - for (BaseWork parent : parentWorks) { - moveWork(sparkWork, parent, targetWork); - } - } else { - // Create a new SparkWork for all the small tables of this work - SparkWork parentWork = - new SparkWork(physicalContext.conf.getVar(HiveConf.ConfVars.HIVEQUERYID)); - // copy cloneToWork to ensure RDD cache still works - parentWork.setCloneToWork(sparkWork.getCloneToWork()); - - dependencyGraph.get(targetWork).add(parentWork); - dependencyGraph.put(parentWork, new ArrayList()); - - // this work is now moved to the parentWork, thus we should - // update this information in sparkWorkMap - sparkWorkMap.put(work, parentWork); - for (BaseWork parent : parentWorks) { - if (containsOp(parent, SparkHashTableSinkOperator.class)) { - moveWork(sparkWork, parent, parentWork); - } else { - moveWork(sparkWork, parent, targetWork); - } - } - } - } - - private void generateLocalWork(SparkTask originalTask) { - SparkWork originalWork = originalTask.getWork(); - Collection allBaseWorks = originalWork.getAllWork(); - Context ctx = physicalContext.getContext(); - - for (BaseWork work : allBaseWorks) { - if (work.getMapRedLocalWork() == null) { - if (containsOp(work, SparkHashTableSinkOperator.class) || - containsOp(work, MapJoinOperator.class)) { - work.setMapRedLocalWork(new MapredLocalWork()); - } - Set> ops = OperatorUtils.getOp(work, MapJoinOperator.class); - if (ops == null || ops.isEmpty()) { - continue; - } - Path tmpPath = Utilities.generateTmpPath(ctx.getMRTmpPath(), originalTask.getId()); - MapredLocalWork bigTableLocalWork = work.getMapRedLocalWork(); - List> dummyOps = - new ArrayList>(work.getDummyOps()); - bigTableLocalWork.setDummyParentOp(dummyOps); - bigTableLocalWork.setTmpPath(tmpPath); - - // In one work, only one map join operator can be bucketed - SparkBucketMapJoinContext bucketMJCxt = null; - for (Operator op : ops) { - MapJoinOperator mapJoinOp = (MapJoinOperator) op; - MapJoinDesc mapJoinDesc = mapJoinOp.getConf(); - if (mapJoinDesc.isBucketMapJoin()) { - bucketMJCxt = new SparkBucketMapJoinContext(mapJoinDesc); - bucketMJCxt.setBucketMatcherClass( - org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); - bucketMJCxt.setPosToAliasMap(mapJoinOp.getPosToAliasMap()); - ((MapWork) work).setUseBucketizedHiveInputFormat(true); - bigTableLocalWork.setBucketMapjoinContext(bucketMJCxt); - bigTableLocalWork.setInputFileChangeSensitive(true); - break; - } - } - - for (BaseWork parentWork : originalWork.getParents(work)) { - Set> hashTableSinkOps = - OperatorUtils.getOp(parentWork, SparkHashTableSinkOperator.class); - if (hashTableSinkOps == null || hashTableSinkOps.isEmpty()) { - continue; - } - MapredLocalWork parentLocalWork = parentWork.getMapRedLocalWork(); - parentLocalWork.setTmpHDFSPath(tmpPath); - if (bucketMJCxt != null) { - // We only need to update the work with the hashtable - // sink operator with the same mapjoin desc. We can tell - // that by comparing the bucket file name mapping map - // instance. They should be exactly the same one due to - // the way how the bucket mapjoin context is constructed. - for (Operator op : hashTableSinkOps) { - SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) op; - SparkHashTableSinkDesc hashTableSinkDesc = hashTableSinkOp.getConf(); - BucketMapJoinContext original = hashTableSinkDesc.getBucketMapjoinContext(); - if (original != null && original.getBucketFileNameMapping() - == bucketMJCxt.getBucketFileNameMapping()) { - ((MapWork) parentWork).setUseBucketizedHiveInputFormat(true); - parentLocalWork.setBucketMapjoinContext(bucketMJCxt); - parentLocalWork.setInputFileChangeSensitive(true); - break; - } - } - } - } - } - } - } - - // Create a new SparkTask for the specified SparkWork, recursively compute - // all the parent SparkTasks that this new task is depend on, if they don't already exists. - private SparkTask createSparkTask(SparkTask originalTask, - SparkWork sparkWork, - Map createdTaskMap, - ConditionalTask conditionalTask) { - if (createdTaskMap.containsKey(sparkWork)) { - return createdTaskMap.get(sparkWork); - } - SparkTask resultTask = originalTask.getWork() == sparkWork ? - originalTask : (SparkTask) TaskFactory.get(sparkWork); - if (!dependencyGraph.get(sparkWork).isEmpty()) { - for (SparkWork parentWork : dependencyGraph.get(sparkWork)) { - SparkTask parentTask = - createSparkTask(originalTask, parentWork, createdTaskMap, conditionalTask); - parentTask.addDependentTask(resultTask); - } - } else { - if (originalTask != resultTask) { - List> parentTasks = originalTask.getParentTasks(); - if (parentTasks != null && parentTasks.size() > 0) { - // avoid concurrent modification - originalTask.setParentTasks(new ArrayList>()); - for (Task parentTask : parentTasks) { - parentTask.addDependentTask(resultTask); - parentTask.removeDependentTask(originalTask); - } - } else { - if (conditionalTask == null) { - physicalContext.addToRootTask(resultTask); - physicalContext.removeFromRootTask(originalTask); - } else { - updateConditionalTask(conditionalTask, originalTask, resultTask); - } - } - } - } - - createdTaskMap.put(sparkWork, resultTask); - return resultTask; - } - - @Override - public Object dispatch(Node nd, Stack stack, Object... nos) - throws SemanticException { - Task currentTask = (Task) nd; - if(currentTask.isMapRedTask()) { - if (currentTask instanceof ConditionalTask) { - List> taskList = - ((ConditionalTask) currentTask).getListTasks(); - for (Task tsk : taskList) { - if (tsk instanceof SparkTask) { - processCurrentTask((SparkTask) tsk, (ConditionalTask) currentTask); - visitedTasks.add(tsk); - } - } - } else if (currentTask instanceof SparkTask) { - processCurrentTask((SparkTask) currentTask, null); - visitedTasks.add(currentTask); - } - } - - return null; - } - - /** - * @param sparkTask The current spark task we're processing. - * @param conditionalTask If conditional task is not null, it means the current task is - * wrapped in its task list. - */ - private void processCurrentTask(SparkTask sparkTask, ConditionalTask conditionalTask) { - SparkWork sparkWork = sparkTask.getWork(); - if (!visitedTasks.contains(sparkTask)) { - dependencyGraph.clear(); - sparkWorkMap.clear(); - - // Generate MapredLocalWorks for MJ and HTS - generateLocalWork(sparkTask); - - dependencyGraph.put(sparkWork, new ArrayList()); - Set leaves = sparkWork.getLeaves(); - for (BaseWork leaf : leaves) { - moveWork(sparkWork, leaf, sparkWork); - } - - // Now remove all BaseWorks in all the childSparkWorks that we created - // from the original SparkWork - for (SparkWork newSparkWork : sparkWorkMap.values()) { - for (BaseWork work : newSparkWork.getAllWorkUnsorted()) { - sparkWork.remove(work); - } - } - - Map createdTaskMap = new LinkedHashMap(); - - // Now create SparkTasks from the SparkWorks, also set up dependency - for (SparkWork work : dependencyGraph.keySet()) { - createSparkTask(sparkTask, work, createdTaskMap, conditionalTask); - } - } else if (conditionalTask != null) { - // We may need to update the conditional task's list. This happens when a common map join - // task exists in the task list and has already been processed. In such a case, - // the current task is the map join task and we need to replace it with - // its parent, i.e. the small table task. - if (sparkTask.getParentTasks() != null && sparkTask.getParentTasks().size() == 1 && - sparkTask.getParentTasks().get(0) instanceof SparkTask) { - SparkTask parent = (SparkTask) sparkTask.getParentTasks().get(0); - if (containsOp(sparkWork, MapJoinOperator.class) && - containsOp(parent.getWork(), SparkHashTableSinkOperator.class)) { - updateConditionalTask(conditionalTask, sparkTask, parent); - } - } - } - } - - /** - * Update the task/work list of this conditional task to replace originalTask with newTask. - * For runtime skew join, also update dirToTaskMap for the conditional resolver - */ - private void updateConditionalTask(ConditionalTask conditionalTask, - SparkTask originalTask, SparkTask newTask) { - ConditionalWork conditionalWork = conditionalTask.getWork(); - SparkWork originWork = originalTask.getWork(); - SparkWork newWork = newTask.getWork(); - List> listTask = conditionalTask.getListTasks(); - List listWork = (List) conditionalWork.getListWorks(); - int taskIndex = listTask.indexOf(originalTask); - int workIndex = listWork.indexOf(originWork); - if (taskIndex < 0 || workIndex < 0) { - return; - } - listTask.set(taskIndex, newTask); - listWork.set(workIndex, newWork); - ConditionalResolver resolver = conditionalTask.getResolver(); - if (resolver instanceof ConditionalResolverSkewJoin) { - // get bigKeysDirToTaskMap - ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx context = - (ConditionalResolverSkewJoin.ConditionalResolverSkewJoinCtx) conditionalTask - .getResolverCtx(); - HashMap> bigKeysDirToTaskMap = context - .getDirToTaskMap(); - // to avoid concurrent modify the hashmap - HashMap> newbigKeysDirToTaskMap = - new HashMap>(); - // reset the resolver - for (Map.Entry> entry : - bigKeysDirToTaskMap.entrySet()) { - Task task = entry.getValue(); - Path bigKeyDir = entry.getKey(); - if (task.equals(originalTask)) { - newbigKeysDirToTaskMap.put(bigKeyDir, newTask); - } else { - newbigKeysDirToTaskMap.put(bigKeyDir, task); - } - } - context.setDirToTaskMap(newbigKeysDirToTaskMap); - // update no skew task - if (context.getNoSkewTask() != null && context.getNoSkewTask().equals(originalTask)) { - List> noSkewTask = new ArrayList<>(); - noSkewTask.add(newTask); - context.setNoSkewTask(noSkewTask); - } - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java index 15f8717ea61..9e6ee9d8e01 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hive.ql.exec.vector.reducesink.*; import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc; import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.Path; @@ -57,7 +56,6 @@ import org.apache.hadoop.hive.ql.exec.*; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.exec.persistence.MapJoinKey; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor; import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinInnerBigOnlyLongOperator; @@ -134,8 +132,6 @@ import org.apache.hadoop.hive.ql.plan.VectorPTFDesc.SupportedFunctionType; import org.apache.hadoop.hive.ql.plan.VectorTableScanDesc; import org.apache.hadoop.hive.ql.plan.VectorGroupByDesc.ProcessingMode; -import org.apache.hadoop.hive.ql.plan.VectorSparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.VectorSparkPartitionPruningSinkDesc; import org.apache.hadoop.hive.ql.plan.VectorTopNKeyDesc; import org.apache.hadoop.hive.ql.plan.VectorLimitDesc; import org.apache.hadoop.hive.ql.plan.VectorMapJoinInfo; @@ -144,9 +140,6 @@ import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; import org.apache.hadoop.hive.ql.plan.ReduceWork; import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; import org.apache.hadoop.hive.ql.plan.TableDesc; import org.apache.hadoop.hive.ql.plan.TableScanDesc; import org.apache.hadoop.hive.ql.plan.TezWork; @@ -967,26 +960,6 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) setReduceWorkExplainConditions(reduceWork); // We are only vectorizing Reduce under Tez/Spark. - if (isReduceVectorizationEnabled) { - convertReduceWork(reduceWork); - } - logReduceWorkExplainVectorization(reduceWork); - } - } - } else if (currTask instanceof SparkTask) { - SparkWork sparkWork = (SparkWork) currTask.getWork(); - for (BaseWork baseWork : sparkWork.getAllWork()) { - if (baseWork instanceof MapWork) { - MapWork mapWork = (MapWork) baseWork; - setMapWorkExplainConditions(mapWork); - convertMapWork(mapWork, /* isTezOrSpark */ true); - logMapWorkExplainVectorization(mapWork); - } else if (baseWork instanceof ReduceWork) { - ReduceWork reduceWork = (ReduceWork) baseWork; - - // Always set the EXPLAIN conditions. - setReduceWorkExplainConditions(reduceWork); - if (isReduceVectorizationEnabled) { convertReduceWork(reduceWork); } @@ -2514,18 +2487,6 @@ private boolean validateMapJoinDesc(MapJoinDesc desc) { return true; } - private boolean validateSparkHashTableSinkOperator(SparkHashTableSinkOperator op) { - SparkHashTableSinkDesc desc = op.getConf(); - byte tag = desc.getTag(); - // it's essentially a MapJoinDesc - List filterExprs = desc.getFilters().get(tag); - List keyExprs = desc.getKeys().get(tag); - List valueExprs = desc.getExprs().get(tag); - return validateExprNodeDesc( - filterExprs, "Filter", VectorExpressionDescriptor.Mode.FILTER, /* allowComplex */ true) && - validateExprNodeDesc(keyExprs, "Key") && validateExprNodeDesc(valueExprs, "Value"); - } - private boolean validateReduceSinkOperator(ReduceSinkOperator op) { List keyDescs = op.getConf().getKeyCols(); List partitionDescs = op.getConf().getPartitionCols(); @@ -5237,33 +5198,6 @@ public Operator validateAndVectorizeOperator(Operator - * 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.hadoop.hive.ql.optimizer.spark; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.PartitionDesc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.TaskGraphWalker; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalPlanResolver; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; - - -/** - * CombineEquivalentWorkResolver would search inside SparkWork, find and combine equivalent - * works. - */ -public class CombineEquivalentWorkResolver implements PhysicalPlanResolver { - protected static transient Logger LOG = LoggerFactory.getLogger(CombineEquivalentWorkResolver.class); - private List removedMapWorkNames = new ArrayList(); - private PhysicalContext pctx; - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - this.pctx = pctx; - List topNodes = new ArrayList(); - topNodes.addAll(pctx.getRootTasks()); - TaskGraphWalker taskWalker = new TaskGraphWalker(new EquivalentWorkMatcher()); - HashMap nodeOutput = Maps.newHashMap(); - taskWalker.startWalking(topNodes, nodeOutput); - return pctx; - } - - class EquivalentWorkMatcher implements Dispatcher { - private Comparator baseWorkComparator = new Comparator() { - @Override - public int compare(BaseWork o1, BaseWork o2) { - return o1.getName().compareTo(o2.getName()); - } - }; - - // maps from a work to the DPPs it contains - private Map> workToDpps = new HashMap<>(); - - @Override - public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws SemanticException { - if (nd instanceof SparkTask) { - SparkTask sparkTask = (SparkTask) nd; - SparkWork sparkWork = sparkTask.getWork(); - // For dpp case, dpp sink will appear in Task1 and the target work of dpp sink will appear in Task2. - // Task2 is the child task of Task1. Task2 will be traversed before task1 because TaskGraphWalker will first - // put children task in the front of task queue. - // If a spark work which is equal to other is found and removed in Task2, the dpp sink can be removed when Task1 - // is traversed(More detailed see HIVE-16948) - if (removedMapWorkNames.size() > 0) { - removeDynamicPartitionPruningSink(removedMapWorkNames, sparkWork); - if (sparkWork.getAllWork().size() == 0) { - removeEmptySparkTask(sparkTask); - } - } - - Set roots = sparkWork.getRoots(); - compareWorksRecursively(roots, sparkWork); - } - return null; - } - - private void compareWorksRecursively(Set works, SparkWork sparkWork) { - workToDpps.clear(); - // find out all equivalent works in the Set. - Set> equivalentWorks = compareChildWorks(works, sparkWork); - // combine equivalent work into single one in SparkWork's work graph. - Set removedWorks = combineEquivalentWorks(equivalentWorks, sparkWork); - - // try to combine next level works recursively. - for (BaseWork work : works) { - if (!removedWorks.contains(work)) { - Set children = Sets.newHashSet(); - children.addAll(sparkWork.getChildren(work)); - if (children.size() > 0) { - compareWorksRecursively(children, sparkWork); - } - } - } - } - - private Set> compareChildWorks(Set children, SparkWork sparkWork) { - Set> equivalentChildren = Sets.newHashSet(); - if (children.size() > 1) { - for (BaseWork work : children) { - boolean assigned = false; - for (Set set : equivalentChildren) { - if (belongToSet(set, work, sparkWork)) { - set.add(work); - assigned = true; - break; - } - } - if (!assigned) { - // sort the works so that we get consistent query plan for multi executions(for test verification). - Set newSet = Sets.newTreeSet(baseWorkComparator); - newSet.add(work); - equivalentChildren.add(newSet); - } - } - } - return equivalentChildren; - } - - private boolean belongToSet(Set set, BaseWork work, SparkWork sparkWork) { - if (set.isEmpty()) { - return true; - } else if (compareWork(set.iterator().next(), work, sparkWork)) { - return true; - } - return false; - } - - private Set combineEquivalentWorks(Set> equivalentWorks, SparkWork sparkWork) { - Set removedWorks = Sets.newHashSet(); - for (Set workSet : equivalentWorks) { - if (workSet.size() > 1) { - Iterator iterator = workSet.iterator(); - BaseWork first = iterator.next(); - List dppList1 = workToDpps.get(first); - while (iterator.hasNext()) { - BaseWork next = iterator.next(); - if (dppList1 != null) { - List dppList2 = workToDpps.get(next); - // equivalent works must have dpp lists of same size - for (int i = 0; i < dppList1.size(); i++) { - combineEquivalentDPPSinks(dppList1.get(i), dppList2.get(i)); - } - } - replaceWork(next, first, sparkWork); - removedWorks.add(next); - } - } - } - return removedWorks; - } - - private void replaceWork(BaseWork previous, BaseWork current, SparkWork sparkWork) { - updateReference(previous, current, sparkWork); - List parents = sparkWork.getParents(previous); - List children = sparkWork.getChildren(previous); - if (parents != null) { - for (BaseWork parent : parents) { - // we do not need to connect its parent to its counterpart, as they have the same parents. - sparkWork.disconnect(parent, previous); - } - } - if (children != null) { - for (BaseWork child : children) { - SparkEdgeProperty edgeProperty = sparkWork.getEdgeProperty(previous, child); - sparkWork.disconnect(previous, child); - sparkWork.connect(current, child, edgeProperty); - } - } - sparkWork.remove(previous); - // In order to fix HIVE-16948 - if (previous instanceof MapWork) { - removedMapWorkNames.add(previous.getName()); - } - } - - /* - * update the Work name which referred by Operators in following Works. - */ - private void updateReference(BaseWork previous, BaseWork current, SparkWork sparkWork) { - String previousName = previous.getName(); - String currentName = current.getName(); - List children = sparkWork.getAllWork(); - for (BaseWork child : children) { - Set> allOperators = child.getAllOperators(); - for (Operator operator : allOperators) { - if (operator instanceof MapJoinOperator) { - MapJoinDesc mapJoinDesc = ((MapJoinOperator) operator).getConf(); - Map parentToInput = mapJoinDesc.getParentToInput(); - for (Integer id : parentToInput.keySet()) { - String parent = parentToInput.get(id); - if (parent.equals(previousName)) { - parentToInput.put(id, currentName); - } - } - } - } - } - } - - private boolean compareWork(BaseWork first, BaseWork second, SparkWork sparkWork) { - - if (!first.getClass().getName().equals(second.getClass().getName())) { - return false; - } - - if (!hasSameParent(first, second, sparkWork)) { - return false; - } - - // leave work's output may be read in further SparkWork/FetchWork, we should not combine - // leave works without notifying further SparkWork/FetchWork. - if (sparkWork.getLeaves().contains(first) && sparkWork.getLeaves().contains(second)) { - Set> leafOps = first.getAllLeafOperators(); - leafOps.addAll(second.getAllLeafOperators()); - for (Operator operator : leafOps) { - // we know how to handle DPP sinks - if (!(operator instanceof SparkPartitionPruningSinkOperator)) { - return false; - } - } - } - - // need to check paths and partition desc for MapWorks - if (first instanceof MapWork && !compareMapWork((MapWork) first, (MapWork) second)) { - return false; - } - - Set> firstRootOperators = first.getAllRootOperators(); - Set> secondRootOperators = second.getAllRootOperators(); - if (firstRootOperators.size() != secondRootOperators.size()) { - return false; - } - - Iterator> firstIterator = firstRootOperators.iterator(); - Iterator> secondIterator = secondRootOperators.iterator(); - while (firstIterator.hasNext()) { - boolean result = compareOperatorChain(firstIterator.next(), secondIterator.next(), - first, second); - if (!result) { - return false; - } - } - - return true; - } - - private boolean compareMapWork(MapWork first, MapWork second) { - Map pathToPartition1 = first.getPathToPartitionInfo(); - Map pathToPartition2 = second.getPathToPartitionInfo(); - if (pathToPartition1.size() == pathToPartition2.size()) { - for (Map.Entry entry : pathToPartition1.entrySet()) { - Path path1 = entry.getKey(); - PartitionDesc partitionDesc1 = entry.getValue(); - PartitionDesc partitionDesc2 = pathToPartition2.get(path1); - if (!partitionDesc1.equals(partitionDesc2)) { - return false; - } - } - return true; - } - return false; - } - - private boolean hasSameParent(BaseWork first, BaseWork second, SparkWork sparkWork) { - boolean result = true; - List firstParents = sparkWork.getParents(first); - List secondParents = sparkWork.getParents(second); - if (firstParents.size() != secondParents.size()) { - result = false; - } - for (BaseWork parent : firstParents) { - if (!secondParents.contains(parent)) { - result = false; - break; - } - } - return result; - } - - private boolean compareOperatorChain(Operator firstOperator, Operator secondOperator, - BaseWork first, BaseWork second) { - boolean result = compareCurrentOperator(firstOperator, secondOperator); - if (!result) { - return false; - } - - List> firstOperatorChildOperators = firstOperator.getChildOperators(); - List> secondOperatorChildOperators = secondOperator.getChildOperators(); - if (firstOperatorChildOperators == null && secondOperatorChildOperators != null) { - return false; - } else if (firstOperatorChildOperators != null && secondOperatorChildOperators == null) { - return false; - } else if (firstOperatorChildOperators != null) { - if (firstOperatorChildOperators.size() != secondOperatorChildOperators.size()) { - return false; - } - int size = firstOperatorChildOperators.size(); - for (int i = 0; i < size; i++) { - result = compareOperatorChain(firstOperatorChildOperators.get(i), - secondOperatorChildOperators.get(i), first, second); - if (!result) { - return false; - } - } - } - - if (firstOperator instanceof SparkPartitionPruningSinkOperator) { - List dpps = workToDpps.computeIfAbsent( - first, k -> new ArrayList<>()); - dpps.add(((SparkPartitionPruningSinkOperator) firstOperator)); - dpps = workToDpps.computeIfAbsent(second, k -> new ArrayList<>()); - dpps.add(((SparkPartitionPruningSinkOperator) secondOperator)); - } - return true; - } - - /** - * Compare Operators through their Explain output string. - * - * @param firstOperator - * @param secondOperator - * @return - */ - private boolean compareCurrentOperator(Operator firstOperator, Operator secondOperator) { - return firstOperator.logicalEquals(secondOperator); - } - - /** - * traverse the children in sparkWork to find the dpp sink operator which target work is included in - * removedMapWorkList - * If there is branch, remove prune sink operator branch in the BaseWork - * If there is no branch, remove the whole BaseWork - * - * @param removedMapWorkList: the name of the map work has been deleted because they are equals to other works. - * @param sparkWork: current spark work - */ - private void removeDynamicPartitionPruningSink(List removedMapWorkList, SparkWork sparkWork) { - List allWorks = sparkWork.getAllWork(); - for (BaseWork baseWork : allWorks) { - Set> rootOperators = baseWork.getAllRootOperators(); - for (Operator root : rootOperators) { - List> pruningList = new ArrayList<>(); - SparkUtilities.collectOp(pruningList, root, SparkPartitionPruningSinkOperator.class); - for (Operator pruneSinkOp : pruningList) { - SparkPartitionPruningSinkOperator sparkPruneSinkOp = (SparkPartitionPruningSinkOperator) pruneSinkOp; - for (String removedName : removedMapWorkList) { - sparkPruneSinkOp.getConf().removeTarget(removedName); - } - if (sparkPruneSinkOp.getConf().getTargetInfos().isEmpty()) { - LOG.debug("ready to remove the sparkPruneSinkOp which target work is " + - sparkPruneSinkOp.getConf().getTargetWorks() + " because the MapWork is equals to other map work and " + - "has been deleted!"); - // If there is branch, remove prune sink operator branch in the baseWork - // If there is no branch, remove the whole baseWork - if (OperatorUtils.isInBranch(sparkPruneSinkOp)) { - OperatorUtils.removeBranch(sparkPruneSinkOp); - } else { - sparkWork.remove(baseWork); - } - } - } - } - } - } - - private void removeEmptySparkTask(SparkTask currTask) { - // If currTask is rootTasks, remove it and add its children to the rootTasks which currTask is its only parent - // task - if (pctx.getRootTasks().contains(currTask)) { - pctx.removeFromRootTask(currTask); - List> newRoots = currTask.getChildTasks(); - for (Task newRoot : newRoots) { - if (newRoot.getParentTasks().size() == 1) { - pctx.addToRootTask(newRoot); - } - } - } - SparkUtilities.removeEmptySparkTask(currTask); - } - } - - // Merge the target works of the second DPP sink into the first DPP sink. - public static void combineEquivalentDPPSinks(SparkPartitionPruningSinkOperator first, - SparkPartitionPruningSinkOperator second) { - SparkPartitionPruningSinkDesc firstConf = first.getConf(); - SparkPartitionPruningSinkDesc secondConf = second.getConf(); - for (SparkPartitionPruningSinkDesc.DPPTargetInfo targetInfo : secondConf.getTargetInfos()) { - MapWork target = targetInfo.work; - firstConf.addTarget(targetInfo.columnName, targetInfo.columnType, targetInfo.partKey, target, - targetInfo.tableScan); - - if (target != null) { - // update the target map work of the second - first.addAsSourceEvent(target, targetInfo.partKey, targetInfo.columnName, - targetInfo.columnType); - second.removeFromSourceEvent(target, targetInfo.partKey, targetInfo.columnName, - targetInfo.columnType); - target.setTmpPathForPartitionPruning(firstConf.getTmpPathOfTargetWork()); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java deleted file mode 100644 index eecb103de00..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SetSparkReducerParallelism.java +++ /dev/null @@ -1,281 +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.hadoop.hive.ql.optimizer.spark; - -import java.util.Collection; -import java.util.EnumSet; -import java.util.List; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.LimitOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.TerminalOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManager; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionManagerImpl; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.stats.StatsUtils; - -import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.UNIFORM; - -/** - * SetSparkReducerParallelism determines how many reducers should - * be run for a given reduce sink, clone from SetReducerParallelism. - */ -public class SetSparkReducerParallelism implements NodeProcessor { - - private static final Logger LOG = LoggerFactory.getLogger(SetSparkReducerParallelism.class.getName()); - - private static final String SPARK_DYNAMIC_ALLOCATION_ENABLED = "spark.dynamicAllocation.enabled"; - - // Spark memory per task, and total number of cores - private ObjectPair sparkMemoryAndCores; - private final boolean useOpStats; - - public SetSparkReducerParallelism(HiveConf conf) { - sparkMemoryAndCores = null; - useOpStats = conf.getBoolVar(HiveConf.ConfVars.SPARK_USE_OP_STATS); - } - - @Override - public Object process(Node nd, Stack stack, - NodeProcessorCtx procContext, Object... nodeOutputs) - throws SemanticException { - - OptimizeSparkProcContext context = (OptimizeSparkProcContext) procContext; - - ReduceSinkOperator sink = (ReduceSinkOperator) nd; - ReduceSinkDesc desc = sink.getConf(); - Set parentSinks = null; - - int maxReducers = context.getConf().getIntVar(HiveConf.ConfVars.MAXREDUCERS); - int constantReducers = context.getConf().getIntVar(HiveConf.ConfVars.HADOOPNUMREDUCERS); - - if (!useOpStats) { - parentSinks = OperatorUtils.findOperatorsUpstream(sink, ReduceSinkOperator.class); - parentSinks.remove(sink); - if (!context.getVisitedReduceSinks().containsAll(parentSinks)) { - // We haven't processed all the parent sinks, and we need - // them to be done in order to compute the parallelism for this sink. - // In this case, skip. We should visit this again from another path. - LOG.debug("Skipping sink " + sink + " for now as we haven't seen all its parents."); - return false; - } - } - - if (context.getVisitedReduceSinks().contains(sink)) { - // skip walking the children - LOG.debug("Already processed reduce sink: " + sink.getName()); - return true; - } - context.getVisitedReduceSinks().add(sink); - - if (needSetParallelism(sink, context.getConf())) { - if (constantReducers > 0) { - LOG.info("Parallelism for reduce sink " + sink + " set by user to " + constantReducers); - desc.setNumReducers(constantReducers); - } else { - //If it's a FileSink to bucketed files, use the bucket count as the reducer number - FileSinkOperator fso = GenSparkUtils.getChildOperator(sink, FileSinkOperator.class); - if (fso != null) { - String bucketCount = fso.getConf().getTableInfo().getProperties().getProperty( - hive_metastoreConstants.BUCKET_COUNT); - int numBuckets = bucketCount == null ? 0 : Integer.parseInt(bucketCount); - if (numBuckets > 0) { - LOG.info("Set parallelism for reduce sink " + sink + " to: " + numBuckets + " (buckets)"); - desc.setNumReducers(numBuckets); - return false; - } - } - - if (useOpStats || parentSinks.isEmpty()) { - long numberOfBytes = 0; - if (useOpStats) { - // we need to add up all the estimates from the siblings of this reduce sink - for (Operator sibling - : sink.getChildOperators().get(0).getParentOperators()) { - if (sibling.getStatistics() != null) { - numberOfBytes = StatsUtils.safeAdd(numberOfBytes, sibling.getStatistics().getDataSize()); - if (LOG.isDebugEnabled()) { - LOG.debug("Sibling " + sibling + " has stats: " + sibling.getStatistics()); - } - } else { - LOG.warn("No stats available from: " + sibling); - } - } - } else { - // Not using OP stats and this is the first sink in the path, meaning that - // we should use TS stats to infer parallelism - for (Operator sibling - : sink.getChildOperators().get(0).getParentOperators()) { - Set sources = - OperatorUtils.findOperatorsUpstream(sibling, TableScanOperator.class); - for (TableScanOperator source : sources) { - if (source.getStatistics() != null) { - numberOfBytes = StatsUtils.safeAdd(numberOfBytes, source.getStatistics().getDataSize()); - if (LOG.isDebugEnabled()) { - LOG.debug("Table source " + source + " has stats: " + source.getStatistics()); - } - } else { - LOG.warn("No stats available from table source: " + source); - } - } - } - LOG.debug("Gathered stats for sink " + sink + ". Total size is " - + numberOfBytes + " bytes."); - } - - // Divide it by 2 so that we can have more reducers - long bytesPerReducer = context.getConf().getLongVar(HiveConf.ConfVars.BYTESPERREDUCER) / 2; - int numReducers = Utilities.estimateReducers(numberOfBytes, bytesPerReducer, - maxReducers, false); - - getSparkMemoryAndCores(context); - if (sparkMemoryAndCores != null && - sparkMemoryAndCores.getFirst() > 0 && sparkMemoryAndCores.getSecond() > 0) { - // warn the user if bytes per reducer is much larger than memory per task - if ((double) sparkMemoryAndCores.getFirst() / bytesPerReducer < 0.5) { - LOG.warn("Average load of a reducer is much larger than its available memory. " + - "Consider decreasing hive.exec.reducers.bytes.per.reducer"); - } - - // If there are more cores, use the number of cores - numReducers = Math.max(numReducers, sparkMemoryAndCores.getSecond()); - } - numReducers = Math.min(numReducers, maxReducers); - LOG.info("Set parallelism for reduce sink " + sink + " to: " + numReducers + - " (calculated)"); - desc.setNumReducers(numReducers); - } else { - // Use the maximum parallelism from all parent reduce sinks - int numberOfReducers = 0; - for (ReduceSinkOperator parent : parentSinks) { - numberOfReducers = Math.max(numberOfReducers, parent.getConf().getNumReducers()); - } - desc.setNumReducers(numberOfReducers); - LOG.debug("Set parallelism for sink " + sink + " to " + numberOfReducers - + " based on its parents"); - } - final Collection keyCols = - ExprNodeDesc.ExprNodeDescEqualityWrapper.transform(desc.getKeyCols()); - final Collection partCols = - ExprNodeDesc.ExprNodeDescEqualityWrapper.transform(desc.getPartitionCols()); - if (keyCols != null && keyCols.equals(partCols)) { - desc.setReducerTraits(EnumSet.of(UNIFORM)); - } - } - } else { - LOG.info("Number of reducers for sink " + sink + " was already determined to be: " + desc.getNumReducers()); - } - - return false; - } - - // tests whether the RS needs automatic setting parallelism - private boolean needSetParallelism(ReduceSinkOperator reduceSink, HiveConf hiveConf) { - ReduceSinkDesc desc = reduceSink.getConf(); - if (desc.getNumReducers() <= 0) { - return true; - } - if (desc.getNumReducers() == 1 && desc.hasOrderBy() && - hiveConf.getBoolVar(HiveConf.ConfVars.HIVESAMPLINGFORORDERBY) && !desc.isDeduplicated()) { - Stack> descendants = new Stack>(); - List> children = reduceSink.getChildOperators(); - if (children != null) { - for (Operator child : children) { - descendants.push(child); - } - } - while (descendants.size() != 0) { - Operator descendant = descendants.pop(); - //If the decendants contains LimitOperator,return false - if (descendant instanceof LimitOperator) { - return false; - } - boolean reachTerminalOperator = (descendant instanceof TerminalOperator); - if (!reachTerminalOperator) { - List> childrenOfDescendant = descendant.getChildOperators(); - if (childrenOfDescendant != null) { - for (Operator childOfDescendant : childrenOfDescendant) { - descendants.push(childOfDescendant); - } - } - } - } - return true; - } - return false; - - } - - private void getSparkMemoryAndCores(OptimizeSparkProcContext context) throws SemanticException { - if (sparkMemoryAndCores != null) { - return; - } - if (context.getConf().getBoolean(SPARK_DYNAMIC_ALLOCATION_ENABLED, false)) { - // If dynamic allocation is enabled, numbers for memory and cores are meaningless. So, we don't - // try to get it. - sparkMemoryAndCores = null; - return; - } - - SparkSessionManager sparkSessionManager = null; - SparkSession sparkSession = null; - try { - sparkSessionManager = SparkSessionManagerImpl.getInstance(); - sparkSession = SparkUtilities.getSparkSession( - context.getConf(), sparkSessionManager); - sparkMemoryAndCores = sparkSession.getMemoryAndCores(); - } catch (HiveException e) { - throw new SemanticException("Failed to get a spark session: " + e); - } catch (Exception e) { - LOG.warn("Failed to get spark memory/core info", e); - } finally { - if (sparkSession != null && sparkSessionManager != null) { - try { - sparkSessionManager.returnSession(sparkSession); - } catch (HiveException ex) { - LOG.error("Failed to return the session to SessionManager: " + ex, ex); - } - } - } - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java deleted file mode 100644 index 884e693d5f7..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinHintOptimizer.java +++ /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. - */ - -package org.apache.hadoop.hive.ql.optimizer.spark; - -import java.util.Stack; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.BucketJoinProcCtx; -import org.apache.hadoop.hive.ql.optimizer.BucketMapjoinProc; -import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; - -/** - * This processes joins in which user specified a hint to identify the small-table. - * Currently it takes a mapjoin already converted from hints, and converts it further - * to BucketMapJoin or SMBMapJoin using same small-table identification. - * - * The idea is eventually to process even hinted Mapjoin hints here, - * but due to code complexity in refactoring, that is still in Optimizer. - */ -public class SparkJoinHintOptimizer implements NodeProcessor { - - private BucketMapjoinProc bucketMapJoinOptimizer; - private SparkSMBJoinHintOptimizer smbMapJoinOptimizer; - - public SparkJoinHintOptimizer(ParseContext parseCtx) { - bucketMapJoinOptimizer = new BucketMapjoinProc(parseCtx); - smbMapJoinOptimizer = new SparkSMBJoinHintOptimizer(parseCtx); - } - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx; - HiveConf hiveConf = context.getParseContext().getConf(); - - // Convert from mapjoin to bucket map join if enabled. - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN) - || hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) { - BucketJoinProcCtx bjProcCtx = new BucketJoinProcCtx(hiveConf); - bucketMapJoinOptimizer.process(nd, stack, bjProcCtx, nodeOutputs); - } - - // Convert from bucket map join to sort merge bucket map join if enabled. - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVEOPTSORTMERGEBUCKETMAPJOIN)) { - SortBucketJoinProcCtx smbJoinCtx = new SortBucketJoinProcCtx(hiveConf); - smbMapJoinOptimizer.process(nd, stack, smbJoinCtx, nodeOutputs); - } - return null; - } -} \ No newline at end of file diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java deleted file mode 100644 index c52aceafc91..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkJoinOptimizer.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.optimizer.spark; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; - -import java.util.Stack; - -/** - * Converts a join to a more optimized join for the Spark path. - * Delegates to a more specialized join processor. - */ -public class SparkJoinOptimizer implements NodeProcessor { - - private SparkSortMergeJoinOptimizer smbJoinOptimizer; - private SparkMapJoinOptimizer mapJoinOptimizer; - - public SparkJoinOptimizer(ParseContext procCtx) { - smbJoinOptimizer = new SparkSortMergeJoinOptimizer(procCtx); - mapJoinOptimizer = new SparkMapJoinOptimizer(); - } - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) throws SemanticException { - OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx; - HiveConf conf = context.getConf(); - - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN_TOMAPJOIN)) { - Object mapJoinOp = mapJoinOptimizer.process(nd, stack, procCtx, nodeOutputs); - if (mapJoinOp == null) { - smbJoinOptimizer.process(nd, stack, procCtx, nodeOutputs); - } - } else { - Object sortMergeJoinOp = smbJoinOptimizer.process(nd, stack, procCtx, nodeOutputs); - if (sortMergeJoinOp == null) { - mapJoinOptimizer.process(nd, stack, procCtx, nodeOutputs); - } - } - return null; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java deleted file mode 100644 index 8e75db9e08d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java +++ /dev/null @@ -1,554 +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.hadoop.hive.ql.optimizer.spark; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - -import com.google.common.collect.Sets; -import org.apache.hadoop.hive.ql.exec.LateralViewForwardOperator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.TerminalOperator; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.GroupByOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.MuxOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.BucketMapjoinProc; -import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.OpTraits; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.Statistics; -import org.apache.hadoop.hive.serde.serdeConstants; -import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe; - -/** - * SparkMapJoinOptimizer cloned from ConvertJoinMapJoin is an optimization that replaces a common join - * (aka shuffle join) with a map join (aka broadcast or fragment replicate - * join when possible. Map joins have restrictions on which joins can be - * converted (e.g.: full outer joins cannot be handled as map joins) as well - * as memory restrictions (one side of the join has to fit into memory). - */ -public class SparkMapJoinOptimizer implements NodeProcessor { - - private static final Logger LOG = LoggerFactory.getLogger(SparkMapJoinOptimizer.class.getName()); - - @Override - /** - * We should ideally not modify the tree we traverse. However, - * since we need to walk the tree at any time when we modify the operator, we - * might as well do it here. - */ - public Object - process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) - throws SemanticException { - - OptimizeSparkProcContext context = (OptimizeSparkProcContext) procCtx; - HiveConf conf = context.getConf(); - JoinOperator joinOp = (JoinOperator) nd; - - if (!conf.getBoolVar(HiveConf.ConfVars.HIVECONVERTJOIN)) { - return null; - } - - LOG.info("Check if operator " + joinOp + " can be converted to map join"); - long[] mapJoinInfo = getMapJoinConversionInfo(joinOp, context); - int mapJoinConversionPos = (int) mapJoinInfo[0]; - - if (mapJoinConversionPos < 0) { - return null; - } - - int numBuckets = -1; - List> bucketColNames = null; - - LOG.info("Convert to non-bucketed map join"); - MapJoinOperator mapJoinOp = convertJoinMapJoin(joinOp, context, mapJoinConversionPos); - // For native vectorized map join, we require the key SerDe to be BinarySortableSerDe - // Note: the MJ may not really get natively-vectorized later, - // but changing SerDe won't hurt correctness - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_MAPJOIN_NATIVE_ENABLED) && - conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED)) { - mapJoinOp.getConf().getKeyTblDesc().getProperties().setProperty( - serdeConstants.SERIALIZATION_LIB, BinarySortableSerDe.class.getName()); - } - if (conf.getBoolVar(HiveConf.ConfVars.HIVEOPTBUCKETMAPJOIN)) { - LOG.info("Check if it can be converted to bucketed map join"); - numBuckets = convertJoinBucketMapJoin(joinOp, mapJoinOp, - context, mapJoinConversionPos); - if (numBuckets > 1) { - LOG.info("Converted to map join with " + numBuckets + " buckets"); - bucketColNames = joinOp.getOpTraits().getBucketColNames(); - mapJoinInfo[2] /= numBuckets; - } else { - LOG.info("Can not convert to bucketed map join"); - } - } - - // we can set the traits for this join operator - OpTraits opTraits = new OpTraits(bucketColNames, numBuckets, null, - joinOp.getOpTraits().getNumReduceSinks(), joinOp.getOpTraits().getBucketingVersion()); - mapJoinOp.setOpTraits(opTraits); - mapJoinOp.setStatistics(joinOp.getStatistics()); - setNumberOfBucketsOnChildren(mapJoinOp); - - context.getMjOpSizes().put(mapJoinOp, mapJoinInfo[1] + mapJoinInfo[2]); - - return mapJoinOp; - } - - private void setNumberOfBucketsOnChildren(Operator currentOp) { - int numBuckets = currentOp.getOpTraits().getNumBuckets(); - for (Operator op : currentOp.getChildOperators()) { - if (!(op instanceof ReduceSinkOperator) && !(op instanceof GroupByOperator)) { - op.getOpTraits().setNumBuckets(numBuckets); - if (numBuckets < 0) { - op.getOpTraits().setBucketColNames(null); - } - setNumberOfBucketsOnChildren(op); - } - } - } - - private int convertJoinBucketMapJoin(JoinOperator joinOp, MapJoinOperator mapJoinOp, - OptimizeSparkProcContext context, int bigTablePosition) throws SemanticException { - ParseContext parseContext = context.getParseContext(); - List joinAliases = new ArrayList(); - String baseBigAlias = null; - Map> posToAliasMap = joinOp.getPosToAliasMap(); - for (Map.Entry> entry: posToAliasMap.entrySet()) { - if (entry.getKey().intValue() == bigTablePosition) { - baseBigAlias = entry.getValue().iterator().next(); - } - for (String alias: entry.getValue()) { - if (!joinAliases.contains(alias)) { - joinAliases.add(alias); - } - } - } - mapJoinOp.setPosToAliasMap(posToAliasMap); - BucketMapjoinProc.checkAndConvertBucketMapJoin( - parseContext, - mapJoinOp, - baseBigAlias, - joinAliases); - MapJoinDesc joinDesc = mapJoinOp.getConf(); - return joinDesc.isBucketMapJoin() - ? joinDesc.getBigTableBucketNumMapping().size() : -1; - } - - /** - * This method returns the big table position in a map-join. If the given join - * cannot be converted to a map-join (This could happen for several reasons - one - * of them being presence of 2 or more big tables that cannot fit in-memory), it returns -1. - * - * Otherwise, it returns an int value that is the index of the big table in the set - * MapJoinProcessor.bigTableCandidateSet - * - * @param joinOp - * @param context - * @return an array of 3 long values, first value is the position, - * second value is the connected map join size, and the third is big table data size. - */ - private long[] getMapJoinConversionInfo( - JoinOperator joinOp, OptimizeSparkProcContext context) { - Set bigTableCandidateSet = - MapJoinProcessor.getBigTableCandidates(joinOp.getConf().getConds()); - - long maxSize = context.getConf().getLongVar( - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); - - int bigTablePosition = -1; - - Statistics bigInputStat = null; - long totalSize = 0; - int pos = 0; - - // bigTableFound means we've encountered a table that's bigger than the - // max. This table is either the big table or we cannot convert. - boolean bigTableFound = false; - boolean useTsStats = context.getConf().getBoolean(HiveConf.ConfVars.SPARK_USE_TS_STATS_FOR_MAPJOIN.varname, false); - - // If we're using TS's stats for mapjoin optimization, check each branch and see if there's any - // upstream operator (e.g., JOIN, LATERAL_VIEW) that can increase output data size. - // If so, mark that branch as the big table branch. - if (useTsStats) { - LOG.debug("Checking map join optimization for operator {} using TS stats", joinOp); - for (Operator parentOp : joinOp.getParentOperators()) { - if (isBigTableBranch(parentOp)) { - if (bigTablePosition < 0 && bigTableCandidateSet.contains(pos) - && !containUnionWithoutRS(parentOp.getParentOperators().get(0))) { - LOG.debug("Found a big table branch with parent operator {} and position {}", parentOp, pos); - bigTablePosition = pos; - bigTableFound = true; - bigInputStat = new Statistics(0, Long.MAX_VALUE); - } else { - // Either we've found multiple big table branches, or the current branch cannot - // be a big table branch. Disable mapjoin for these cases. - LOG.debug("Cannot enable map join optimization for operator {}", joinOp); - return new long[]{-1, 0, 0}; - } - } - pos++; - } - } - - pos = 0; - - for (Operator parentOp : joinOp.getParentOperators()) { - // Skip the potential big table identified above - if (pos == bigTablePosition) { - pos++; - continue; - } - - Statistics currInputStat = null; - if (useTsStats) { - // Find all root TSs and add up all data sizes - // Not adding other stats (e.g., # of rows, col stats) since only data size is used here - for (TableScanOperator root : OperatorUtils.findOperatorsUpstream(parentOp, TableScanOperator.class)) { - if (currInputStat == null) { - currInputStat = root.getStatistics().clone(); - } else { - currInputStat.addBasicStats(root.getStatistics()); - } - } - } else { - currInputStat = parentOp.getStatistics(); - } - - if (currInputStat == null) { - LOG.warn("Couldn't get statistics from: " + parentOp); - return new long[]{-1, 0, 0}; - } - - // Union is hard to handle. For instance, the following case: - // TS TS - // | | - // FIL FIL - // | | - // SEL SEL - // \ / - // UNION - // | - // RS - // | - // JOIN - // If we treat this as a MJ case, then after the RS is removed, we would - // create two MapWorks, for each of the TS. Each of these MapWork will contain - // a MJ operator, which is wrong. - // Otherwise, we could try to break the op tree at the UNION, and create two MapWorks - // for the branches above. Then, MJ will be in the following ReduceWork. - // But, this is tricky to implement, and we'll leave it as a future work for now. - if (containUnionWithoutRS(parentOp.getParentOperators().get(0))) { - return new long[]{-1, 0, 0}; - } - - long inputSize = currInputStat.getDataSize(); - - if (bigInputStat == null || inputSize > bigInputStat.getDataSize()) { - - if (bigTableFound) { - // cannot convert to map join; we've already chosen a big table - // on size and there's another one that's bigger. - return new long[]{-1, 0, 0}; - } - - if (inputSize > maxSize) { - if (!bigTableCandidateSet.contains(pos)) { - // can't use the current table as the big table, but it's too - // big for the map side. - return new long[]{-1, 0, 0}; - } - - bigTableFound = true; - } - - if (bigInputStat != null) { - // we're replacing the current big table with a new one. Need - // to count the current one as a map table then. - totalSize += bigInputStat.getDataSize(); - } - - if (totalSize > maxSize) { - // sum of small tables size in this join exceeds configured limit - // hence cannot convert. - return new long[]{-1, 0, 0}; - } - - if (bigTableCandidateSet.contains(pos)) { - bigTablePosition = pos; - bigInputStat = currInputStat; - } - } else { - totalSize += currInputStat.getDataSize(); - if (totalSize > maxSize) { - // cannot hold all map tables in memory. Cannot convert. - return new long[]{-1, 0, 0}; - } - } - pos++; - } - - if (bigTablePosition == -1) { - //No big table candidates. - return new long[]{-1, 0, 0}; - } - - //Final check, find size of already-calculated Mapjoin Operators in same work (spark-stage). - //We need to factor this in to prevent overwhelming Spark executor-memory. - long connectedMapJoinSize = getConnectedMapJoinSize(joinOp.getParentOperators(). - get(bigTablePosition), joinOp, context); - if ((connectedMapJoinSize + totalSize) > maxSize) { - return new long[]{-1, 0, 0}; - } - - return new long[]{bigTablePosition, connectedMapJoinSize, totalSize}; - } - - /** - * Check whether the branch starting from 'op' is a potential big table branch. - * This is true if the branch contains any operator that could potentially increase - * output data size, such as JOIN and LATERAL_VIEW. If this is the case, we assume - * the worst and mark the branch as big table branch in the MapJoin optimization. - * - * @return True if the branch starting at 'op' is a big table branch. False otherwise. - */ - private boolean isBigTableBranch(Operator op) { - for (Class> clazz : - Sets.newHashSet(JoinOperator.class, LateralViewForwardOperator.class)) { - Set> parentSinks = OperatorUtils.findOperatorsUpstream(op, clazz); - if (!parentSinks.isEmpty()) { - return true; - } - } - return false; - } - - /** - * Examines this operator and all the connected operators, for mapjoins that will be in the same work. - * @param parentOp potential big-table parent operator, explore up from this. - * @param joinOp potential mapjoin operator, explore down from this. - * @param ctx context to pass information. - * @return total size of parent mapjoins in same work as this operator. - */ - @SuppressWarnings({"rawtypes", "unchecked"}) - private long getConnectedMapJoinSize(Operator parentOp, Operator joinOp, - OptimizeSparkProcContext ctx) { - long result = 0; - for (Operator grandParentOp : parentOp.getParentOperators()) { - result += getConnectedParentMapJoinSize(grandParentOp, ctx); - } - result += getConnectedChildMapJoinSize(joinOp, ctx); - return result; - } - - /** - * Examines this operator and all the parents, for mapjoins that will be in the same work. - * @param op given operator - * @param ctx context to pass information. - * @return total size of parent mapjoins in same work as this operator. - */ - private long getConnectedParentMapJoinSize(Operator op, OptimizeSparkProcContext ctx) { - if ((op instanceof UnionOperator) || (op instanceof ReduceSinkOperator)) { - //Work Boundary, stop exploring. - return 0; - } - - if (op instanceof MapJoinOperator) { - //found parent mapjoin operator. Its size should already reflect any other mapjoins connected to it. - long mjSize = ctx.getMjOpSizes().get(op); - return mjSize; - } - - long result = 0; - for (Operator parentOp : op.getParentOperators()) { - //Else, recurse up the parents. - result += getConnectedParentMapJoinSize(parentOp, ctx); - } - return result; - } - - /** - * Examines this operator and all the children, for mapjoins that will be in the same work. - * @param op given operator - * @param ctx context to pass information. - * @return total size of child mapjoins in same work as this operator. - */ - private long getConnectedChildMapJoinSize(Operator op, OptimizeSparkProcContext ctx) { - if ((op instanceof UnionOperator) || (op instanceof ReduceSinkOperator)) { - //Work Boundary, stop exploring. - return 0; - } - - if (op instanceof MapJoinOperator) { - //Found child mapjoin operator. - //Its size should already reflect any mapjoins connected to it, so stop processing. - long mjSize = ctx.getMjOpSizes().get(op); - return mjSize; - } - - long result = 0; - for (Operator childOp : op.getChildOperators()) { - //Else, recurse to the children. - result += getConnectedChildMapJoinSize(childOp, ctx); - } - return result; - } - - /* - * Once we have decided on the map join, the tree would transform from - * - * | | - * Join MapJoin - * / \ / \ - * RS RS ---> RS TS (big table) - * / \ / - * TS TS TS (small table) - * - * for spark. - */ - - public MapJoinOperator convertJoinMapJoin(JoinOperator joinOp, OptimizeSparkProcContext context, - int bigTablePosition) throws SemanticException { - // bail on mux operator because currently the mux operator masks the emit keys - // of the constituent reduce sinks. - for (Operator parentOp : joinOp.getParentOperators()) { - if (parentOp instanceof MuxOperator) { - return null; - } - } - - //can safely convert the join to a map join. - MapJoinOperator mapJoinOp = - MapJoinProcessor.convertJoinOpMapJoinOp(context.getConf(), joinOp, - joinOp.getConf().isLeftInputJoin(), joinOp.getConf().getBaseSrc(), - joinOp.getConf().getMapAliases(), bigTablePosition, true); - - Operator parentBigTableOp = - mapJoinOp.getParentOperators().get(bigTablePosition); - if (parentBigTableOp instanceof ReduceSinkOperator) { - - for (Operator parentOp : parentBigTableOp.getParentOperators()) { - // we might have generated a dynamic partition operator chain. Since - // we're removing the reduce sink we need do remove that too. - Set partitionPruningSinkOps = new HashSet<>(); - for (Operator childOp : parentOp.getChildOperators()) { - SparkPartitionPruningSinkOperator partitionPruningSinkOp = findPartitionPruningSinkOperator(childOp); - if (partitionPruningSinkOp != null) { - partitionPruningSinkOps.add(partitionPruningSinkOp); - } - } - - for (SparkPartitionPruningSinkOperator partitionPruningSinkOp : partitionPruningSinkOps) { - OperatorUtils.removeBranch(partitionPruningSinkOp); - // at this point we've found the fork in the op pipeline that has the pruning as a child plan. - LOG.info("Disabling dynamic pruning for: " - + (partitionPruningSinkOp.getConf()).getTableScanNames() - + ". Need to be removed together with reduce sink"); - } - } - - mapJoinOp.getParentOperators().remove(bigTablePosition); - if (!(mapJoinOp.getParentOperators().contains(parentBigTableOp.getParentOperators().get(0)))) { - mapJoinOp.getParentOperators().add(bigTablePosition, - parentBigTableOp.getParentOperators().get(0)); - } - parentBigTableOp.getParentOperators().get(0).removeChild(parentBigTableOp); - for (Operator op : mapJoinOp.getParentOperators()) { - if (!(op.getChildOperators().contains(mapJoinOp))) { - op.getChildOperators().add(mapJoinOp); - } - op.getChildOperators().remove(joinOp); - } - } - - // Data structures - mapJoinOp.getConf().setQBJoinTreeProps(joinOp.getConf()); - - return mapJoinOp; - } - - private SparkPartitionPruningSinkOperator findPartitionPruningSinkOperator(Operator parent) { - - for (Operator op : parent.getChildOperators()) { - while (op != null) { - if (op instanceof SparkPartitionPruningSinkOperator && op.getConf() instanceof SparkPartitionPruningSinkDesc) { - // found dynamic partition pruning operator - return (SparkPartitionPruningSinkOperator) op; - } - if (op instanceof TerminalOperator) { - // crossing reduce sink or file sink means the pruning isn't for this parent. - break; - } - - if (op.getChildOperators().size() != 1) { - // dynamic partition pruning pipeline doesn't have multiple children - break; - } - - op = op.getChildOperators().get(0); - } - } - - return null; - } - - - private boolean containUnionWithoutRS(Operator op) { - boolean result = false; - if (op instanceof UnionOperator) { - for (Operator pop : op.getParentOperators()) { - if (!(pop instanceof ReduceSinkOperator)) { - result = true; - break; - } - } - } else if (op instanceof ReduceSinkOperator) { - result = false; - } else { - for (Operator pop : op.getParentOperators()) { - if (containUnionWithoutRS(pop)) { - result = true; - break; - } - } - } - return result; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java deleted file mode 100644 index 1607a3f6951..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java +++ /dev/null @@ -1,144 +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.hadoop.hive.ql.optimizer.spark; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.optimizer.signature.Signature; -import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc; -import org.apache.hadoop.hive.ql.plan.Explain; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - -@Explain(displayName = "Spark Partition Pruning Sink Operator") -public class SparkPartitionPruningSinkDesc extends AbstractOperatorDesc { - - public static class DPPTargetInfo { - // column in the target table that will be pruned against - public String columnName; - // type of target column - public String columnType; - // the partition column we're interested in - public ExprNodeDesc partKey; - public MapWork work; - public transient TableScanOperator tableScan; - - DPPTargetInfo(String columnName, String columnType, ExprNodeDesc partKey, MapWork work, - TableScanOperator tableScan) { - this.columnName = columnName; - this.columnType = columnType; - this.partKey = partKey; - this.work = work; - this.tableScan = tableScan; - } - } - - private List targetInfos = new ArrayList<>(); - - private TableDesc table; - - private Path path; - - public List getTargetInfos() { - return targetInfos; - } - - public void addTarget(String colName, String colType, ExprNodeDesc partKey, MapWork mapWork, - TableScanOperator tableScan) { - targetInfos.add(new DPPTargetInfo(colName, colType, partKey, mapWork, tableScan)); - } - - public Path getTmpPathOfTargetWork() { - return targetInfos.isEmpty() ? null : targetInfos.get(0).work.getTmpPathForPartitionPruning(); - } - - @Explain(displayName = "tmp Path", explainLevels = {Explain.Level.EXTENDED}) - public Path getPath() { - return path; - } - - public void setPath(Path path) { - this.path = path; - } - - public String getTargetWorks() { - return Arrays.toString(targetInfos.stream().map(info -> info.work.getName()).toArray()); - } - - public String getTableScanNames() { - return Arrays.toString(targetInfos.stream().map(info -> info.tableScan.getName()).toArray()); - } - - @Signature - public TableDesc getTable() { - return table; - } - - public void setTable(TableDesc table) { - this.table = table; - } - - @Explain(displayName = "Target Columns") - public String displayTargetColumns() { - // The target column list has the format "TargetWork -> [colName:colType(expression), ...], ..." - Map> map = new TreeMap<>(); - for (DPPTargetInfo info : targetInfos) { - List columns = map.computeIfAbsent(info.work.getName(), v -> new ArrayList<>()); - String name = info.columnName.substring(info.columnName.indexOf(':') + 1); - columns.add(name + ":" + info.columnType + " (" + info.partKey.getExprString() + ")"); - } - StringBuilder builder = new StringBuilder(); - builder.append("["); - for (String work : map.keySet()) { - if (builder.length() > 1) { - builder.append(", "); - } - builder.append(work).append(" -> ").append(map.get(work)); - } - builder.append("]"); - return builder.toString(); - } - - @Override - public boolean isSame(OperatorDesc other) { - if (getClass().getName().equals(other.getClass().getName())) { - SparkPartitionPruningSinkDesc otherDesc = (SparkPartitionPruningSinkDesc) other; - return getTable().equals(otherDesc.getTable()); - } - return false; - } - - public void removeTarget(String name) { - List toRemove = new ArrayList<>(); - for (DPPTargetInfo targetInfo : targetInfos) { - if (targetInfo.work.getName().equals(name)) { - toRemove.add(targetInfo); - } - } - targetInfos.removeAll(toRemove); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java deleted file mode 100644 index ac5caa6135a..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java +++ /dev/null @@ -1,293 +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.hadoop.hive.ql.optimizer.spark; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.GroupByOperator; -import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorFactory; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.RowSchema; -import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; -import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.GraphWalker; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.lib.Rule; -import org.apache.hadoop.hive.ql.lib.RuleRegExp; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkProcContext; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc; -import org.apache.hadoop.hive.ql.plan.MapJoinDesc; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.PlanUtils; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; - -public class SparkReduceSinkMapJoinProc implements NodeProcessor { - - public static final Logger LOG = LoggerFactory.getLogger(SparkReduceSinkMapJoinProc.class.getName()); - - public static class SparkMapJoinFollowedByGroupByProcessor implements NodeProcessor { - private boolean hasGroupBy = false; - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procCtx; - hasGroupBy = true; - GroupByOperator op = (GroupByOperator) nd; - float groupByMemoryUsage = context.conf.getFloatVar( - HiveConf.ConfVars.HIVEMAPJOINFOLLOWEDBYMAPAGGRHASHMEMORY); - op.getConf().setGroupByMemoryUsage(groupByMemoryUsage); - return null; - } - - public boolean getHasGroupBy() { - return hasGroupBy; - } - } - - private boolean hasGroupBy(Operator mapjoinOp, - GenSparkProcContext context) throws SemanticException { - List> childOps = mapjoinOp.getChildOperators(); - Map rules = new LinkedHashMap(); - SparkMapJoinFollowedByGroupByProcessor processor = new SparkMapJoinFollowedByGroupByProcessor(); - rules.put(new RuleRegExp("GBY", GroupByOperator.getOperatorName() + "%"), processor); - Dispatcher disp = new DefaultRuleDispatcher(null, rules, context); - GraphWalker ogw = new DefaultGraphWalker(disp); - ArrayList topNodes = new ArrayList(); - topNodes.addAll(childOps); - ogw.startWalking(topNodes, null); - return processor.getHasGroupBy(); - } - - /* (non-Javadoc) - * This processor addresses the RS-MJ case that occurs in spark on the small/hash - * table side of things. The work that RS will be a part of must be connected - * to the MJ work via be a broadcast edge. - * We should not walk down the tree when we encounter this pattern because: - * the type of work (map work or reduce work) needs to be determined - * on the basis of the big table side because it may be a mapwork (no need for shuffle) - * or reduce work. - */ - @SuppressWarnings("unchecked") - @Override - public Object process(Node nd, Stack stack, - NodeProcessorCtx procContext, Object... nodeOutputs) - throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procContext; - - if (!nd.getClass().equals(MapJoinOperator.class)) { - return null; - } - - MapJoinOperator mapJoinOp = (MapJoinOperator)nd; - - if (stack.size() < 2 || !(stack.get(stack.size() - 2) instanceof ReduceSinkOperator)) { - context.currentMapJoinOperators.add(mapJoinOp); - return null; - } - - context.preceedingWork = null; - context.currentRootOperator = null; - - ReduceSinkOperator parentRS = (ReduceSinkOperator)stack.get(stack.size() - 2); - // remove the tag for in-memory side of mapjoin - parentRS.getConf().setSkipTag(true); - parentRS.setSkipTag(true); - // remember the original parent list before we start modifying it. - if (!context.mapJoinParentMap.containsKey(mapJoinOp)) { - List> parents = new ArrayList>(mapJoinOp.getParentOperators()); - context.mapJoinParentMap.put(mapJoinOp, parents); - } - - List mapJoinWork; - - /* - * If there was a pre-existing work generated for the big-table mapjoin side, - * we need to hook the work generated for the RS (associated with the RS-MJ pattern) - * with the pre-existing work. - * - * Otherwise, we need to associate that the mapjoin op - * to be linked to the RS work (associated with the RS-MJ pattern). - * - */ - mapJoinWork = context.mapJoinWorkMap.get(mapJoinOp); - int workMapSize = context.childToWorkMap.get(parentRS).size(); - Preconditions.checkArgument(workMapSize == 1, - "AssertionError: expected context.childToWorkMap.get(parentRS).size() to be 1, but was " + workMapSize); - BaseWork parentWork = context.childToWorkMap.get(parentRS).get(0); - - // set the link between mapjoin and parent vertex - int pos = context.mapJoinParentMap.get(mapJoinOp).indexOf(parentRS); - if (pos == -1) { - throw new SemanticException("Cannot find position of parent in mapjoin"); - } - LOG.debug("Mapjoin "+mapJoinOp+", pos: "+pos+" --> "+parentWork.getName()); - mapJoinOp.getConf().getParentToInput().put(pos, parentWork.getName()); - - SparkEdgeProperty edgeProp = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE); - - if (mapJoinWork != null) { - for (BaseWork myWork: mapJoinWork) { - // link the work with the work associated with the reduce sink that triggered this rule - SparkWork sparkWork = context.currentTask.getWork(); - LOG.debug("connecting "+parentWork.getName()+" with "+myWork.getName()); - sparkWork.connect(parentWork, myWork, edgeProp); - } - } - - // remember in case we need to connect additional work later - Map linkWorkMap = null; - if (context.linkOpWithWorkMap.containsKey(mapJoinOp)) { - linkWorkMap = context.linkOpWithWorkMap.get(mapJoinOp); - } else { - linkWorkMap = new HashMap(); - } - linkWorkMap.put(parentWork, edgeProp); - context.linkOpWithWorkMap.put(mapJoinOp, linkWorkMap); - - List reduceSinks - = context.linkWorkWithReduceSinkMap.get(parentWork); - if (reduceSinks == null) { - reduceSinks = new ArrayList(); - } - reduceSinks.add(parentRS); - context.linkWorkWithReduceSinkMap.put(parentWork, reduceSinks); - - // create the dummy operators - List> dummyOperators = new ArrayList>(); - - // create an new operator: HashTableDummyOperator, which share the table desc - HashTableDummyDesc desc = new HashTableDummyDesc(); - HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get( - mapJoinOp.getCompilationOpContext(), desc); - TableDesc tbl; - - // need to create the correct table descriptor for key/value - RowSchema rowSchema = parentRS.getParentOperators().get(0).getSchema(); - tbl = PlanUtils.getReduceValueTableDesc(PlanUtils.getFieldSchemasFromRowSchema(rowSchema, "")); - dummyOp.getConf().setTbl(tbl); - - Map> keyExprMap = mapJoinOp.getConf().getKeys(); - List keyCols = keyExprMap.get(Byte.valueOf((byte) 0)); - StringBuilder keyOrder = new StringBuilder(); - StringBuilder keyNullOrder = new StringBuilder(); - for (int i = 0; i < keyCols.size(); i++) { - keyOrder.append("+"); - keyNullOrder.append("a"); - } - TableDesc keyTableDesc = PlanUtils.getReduceKeyTableDesc(PlanUtils - .getFieldSchemasFromColumnList(keyCols, "mapjoinkey"), keyOrder.toString(), - keyNullOrder.toString()); - mapJoinOp.getConf().setKeyTableDesc(keyTableDesc); - - // let the dummy op be the parent of mapjoin op - mapJoinOp.replaceParent(parentRS, dummyOp); - List> dummyChildren = - new ArrayList>(); - dummyChildren.add(mapJoinOp); - dummyOp.setChildOperators(dummyChildren); - dummyOperators.add(dummyOp); - - // cut the operator tree so as to not retain connections from the parent RS downstream - List> childOperators = parentRS.getChildOperators(); - int childIndex = childOperators.indexOf(mapJoinOp); - childOperators.remove(childIndex); - - // the "work" needs to know about the dummy operators. They have to be separately initialized - // at task startup - if (mapJoinWork != null) { - for (BaseWork myWork: mapJoinWork) { - myWork.addDummyOp(dummyOp); - } - } - if (context.linkChildOpWithDummyOp.containsKey(mapJoinOp)) { - for (Operator op: context.linkChildOpWithDummyOp.get(mapJoinOp)) { - dummyOperators.add(op); - } - } - context.linkChildOpWithDummyOp.put(mapJoinOp, dummyOperators); - - // replace ReduceSinkOp with HashTableSinkOp for the RSops which are parents of MJop - MapJoinDesc mjDesc = mapJoinOp.getConf(); - HiveConf conf = context.conf; - - // Unlike in MR, we may call this method multiple times, for each - // small table HTS. But, since it's idempotent, it should be OK. - mjDesc.resetOrder(); - - float hashtableMemoryUsage; - if (hasGroupBy(mapJoinOp, context)) { - hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE); - } else { - hashtableMemoryUsage = conf.getFloatVar( - HiveConf.ConfVars.HIVEHASHTABLEMAXMEMORYUSAGE); - } - mjDesc.setHashTableMemoryUsage(hashtableMemoryUsage); - - SparkHashTableSinkDesc hashTableSinkDesc = new SparkHashTableSinkDesc(mjDesc); - SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) OperatorFactory.get( - mapJoinOp.getCompilationOpContext(), hashTableSinkDesc); - - byte tag = (byte) pos; - int[] valueIndex = mjDesc.getValueIndex(tag); - if (valueIndex != null) { - List newValues = new ArrayList(); - List values = hashTableSinkDesc.getExprs().get(tag); - for (int index = 0; index < values.size(); index++) { - if (valueIndex[index] < 0) { - newValues.add(values.get(index)); - } - } - hashTableSinkDesc.getExprs().put(tag, newValues); - } - - //get all parents of reduce sink - List> rsParentOps = parentRS.getParentOperators(); - for (Operator parent : rsParentOps) { - parent.replaceChild(parentRS, hashTableSinkOp); - } - hashTableSinkOp.setParentOperators(rsParentOps); - hashTableSinkOp.getConf().setTag(tag); - return true; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java deleted file mode 100644 index e54e517904b..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSMBJoinHintOptimizer.java +++ /dev/null @@ -1,102 +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.hadoop.hive.ql.optimizer.spark; - -import java.util.List; -import java.util.Stack; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.AbstractSMBJoinProc; -import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SMBJoinDesc; - -import com.clearspring.analytics.util.Preconditions; - -/** - * Converts from a bucket-mapjoin created from hints to SMB mapjoin. - */ -public class SparkSMBJoinHintOptimizer extends AbstractSMBJoinProc implements NodeProcessor { - - public SparkSMBJoinHintOptimizer(ParseContext pctx) { - super(pctx); - } - - public SparkSMBJoinHintOptimizer() { - } - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - MapJoinOperator mapJoinOp = (MapJoinOperator) nd; - SortBucketJoinProcCtx smbJoinContext = (SortBucketJoinProcCtx) procCtx; - - boolean convert = - canConvertBucketMapJoinToSMBJoin(mapJoinOp, stack, smbJoinContext, nodeOutputs); - - // Throw an error if the user asked for sort merge bucketed mapjoin to be enforced - // and sort merge bucketed mapjoin cannot be performed - if (!convert - && pGraphContext.getConf().getBoolVar( - HiveConf.ConfVars.HIVEENFORCESORTMERGEBUCKETMAPJOIN)) { - throw new SemanticException(ErrorMsg.SORTMERGE_MAPJOIN_FAILED.getMsg()); - } - - if (convert) { - removeSmallTableReduceSink(mapJoinOp); - convertBucketMapJoinToSMBJoin(mapJoinOp, smbJoinContext); - } - return null; - } - - /** - * In bucket mapjoin, there are ReduceSinks that mark a small table parent (Reduce Sink are removed from big-table). - * In SMB join these are not expected for any parents, either from small or big tables. - * @param mapJoinOp - */ - @SuppressWarnings("unchecked") - private void removeSmallTableReduceSink(MapJoinOperator mapJoinOp) { - SMBJoinDesc smbJoinDesc = new SMBJoinDesc(mapJoinOp.getConf()); - List> parentOperators = mapJoinOp.getParentOperators(); - for (int i = 0; i < parentOperators.size(); i++) { - Operator par = parentOperators.get(i); - if (i != smbJoinDesc.getPosBigTable()) { - if (par instanceof ReduceSinkOperator) { - List> grandParents = par.getParentOperators(); - Preconditions.checkArgument(grandParents.size() == 1, - "AssertionError: expect # of parents to be 1, but was " + grandParents.size()); - Operator grandParent = grandParents.get(0); - grandParent.removeChild(par); - grandParent.setChildOperators(Utilities.makeList(mapJoinOp)); - mapJoinOp.getParentOperators().set(i, grandParent); - } - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java deleted file mode 100644 index a7c18b0f206..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinProcFactory.java +++ /dev/null @@ -1,238 +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.hadoop.hive.ql.optimizer.spark; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.optimizer.physical.GenMRSkewJoinProcessor; -import org.apache.hadoop.hive.ql.optimizer.physical.GenSparkSkewJoinProcessor; -import org.apache.hadoop.hive.ql.optimizer.physical.SkewJoinProcFactory; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.PlanUtils; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; - -import java.io.Serializable; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.Stack; - -/** - * Spark-version of SkewJoinProcFactory. - */ -public class SparkSkewJoinProcFactory { - // let's remember the join operators we have processed - private static final Set visitedJoinOp = new HashSet(); - - private SparkSkewJoinProcFactory() { - // prevent instantiation - } - - public static NodeProcessor getDefaultProc() { - return SkewJoinProcFactory.getDefaultProc(); - } - - public static NodeProcessor getJoinProc() { - return new SparkSkewJoinJoinProcessor(); - } - - public static class SparkSkewJoinJoinProcessor implements NodeProcessor { - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - SparkSkewJoinResolver.SparkSkewJoinProcCtx context = - (SparkSkewJoinResolver.SparkSkewJoinProcCtx) procCtx; - Task currentTsk = context.getCurrentTask(); - JoinOperator op = (JoinOperator) nd; - ReduceWork reduceWork = context.getReducerToReduceWork().get(op); - ParseContext parseContext = context.getParseCtx(); - if (reduceWork != null && !visitedJoinOp.contains(op) && - supportRuntimeSkewJoin(op, reduceWork, currentTsk, parseContext.getConf())) { - // first we try to split the task - splitTask((SparkTask) currentTsk, reduceWork, parseContext); - GenSparkSkewJoinProcessor.processSkewJoin(op, currentTsk, reduceWork, parseContext); - visitedJoinOp.add(op); - } - return null; - } - } - - /** - * If the join is not in a leaf ReduceWork, the spark task has to be split into 2 tasks. - */ - private static void splitTask(SparkTask currentTask, ReduceWork reduceWork, - ParseContext parseContext) throws SemanticException { - SparkWork currentWork = currentTask.getWork(); - Set> reduceSinkSet = - OperatorUtils.getOp(reduceWork, ReduceSinkOperator.class); - if (currentWork.getChildren(reduceWork).size() == 1 && canSplit(currentWork) - && reduceSinkSet.size() == 1) { - ReduceSinkOperator reduceSink = (ReduceSinkOperator) reduceSinkSet.iterator().next(); - BaseWork childWork = currentWork.getChildren(reduceWork).get(0); - SparkEdgeProperty originEdge = currentWork.getEdgeProperty(reduceWork, childWork); - // disconnect the reduce work from its child. this should produce two isolated sub graphs - currentWork.disconnect(reduceWork, childWork); - // move works following the current reduce work into a new spark work - SparkWork newWork = - new SparkWork(parseContext.getConf().getVar(HiveConf.ConfVars.HIVEQUERYID)); - newWork.add(childWork); - copyWorkGraph(currentWork, newWork, childWork); - // remove them from current spark work - for (BaseWork baseWork : newWork.getAllWorkUnsorted()) { - currentWork.remove(baseWork); - currentWork.getCloneToWork().remove(baseWork); - } - // create TS to read intermediate data - Context baseCtx = parseContext.getContext(); - Path taskTmpDir = baseCtx.getMRTmpPath(); - Operator rsParent = reduceSink.getParentOperators().get(0); - TableDesc tableDesc = PlanUtils.getIntermediateFileTableDesc(PlanUtils - .getFieldSchemasFromRowSchema(rsParent.getSchema(), "temporarycol")); - // this will insert FS and TS between the RS and its parent - TableScanOperator tableScanOp = GenMapRedUtils.createTemporaryFile( - rsParent, reduceSink, taskTmpDir, tableDesc, parseContext); - // create new MapWork - MapWork mapWork = PlanUtils.getMapRedWork().getMapWork(); - mapWork.setName("Map " + GenSparkUtils.getUtils().getNextSeqNumber()); - newWork.add(mapWork); - newWork.connect(mapWork, childWork, originEdge); - // setup the new map work - String streamDesc = taskTmpDir.toUri().toString(); - if (GenMapRedUtils.needsTagging((ReduceWork) childWork)) { - Operator childReducer = ((ReduceWork) childWork).getReducer(); - String id = null; - if (childReducer instanceof JoinOperator) { - if (parseContext.getJoinOps().contains(childReducer)) { - id = ((JoinOperator)childReducer).getConf().getId(); - } - } else if (childReducer instanceof MapJoinOperator) { - if (parseContext.getMapJoinOps().contains(childReducer)) { - id = ((MapJoinOperator)childReducer).getConf().getId(); - } - } else if (childReducer instanceof SMBMapJoinOperator) { - if (parseContext.getSmbMapJoinOps().contains(childReducer)) { - id = ((SMBMapJoinOperator)childReducer).getConf().getId(); - } - } - if (id != null) { - streamDesc = id + ":$INTNAME"; - } else { - streamDesc = "$INTNAME"; - } - String origStreamDesc = streamDesc; - int pos = 0; - while (mapWork.getAliasToWork().get(streamDesc) != null) { - streamDesc = origStreamDesc.concat(String.valueOf(++pos)); - } - } - GenMapRedUtils.setTaskPlan(taskTmpDir, streamDesc, - tableScanOp, mapWork, false, tableDesc); - // insert the new task between current task and its child - @SuppressWarnings("unchecked") - Task newTask = TaskFactory.get(newWork); - List> childTasks = currentTask.getChildTasks(); - // must have at most one child - if (childTasks != null && childTasks.size() > 0) { - Task childTask = childTasks.get(0); - currentTask.removeDependentTask(childTask); - newTask.addDependentTask(childTask); - } - currentTask.addDependentTask(newTask); - newTask.setFetchSource(currentTask.isFetchSource()); - } - } - - /** - * Whether we can split at reduceWork. For simplicity, let's require each work can - * have at most one child work. This may be relaxed by checking connectivity of the - * work graph after disconnect the current reduce work from its child - */ - private static boolean canSplit(SparkWork sparkWork) { - for (BaseWork baseWork : sparkWork.getAllWorkUnsorted()) { - if (sparkWork.getChildren(baseWork).size() > 1) { - return false; - } - } - return true; - } - - /** - * Copy a sub-graph from originWork to newWork. - */ - private static void copyWorkGraph(SparkWork originWork, SparkWork newWork, BaseWork baseWork) { - for (BaseWork child : originWork.getChildren(baseWork)) { - if (!newWork.contains(child)) { - newWork.add(child); - SparkEdgeProperty edgeProperty = originWork.getEdgeProperty(baseWork, child); - newWork.connect(baseWork, child, edgeProperty); - copyWorkGraph(originWork, newWork, child); - } - } - for (BaseWork parent : originWork.getParents(baseWork)) { - if (!newWork.contains(parent)) { - newWork.add(parent); - SparkEdgeProperty edgeProperty = originWork.getEdgeProperty(parent, baseWork); - newWork.connect(parent, baseWork, edgeProperty); - copyWorkGraph(originWork, newWork, parent); - } - } - } - - public static Set getVisitedJoinOp() { - return visitedJoinOp; - } - - private static boolean supportRuntimeSkewJoin(JoinOperator joinOp, ReduceWork reduceWork, - Task currTask, HiveConf hiveConf) { - if (currTask instanceof SparkTask && - GenMRSkewJoinProcessor.skewJoinEnabled(hiveConf, joinOp)) { - SparkWork sparkWork = ((SparkTask) currTask).getWork(); - List> children = currTask.getChildTasks(); - return !joinOp.getConf().isFixedAsSorted() && sparkWork.contains(reduceWork) && - (children == null || children.size() <= 1) && - OperatorUtils.getOp(reduceWork, CommonJoinOperator.class).size() == 1; - } - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java deleted file mode 100644 index 089438ae29d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSkewJoinResolver.java +++ /dev/null @@ -1,127 +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.hadoop.hive.ql.optimizer.spark; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.exec.CommonJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; -import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.GraphWalker; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.PreOrderWalker; -import org.apache.hadoop.hive.ql.lib.Rule; -import org.apache.hadoop.hive.ql.lib.RuleRegExp; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalPlanResolver; -import org.apache.hadoop.hive.ql.optimizer.physical.SkewJoinResolver; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; - -/** - * Spark version of SkewJoinResolver. - */ -public class SparkSkewJoinResolver implements PhysicalPlanResolver { - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - SparkSkewJoinProcFactory.getVisitedJoinOp().clear(); - Dispatcher disp = new SparkSkewJoinTaskDispatcher(pctx); - // since we may split current task, use a pre-order walker - GraphWalker ogw = new PreOrderWalker(disp); - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pctx.getRootTasks()); - ogw.startWalking(topNodes, null); - return pctx; - } - - class SparkSkewJoinTaskDispatcher implements Dispatcher { - private PhysicalContext physicalContext; - - public SparkSkewJoinTaskDispatcher(PhysicalContext context) { - super(); - physicalContext = context; - } - - @Override - public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) - throws SemanticException { - - @SuppressWarnings("unchecked") - Task task = (Task) nd; - if (task instanceof SparkTask) { - SparkWork sparkWork = ((SparkTask) task).getWork(); - SparkSkewJoinProcCtx skewJoinProcCtx = - new SparkSkewJoinProcCtx(task, physicalContext.getParseContext()); - Map opRules = new LinkedHashMap(); - opRules.put(new RuleRegExp("R1", CommonJoinOperator.getOperatorName() + "%"), - SparkSkewJoinProcFactory.getJoinProc()); - Dispatcher disp = new DefaultRuleDispatcher( - SparkSkewJoinProcFactory.getDefaultProc(), opRules, skewJoinProcCtx); - GraphWalker ogw = new DefaultGraphWalker(disp); - ArrayList topNodes = new ArrayList(); - // since we may need to split the task, let's walk the graph bottom-up - List reduceWorkList = sparkWork.getAllReduceWork(); - Collections.reverse(reduceWorkList); - for (ReduceWork reduceWork : reduceWorkList) { - topNodes.add(reduceWork.getReducer()); - skewJoinProcCtx.getReducerToReduceWork().put(reduceWork.getReducer(), reduceWork); - } - ogw.startWalking(topNodes, null); - } - return null; - } - - public PhysicalContext getPhysicalContext() { - return physicalContext; - } - - public void setPhysicalContext(PhysicalContext physicalContext) { - this.physicalContext = physicalContext; - } - } - - public static class SparkSkewJoinProcCtx extends SkewJoinResolver.SkewJoinProcCtx { - // need a map from the reducer to the corresponding ReduceWork - private Map, ReduceWork> reducerToReduceWork; - - public SparkSkewJoinProcCtx(Task task, - ParseContext parseCtx) { - super(task, parseCtx); - reducerToReduceWork = new HashMap, ReduceWork>(); - } - - public Map, ReduceWork> getReducerToReduceWork() { - return reducerToReduceWork; - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java deleted file mode 100644 index 32be5c8ac5f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinFactory.java +++ /dev/null @@ -1,117 +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.hadoop.hive.ql.optimizer.spark; - -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkProcContext; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; - -/** - * Operator factory for Spark SMBJoin processing. - */ -public final class SparkSortMergeJoinFactory { - - private SparkSortMergeJoinFactory() { - // prevent instantiation - } - - /** - * Annotate MapWork, input is a SMBJoinOp that is part of a MapWork, and its root TS operator. - * - * 1. Initializes the MapWork's aliasToWork, pointing to big-table's TS. - * 2. Adds the bucketing information to the MapWork. - * 3. Adds localwork to the MapWork, with localWork's aliasToWork pointing to small-table's TS. - * @param context proc walker context - * @param mapWork mapwork to annotate - * @param smbMapJoinOp SMB Map Join operator to get data - * @param ts Table Scan operator to get data - * @param local Whether ts is from a 'local' source (small-table that will be loaded by SMBJoin 'local' task) - */ - public static void annotateMapWork(GenSparkProcContext context, MapWork mapWork, - SMBMapJoinOperator smbMapJoinOp, TableScanOperator ts, boolean local) - throws SemanticException { - initSMBJoinPlan(context, mapWork, ts, local); - setupBucketMapJoinInfo(mapWork, smbMapJoinOp); - } - - private static void setupBucketMapJoinInfo(MapWork plan, SMBMapJoinOperator currMapJoinOp) { - if (currMapJoinOp != null) { - Map>> aliasBucketFileNameMapping = - currMapJoinOp.getConf().getAliasBucketFileNameMapping(); - if (aliasBucketFileNameMapping != null) { - MapredLocalWork localPlan = plan.getMapRedLocalWork(); - if (localPlan == null) { - localPlan = currMapJoinOp.getConf().getLocalWork(); - } else { - // local plan is not null, we want to merge it into SMBMapJoinOperator's local work - MapredLocalWork smbLocalWork = currMapJoinOp.getConf().getLocalWork(); - if (smbLocalWork != null) { - localPlan.getAliasToFetchWork().putAll(smbLocalWork.getAliasToFetchWork()); - localPlan.getAliasToWork().putAll(smbLocalWork.getAliasToWork()); - } - } - - if (localPlan == null) { - return; - } - plan.setMapRedLocalWork(null); - currMapJoinOp.getConf().setLocalWork(localPlan); - - BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext(); - localPlan.setBucketMapjoinContext(bucketMJCxt); - bucketMJCxt.setAliasBucketFileNameMapping(aliasBucketFileNameMapping); - bucketMJCxt.setBucketFileNameMapping( - currMapJoinOp.getConf().getBigTableBucketNumMapping()); - localPlan.setInputFileChangeSensitive(true); - bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias()); - bucketMJCxt - .setBucketMatcherClass(org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class); - bucketMJCxt.setBigTablePartSpecToFileMapping( - currMapJoinOp.getConf().getBigTablePartSpecToFileMapping()); - - plan.setUseBucketizedHiveInputFormat(true); - - } - } - } - - private static void initSMBJoinPlan(GenSparkProcContext opProcCtx, - MapWork mapWork, TableScanOperator currentRootOperator, boolean local) - throws SemanticException { - String currAliasId = findAliasId(opProcCtx, currentRootOperator); - GenMapRedUtils.setMapWork(mapWork, opProcCtx.parseContext, - opProcCtx.inputs, null, currentRootOperator, currAliasId, opProcCtx.conf, local); - } - - private static String findAliasId(GenSparkProcContext opProcCtx, TableScanOperator ts) { - for (String alias : opProcCtx.topOps.keySet()) { - if (opProcCtx.topOps.get(alias) == ts) { - return alias; - } - } - return null; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java deleted file mode 100644 index 97b2989b100..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkSortMergeJoinOptimizer.java +++ /dev/null @@ -1,112 +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.hadoop.hive.ql.optimizer.spark; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.AbstractSMBJoinProc; -import org.apache.hadoop.hive.ql.optimizer.SortBucketJoinProcCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.OptimizeSparkProcContext; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; - -import java.util.Stack; - -/** - * Converts a common join operator to an SMB join if eligible. Handles auto SMB conversion. - */ -public class SparkSortMergeJoinOptimizer extends AbstractSMBJoinProc implements NodeProcessor { - - public SparkSortMergeJoinOptimizer(ParseContext pctx) { - super(pctx); - } - - public SparkSortMergeJoinOptimizer() { - } - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - - JoinOperator joinOp = (JoinOperator) nd; - HiveConf conf = ((OptimizeSparkProcContext) procCtx).getParseContext().getConf(); - - if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_AUTO_SORTMERGE_JOIN)) { - return null; - } - - SortBucketJoinProcCtx smbJoinContext = new SortBucketJoinProcCtx(conf); - - boolean convert = - canConvertJoinToSMBJoin( - joinOp, smbJoinContext, pGraphContext, stack); - - if (convert) { - return convertJoinToSMBJoinAndReturn(joinOp, smbJoinContext); - } - return null; - } - - protected boolean canConvertJoinToSMBJoin(JoinOperator joinOperator, - SortBucketJoinProcCtx smbJoinContext, ParseContext pGraphContext, - Stack stack) throws SemanticException { - if (!supportBucketMapJoin(stack)) { - return false; - } - return canConvertJoinToSMBJoin(joinOperator, smbJoinContext); - } - - //Preliminary checks. In the MR version of the code, these used to be done via another walk, - //here it is done inline. - private boolean supportBucketMapJoin(Stack stack) { - int size = stack.size(); - if (!(stack.get(size - 1) instanceof JoinOperator) - || !(stack.get(size - 2) instanceof ReduceSinkOperator)) { - return false; - } - - // If any operator in the stack does not support a auto-conversion, this join should - // not be converted. - for (int pos = size - 3; pos >= 0; pos--) { - @SuppressWarnings("unchecked") - Operator op = (Operator) stack.get(pos); - if (!op.supportAutomaticSortMergeJoin()) { - return false; - } - } - return true; - } - - protected SMBMapJoinOperator convertJoinToSMBJoinAndReturn( - JoinOperator joinOp, - SortBucketJoinProcCtx smbJoinContext) throws SemanticException { - MapJoinOperator mapJoinOp = convertJoinToBucketMapJoin(joinOp, smbJoinContext); - SMBMapJoinOperator smbMapJoinOp = - convertBucketMapJoinToSMBJoin(mapJoinOp, smbJoinContext); - smbMapJoinOp.setConvertedAutomaticallySMBJoin(true); - return smbMapJoinOp; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java deleted file mode 100644 index 3e1f85d11fb..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SplitSparkWorkResolver.java +++ /dev/null @@ -1,193 +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.hadoop.hive.ql.optimizer.spark; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.Set; - -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalPlanResolver; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.spark.GenSparkUtils; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; - -import com.google.common.base.Preconditions; - - -/** - * Do a BFS on the sparkWork graph, and look for any work that has more than one child. - * If we found such a work, we split it into multiple ones, one for each of its child. - */ -public class SplitSparkWorkResolver implements PhysicalPlanResolver { - @Override - public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { - for (Task task : pctx.getRootTasks()) { - if (task instanceof SparkTask) { - splitSparkWork(((SparkTask) task).getWork()); - } - } - return pctx; - } - - private void splitSparkWork(SparkWork sparkWork) { - Queue queue = new LinkedList(); - Set visited = new HashSet(); - queue.addAll(sparkWork.getRoots()); - while (!queue.isEmpty()) { - BaseWork work = queue.poll(); - if (!visited.add(work)) { - continue; - } - - List childWorks = sparkWork.getChildren(work); - // First, add all children of this work into queue, to be processed later. - for (BaseWork w : childWorks) { - queue.add(w); - } - - // Second, check if this work has multiple reduceSinks. If so, do split. - splitBaseWork(sparkWork, work, childWorks); - } - } - - // Split work into multiple branches, one for each childWork in childWorks. - // It also set up the connection between each parent work and child work. - private void splitBaseWork(SparkWork sparkWork, BaseWork parentWork, List childWorks) { - if (getAllReduceSinks(parentWork).size() <= 1) { - return; - } - - // Grand-parent works - we need to set these to be the parents of the cloned works. - List grandParentWorks = sparkWork.getParents(parentWork); - boolean isFirst = true; - - for (BaseWork childWork : childWorks) { - BaseWork clonedParentWork = SerializationUtilities.cloneBaseWork(parentWork); - // give the cloned work a different name - clonedParentWork.setName(clonedParentWork.getName().replaceAll("^([a-zA-Z]+)(\\s+)(\\d+)", - "$1$2" + GenSparkUtils.getUtils().getNextSeqNumber())); - setStatistics(parentWork, clonedParentWork); - String childReducerName = childWork.getName(); - SparkEdgeProperty clonedEdgeProperty = sparkWork.getEdgeProperty(parentWork, childWork); - - // We need to remove those branches that - // 1, ended with a ReduceSinkOperator, and - // 2, the ReduceSinkOperator's name is not the same as childReducerName. - // Also, if the cloned work is not the first, we remove ALL leaf operators except - // the corresponding ReduceSinkOperator. - for (Operator op : clonedParentWork.getAllLeafOperators()) { - if (op instanceof ReduceSinkOperator) { - if (!((ReduceSinkOperator) op).getConf().getOutputName().equals(childReducerName)) { - removeOpRecursive(op); - } - } else if (!isFirst) { - removeOpRecursive(op); - } - } - - isFirst = false; - - // Then, we need to set up the graph connection. Especially: - // 1, we need to connect this cloned parent work with all the grand-parent works. - // 2, we need to connect this cloned parent work with the corresponding child work. - sparkWork.add(clonedParentWork); - for (BaseWork gpw : grandParentWorks) { - sparkWork.connect(gpw, clonedParentWork, sparkWork.getEdgeProperty(gpw, parentWork)); - } - sparkWork.connect(clonedParentWork, childWork, clonedEdgeProperty); - sparkWork.getCloneToWork().put(clonedParentWork, parentWork); - } - - sparkWork.remove(parentWork); - } - - private Set> getAllReduceSinks(BaseWork work) { - Set> resultSet = work.getAllLeafOperators(); - Iterator> it = resultSet.iterator(); - while (it.hasNext()) { - if (!(it.next() instanceof ReduceSinkOperator)) { - it.remove(); - } - } - return resultSet; - } - - // Remove op from all its parents' child list. - // Recursively remove any of its parent who only have this op as child. - private void removeOpRecursive(Operator operator) { - List> parentOperators = new ArrayList>(); - for (Operator op : operator.getParentOperators()) { - parentOperators.add(op); - } - for (Operator parentOperator : parentOperators) { - Preconditions.checkArgument(parentOperator.getChildOperators().contains(operator), - "AssertionError: parent of " + operator.getName() + " doesn't have it as child."); - parentOperator.removeChild(operator); - if (parentOperator.getNumChild() == 0) { - removeOpRecursive(parentOperator); - } - } - } - - // we lost statistics & opTraits through cloning, try to get them back - private void setStatistics(BaseWork origin, BaseWork clone) { - if (origin instanceof MapWork && clone instanceof MapWork) { - MapWork originMW = (MapWork) origin; - MapWork cloneMW = (MapWork) clone; - for (Map.Entry> entry - : originMW.getAliasToWork().entrySet()) { - String alias = entry.getKey(); - Operator cloneOP = cloneMW.getAliasToWork().get(alias); - if (cloneOP != null) { - setStatistics(entry.getValue(), cloneOP); - } - } - } else if (origin instanceof ReduceWork && clone instanceof ReduceWork) { - setStatistics(((ReduceWork) origin).getReducer(), ((ReduceWork) clone).getReducer()); - } - } - - private void setStatistics(Operator origin, - Operator clone) { - clone.getConf().setStatistics(origin.getConf().getStatistics()); - clone.getConf().setTraits(origin.getConf().getTraits()); - if (origin.getChildOperators().size() == clone.getChildOperators().size()) { - for (int i = 0; i < clone.getChildOperators().size(); i++) { - setStatistics(origin.getChildOperators().get(i), clone.getChildOperators().get(i)); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompilerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompilerFactory.java index 0d49c763579..324df7681bc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompilerFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompilerFactory.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.parse; import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.parse.spark.SparkCompiler; /** * TaskCompilerFactory is a factory class to choose the appropriate diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java deleted file mode 100644 index ede9abc334d..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ /dev/null @@ -1,193 +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.hadoop.hive.ql.parse.spark; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.hooks.ReadEntity; -import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork; -import org.apache.hadoop.hive.ql.plan.FileSinkDesc; -import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * GenSparkProcContext maintains information about the tasks and operators - * as we walk the operator tree to break them into SparkTasks. - * - * Cloned from GenTezProcContext. - * - */ -public class GenSparkProcContext implements NodeProcessorCtx { - public final ParseContext parseContext; - public final HiveConf conf; - public final List> moveTask; - - // rootTasks is the entry point for all generated tasks - public final List> rootTasks; - - public final Set inputs; - public final Set outputs; - - // holds the root of the operator tree we're currently processing - // this could be a table scan, but also a join, ptf, etc (i.e.: - // first operator of a reduce task. - public Operator currentRootOperator; - - // this is the original parent of the currentRootOperator as we scan - // through the graph. A root operator might have multiple parents and - // we just use this one to remember where we came from in the current - // walk. - public Operator parentOfRoot; - - // Spark task we're currently processing - public SparkTask currentTask; - - // last work we've processed (in order to hook it up to the current - // one. - public BaseWork preceedingWork; - - // map that keeps track of the last operator of a task to the following work - // of this operator. This is used for connecting them later. - public final Map> - leafOpToFollowingWorkInfo; - - // a map that keeps track of work that need to be linked while - // traversing an operator tree - public final Map, Map> linkOpWithWorkMap; - - // a map to keep track of what reduce sinks have to be hooked up to - // map join work - public final Map> linkWorkWithReduceSinkMap; - - // map that says which mapjoin belongs to which work item - public final Map> mapJoinWorkMap; - - // Map to keep track of which SMB Join operators and their information to annotate their MapWork with. - public final Map smbMapJoinCtxMap; - - // a map to keep track of which root generated which work - public final Map, BaseWork> rootToWorkMap; - - // a map to keep track of which child generated with work - public final Map, List> childToWorkMap; - - // we need to keep the original list of operators in the map join to know - // what position in the mapjoin the different parent work items will have. - public final Map>> mapJoinParentMap; - - // remember the dummy ops we created - public final Map, List>> linkChildOpWithDummyOp; - - // used to group dependent tasks for multi table inserts - public final DependencyCollectionTask dependencyTask; - - // remember map joins as we encounter them. - public final Set currentMapJoinOperators; - - // used to hook up unions - public final Map, BaseWork> unionWorkMap; - public final List currentUnionOperators; - public final Set workWithUnionOperators; - - // we link filesink that will write to the same final location - public final Map> linkedFileSinks; - public final Set fileSinkSet; - public final Map> fileSinkMap; - - // Alias to operator map, from the semantic analyzer. - // This is necessary as sometimes semantic analyzer's mapping is different than operator's own alias. - public final Map topOps; - - // The set of pruning sinks - public final Set> pruningSinkSet; - - // The set of TableScanOperators for pruning OP trees - public final Set> clonedPruningTableScanSet; - - - @SuppressWarnings("unchecked") - public GenSparkProcContext(HiveConf conf, - ParseContext parseContext, - List> moveTask, - List> rootTasks, - Set inputs, - Set outputs, - Map topOps) { - this.conf = conf; - this.parseContext = parseContext; - this.moveTask = moveTask; - this.rootTasks = rootTasks; - this.inputs = inputs; - this.outputs = outputs; - this.topOps = topOps; - this.currentTask = SparkUtilities.createSparkTask(conf); - this.rootTasks.add(currentTask); - this.leafOpToFollowingWorkInfo = - new LinkedHashMap>(); - this.linkOpWithWorkMap = new LinkedHashMap, Map>(); - this.linkWorkWithReduceSinkMap = new LinkedHashMap>(); - this.smbMapJoinCtxMap = new HashMap(); - this.mapJoinWorkMap = new LinkedHashMap>(); - this.rootToWorkMap = new LinkedHashMap, BaseWork>(); - this.childToWorkMap = new LinkedHashMap, List>(); - this.mapJoinParentMap = new LinkedHashMap>>(); - this.currentMapJoinOperators = new LinkedHashSet(); - this.linkChildOpWithDummyOp = new LinkedHashMap, List>>(); - this.dependencyTask = conf.getBoolVar( - HiveConf.ConfVars.HIVE_MULTI_INSERT_MOVE_TASKS_SHARE_DEPENDENCIES) - ? (DependencyCollectionTask) TaskFactory.get(new DependencyCollectionWork()) - : null; - this.unionWorkMap = new LinkedHashMap, BaseWork>(); - this.currentUnionOperators = new LinkedList(); - this.workWithUnionOperators = new LinkedHashSet(); - this.linkedFileSinks = new LinkedHashMap<>(); - this.fileSinkSet = new LinkedHashSet(); - this.fileSinkMap = new LinkedHashMap>(); - this.pruningSinkSet = new LinkedHashSet>(); - this.clonedPruningTableScanSet = new LinkedHashSet>(); - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java deleted file mode 100644 index 900a80000d8..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ /dev/null @@ -1,662 +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.hadoop.hive.ql.parse.spark; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Deque; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask; -import org.apache.hadoop.hive.ql.exec.FetchTask; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.ForwardOperator; -import org.apache.hadoop.hive.ql.exec.GroupByOperator; -import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.FileSinkDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.base.Preconditions; -import com.google.common.base.Strings; - -/** - * GenSparkUtils is a collection of shared helper methods to produce SparkWork - * Cloned from GenTezUtils. - */ -public class GenSparkUtils { - private static final Logger LOG = LoggerFactory.getLogger(GenSparkUtils.class.getName()); - - // sequence number is used to name vertices (e.g.: Map 1, Reduce 14, ...) - private int sequenceNumber = 0; - - // singleton - private static GenSparkUtils utils; - - public static GenSparkUtils getUtils() { - if (utils == null) { - utils = new GenSparkUtils(); - } - return utils; - } - - protected GenSparkUtils() { - } - - public void resetSequenceNumber() { - sequenceNumber = 0; - } - - public ReduceWork createReduceWork(GenSparkProcContext context, Operator root, - SparkWork sparkWork) throws SemanticException { - Preconditions.checkArgument(!root.getParentOperators().isEmpty(), - "AssertionError: expected root.getParentOperators() to be non-empty"); - - ReduceWork reduceWork = new ReduceWork("Reducer " + (++sequenceNumber)); - LOG.debug("Adding reduce work (" + reduceWork.getName() + ") for " + root); - reduceWork.setReducer(root); - reduceWork.setNeedsTagging(GenMapRedUtils.needsTagging(reduceWork)); - - // Pick the maximum # reducers across all parents as the # of reduce tasks. - int maxExecutors = -1; - for (Operator parentOfRoot : root.getParentOperators()) { - Preconditions.checkArgument(parentOfRoot instanceof ReduceSinkOperator, - "AssertionError: expected parentOfRoot to be an " - + "instance of ReduceSinkOperator, but was " - + parentOfRoot.getClass().getName()); - ReduceSinkOperator reduceSink = (ReduceSinkOperator) parentOfRoot; - maxExecutors = Math.max(maxExecutors, reduceSink.getConf().getNumReducers()); - } - reduceWork.setNumReduceTasks(maxExecutors); - - ReduceSinkOperator reduceSink = (ReduceSinkOperator) context.parentOfRoot; - setupReduceSink(context, reduceWork, reduceSink); - sparkWork.add(reduceWork); - SparkEdgeProperty edgeProp = getEdgeProperty(context.conf, reduceSink, reduceWork); - - sparkWork.connect(context.preceedingWork, reduceWork, edgeProp); - - return reduceWork; - } - - protected void setupReduceSink(GenSparkProcContext context, ReduceWork reduceWork, - ReduceSinkOperator reduceSink) { - - LOG.debug("Setting up reduce sink: " + reduceSink - + " with following reduce work: " + reduceWork.getName()); - - // need to fill in information about the key and value in the reducer - GenMapRedUtils.setKeyAndValueDesc(reduceWork, reduceSink); - - // remember which parent belongs to which tag - reduceWork.getTagToInput().put(reduceSink.getConf().getTag(), - context.preceedingWork.getName()); - - // remember the output name of the reduce sink - reduceSink.getConf().setOutputName(reduceWork.getName()); - } - - public MapWork createMapWork(GenSparkProcContext context, Operator root, - SparkWork sparkWork, PrunedPartitionList partitions) throws SemanticException { - return createMapWork(context, root, sparkWork, partitions, false); - } - - public MapWork createMapWork(GenSparkProcContext context, Operator root, - SparkWork sparkWork, PrunedPartitionList partitions, boolean deferSetup) throws SemanticException { - Preconditions.checkArgument(root.getParentOperators().isEmpty(), - "AssertionError: expected root.getParentOperators() to be empty"); - MapWork mapWork = new MapWork("Map " + (++sequenceNumber)); - LOG.debug("Adding map work (" + mapWork.getName() + ") for " + root); - - // map work starts with table scan operators - Preconditions.checkArgument(root instanceof TableScanOperator, - "AssertionError: expected root to be an instance of TableScanOperator, but was " - + root.getClass().getName()); - String alias_id = null; - if (context.parseContext != null && context.parseContext.getTopOps() != null) { - for (String currentAliasID : context.parseContext.getTopOps().keySet()) { - Operator currOp = context.parseContext.getTopOps().get(currentAliasID); - if (currOp == root) { - alias_id = currentAliasID; - break; - } - } - } - if (alias_id == null) - alias_id = ((TableScanOperator) root).getConf().getAlias(); - if (!deferSetup) { - setupMapWork(mapWork, context, partitions,(TableScanOperator) root, alias_id); - } - - // add new item to the Spark work - sparkWork.add(mapWork); - - return mapWork; - } - - // this method's main use is to help unit testing this class - protected void setupMapWork(MapWork mapWork, GenSparkProcContext context, - PrunedPartitionList partitions, TableScanOperator root, - String alias_id) throws SemanticException { - // All the setup is done in GenMapRedUtils - GenMapRedUtils.setMapWork(mapWork, context.parseContext, - context.inputs, partitions, root, alias_id, context.conf, false); - } - - private void collectOperators(Operator op, List> opList) { - opList.add(op); - for (Object child : op.getChildOperators()) { - if (child != null) { - collectOperators((Operator) child, opList); - } - } - } - - // removes any union operator and clones the plan - public void removeUnionOperators(GenSparkProcContext context, BaseWork work) - throws SemanticException { - - List> roots = new ArrayList>(); - - // For MapWork, getAllRootOperators is not suitable, since it checks - // getPathToAliases, and will return null if this is empty. Here we are - // replacing getAliasToWork, so should use that information instead. - if (work instanceof MapWork) { - roots.addAll(((MapWork) work).getAliasToWork().values()); - } else { - roots.addAll(work.getAllRootOperators()); - } - if (work.getDummyOps() != null) { - roots.addAll(work.getDummyOps()); - } - - // need to clone the plan. - List> newRoots = SerializationUtilities.cloneOperatorTree(roots); - - // Build a map to map the original FileSinkOperator and the cloned FileSinkOperators - // This map is used for set the stats flag for the cloned FileSinkOperators in later process - Iterator> newRootsIt = newRoots.iterator(); - for (Operator root : roots) { - Operator newRoot = newRootsIt.next(); - List> newOpQueue = new LinkedList>(); - collectOperators(newRoot, newOpQueue); - List> opQueue = new LinkedList>(); - collectOperators(root, opQueue); - Iterator> newOpQueueIt = newOpQueue.iterator(); - for (Operator op : opQueue) { - Operator newOp = newOpQueueIt.next(); - - // We need to update rootToWorkMap in case the op is a key, since even - // though we clone the op tree, we're still using the same MapWork/ReduceWork. - if (context.rootToWorkMap.containsKey(op)) { - context.rootToWorkMap.put(newOp, context.rootToWorkMap.get(op)); - } - // Don't remove the old entry - in SparkPartitionPruningSink it still - // refers to the old TS, and we need to lookup it later in - // processPartitionPruningSink. - - if (op instanceof FileSinkOperator) { - List fileSinkList = context.fileSinkMap.get(op); - if (fileSinkList == null) { - fileSinkList = new LinkedList(); - } - fileSinkList.add((FileSinkOperator) newOp); - context.fileSinkMap.put((FileSinkOperator) op, fileSinkList); - } else if (op instanceof SparkPartitionPruningSinkOperator) { - SparkPartitionPruningSinkOperator oldPruningSink = (SparkPartitionPruningSinkOperator) op; - SparkPartitionPruningSinkOperator newPruningSink = (SparkPartitionPruningSinkOperator) newOp; - for (int i = 0; i < oldPruningSink.getConf().getTargetInfos().size(); i++) { - newPruningSink.getConf().getTargetInfos().get(i).tableScan = - oldPruningSink.getConf().getTargetInfos().get(i).tableScan; - } - context.pruningSinkSet.add(newPruningSink); - context.pruningSinkSet.remove(oldPruningSink); - } - } - } - - // we're cloning the operator plan but we're retaining the original work. That means - // that root operators have to be replaced with the cloned ops. The replacement map - // tells you what that mapping is. - Map, Operator> replacementMap = new HashMap, Operator>(); - - // there's some special handling for dummyOps required. Mapjoins won't be properly - // initialized if their dummy parents aren't initialized. Since we cloned the plan - // we need to replace the dummy operators in the work with the cloned ones. - List dummyOps = new LinkedList(); - - Iterator> it = newRoots.iterator(); - for (Operator orig: roots) { - Set fsOpSet = OperatorUtils.findOperators(orig, FileSinkOperator.class); - for (FileSinkOperator fsOp : fsOpSet) { - context.fileSinkSet.remove(fsOp); - } - - Operator newRoot = it.next(); - if (newRoot instanceof HashTableDummyOperator) { - dummyOps.add((HashTableDummyOperator) newRoot); - it.remove(); - } else { - replacementMap.put(orig, newRoot); - } - } - - // now we remove all the unions. we throw away any branch that's not reachable from - // the current set of roots. The reason is that those branches will be handled in - // different tasks. - Deque> operators = new LinkedList>(); - operators.addAll(newRoots); - - Set> seen = new HashSet>(); - - while (!operators.isEmpty()) { - Operator current = operators.pop(); - seen.add(current); - - if (current instanceof FileSinkOperator) { - FileSinkOperator fileSink = (FileSinkOperator)current; - - // remember it for additional processing later - context.fileSinkSet.add(fileSink); - - FileSinkDesc desc = fileSink.getConf(); - Path path = desc.getDirName(); - List linked; - - if (!context.linkedFileSinks.containsKey(path)) { - linked = new ArrayList(); - context.linkedFileSinks.put(path, linked); - } - linked = context.linkedFileSinks.get(path); - linked.add(desc); - - desc.setLinkedFileSinkDesc(linked); - } - - if (current instanceof UnionOperator) { - Operator parent = null; - int count = 0; - - for (Operator op: current.getParentOperators()) { - if (seen.contains(op)) { - ++count; - parent = op; - } - } - - // we should have been able to reach the union from only one side. - Preconditions.checkArgument(count <= 1, - "AssertionError: expected count to be <= 1, but was " + count); - - if (parent == null) { - // root operator is union (can happen in reducers) - replacementMap.put(current, current.getChildOperators().get(0)); - } else { - parent.removeChildAndAdoptItsChildren(current); - } - } - - if (current instanceof FileSinkOperator - || current instanceof ReduceSinkOperator) { - current.setChildOperators(null); - } else { - operators.addAll(current.getChildOperators()); - } - } - work.setDummyOps(dummyOps); - work.replaceRoots(replacementMap); - } - - public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSink) - throws SemanticException { - - ParseContext parseContext = context.parseContext; - - boolean isInsertTable = // is INSERT OVERWRITE TABLE - GenMapRedUtils.isInsertInto(parseContext, fileSink); - HiveConf hconf = parseContext.getConf(); - - boolean chDir = GenMapRedUtils.isMergeRequired(context.moveTask, - hconf, fileSink, context.currentTask, isInsertTable); - // Set stats config for FileSinkOperators which are cloned from the fileSink - List fileSinkList = context.fileSinkMap.get(fileSink); - if (fileSinkList != null) { - for (FileSinkOperator fsOp : fileSinkList) { - fsOp.getConf().setGatherStats(fileSink.getConf().isGatherStats()); - fsOp.getConf().setStatsReliable(fileSink.getConf().isStatsReliable()); - } - } - - Path finalName = createMoveTask(context.currentTask, - chDir, fileSink, parseContext, context.moveTask, hconf, context.dependencyTask); - - if (chDir) { - // Merge the files in the destination table/partitions by creating Map-only merge job - // If underlying data is RCFile a RCFileBlockMerge task would be created. - LOG.info("using CombineHiveInputformat for the merge job"); - GenMapRedUtils.createMRWorkForMergingFiles(fileSink, finalName, - context.dependencyTask, context.moveTask, - hconf, context.currentTask, parseContext.getQueryState().getLineageState()); - } - - FetchTask fetchTask = parseContext.getFetchTask(); - if (fetchTask != null && context.currentTask.getNumChild() == 0) { - if (fetchTask.isFetchFrom(fileSink.getConf())) { - context.currentTask.setFetchSource(true); - } - } - } - - /** - * Create and add any dependent move tasks. - * - * This is forked from {@link GenMapRedUtils}. The difference is that it doesn't check - * 'isLinkedFileSink' and does not set parent dir for the linked file sinks. - */ - public static Path createMoveTask(Task currTask, boolean chDir, - FileSinkOperator fsOp, ParseContext parseCtx, List> mvTasks, - HiveConf hconf, DependencyCollectionTask dependencyTask) { - - Path dest = null; - FileSinkDesc fileSinkDesc = fsOp.getConf(); - - if (chDir) { - dest = fsOp.getConf().getFinalDirName(); - - // generate the temporary file - // it must be on the same file system as the current destination - Context baseCtx = parseCtx.getContext(); - - Path tmpDir = baseCtx.getExternalTmpPath(dest); - - // Change all the linked file sink descriptors - if (fileSinkDesc.getLinkedFileSinkDesc() != null) { - for (FileSinkDesc fsConf : fileSinkDesc.getLinkedFileSinkDesc()) { - fsConf.setDirName(tmpDir); - } - } else { - fileSinkDesc.setDirName(tmpDir); - } - } - - Task mvTask = null; - - if (!chDir) { - mvTask = GenMapRedUtils.findMoveTaskForFsopOutput(mvTasks, fileSinkDesc.getFinalDirName(), false); - } - - // Set the move task to be dependent on the current task - if (mvTask != null) { - GenMapRedUtils.addDependentMoveTasks(mvTask, hconf, currTask, dependencyTask); - } - - return dest; - } - - /** - * Populate partition pruning information from the pruning sink operator to the - * target MapWork (the MapWork for the big table side). The information include the source table - * name, column name, and partition key expression. It also set up the temporary path used to - * communicate between the target MapWork and source BaseWork. - * - * Here "source" refers to the small table side, while "target" refers to the big - * table side. - * - * @param context the spark context. - * @param pruningSink the pruner sink operator being processed. - */ - public void processPartitionPruningSink(GenSparkProcContext context, - SparkPartitionPruningSinkOperator pruningSink) { - SparkPartitionPruningSinkDesc desc = pruningSink.getConf(); - final Path outputBase = getDPPOutputPath(context.parseContext.getContext()); - final String sourceId = pruningSink.getUniqueId(); - desc.setPath(new Path(outputBase, sourceId)); - - for (SparkPartitionPruningSinkDesc.DPPTargetInfo targetInfo : desc.getTargetInfos()) { - TableScanOperator ts = targetInfo.tableScan; - MapWork targetWork = (MapWork) context.rootToWorkMap.get(ts); - Preconditions.checkNotNull(targetWork, "No targetWork found for tablescan " + ts); - - // set up temporary path to communicate between the small/big table - if (targetWork.getTmpPathForPartitionPruning() == null) { - targetWork.setTmpPathForPartitionPruning(outputBase); - LOG.info("Setting tmp path between source work and target work:\n" + outputBase); - } - - targetInfo.work = targetWork; - targetInfo.columnName = SparkUtilities.getWorkId(targetWork) + ":" + targetInfo.columnName; - - pruningSink.addAsSourceEvent(targetWork, targetInfo.partKey, targetInfo.columnName, - targetInfo.columnType); - } - } - - private Path getDPPOutputPath(Context context) { - return new Path(context.getMRScratchDir(), "_dpp_output_"); - } - - public static SparkEdgeProperty getEdgeProperty(HiveConf conf, ReduceSinkOperator reduceSink, - ReduceWork reduceWork) throws SemanticException { - boolean useSparkGroupBy = conf.getBoolVar(HiveConf.ConfVars.SPARK_USE_GROUPBY_SHUFFLE); - SparkEdgeProperty edgeProperty = new SparkEdgeProperty(SparkEdgeProperty.SHUFFLE_NONE); - edgeProperty.setNumPartitions(reduceWork.getNumReduceTasks()); - String sortOrder = Strings.nullToEmpty(reduceSink.getConf().getOrder()).trim(); - - if (hasGBYOperator(reduceSink)) { - edgeProperty.setShuffleGroup(); - // test if the group by needs partition level sort, if so, use the MR style shuffle - // SHUFFLE_SORT shouldn't be used for this purpose, see HIVE-8542 - if (!useSparkGroupBy || (!sortOrder.isEmpty() && groupByNeedParLevelOrder(reduceSink))) { - if (!useSparkGroupBy) { - LOG.info("hive.spark.use.groupby.shuffle is off. Use repartition shuffle instead."); - } - edgeProperty.setMRShuffle(); - } - } - - if (reduceWork.getReducer() instanceof JoinOperator) { - //reduce-side join, use MR-style shuffle - edgeProperty.setMRShuffle(); - } - - //If its a FileSink to bucketed files, also use MR-style shuffle to - // get compatible taskId for bucket-name - FileSinkOperator fso = getChildOperator(reduceWork.getReducer(), FileSinkOperator.class); - if (fso != null) { - String bucketCount = fso.getConf().getTableInfo().getProperties().getProperty( - hive_metastoreConstants.BUCKET_COUNT); - if (bucketCount != null && Integer.parseInt(bucketCount) > 1) { - edgeProperty.setMRShuffle(); - } - } - - // test if we need partition/global order, SHUFFLE_SORT should only be used for global order - if (edgeProperty.isShuffleNone() && !sortOrder.isEmpty()) { - if ((reduceSink.getConf().getPartitionCols() == null - || reduceSink.getConf().getPartitionCols().isEmpty() - || isSame(reduceSink.getConf().getPartitionCols(), reduceSink.getConf().getKeyCols())) - && reduceSink.getConf().hasOrderBy()) { - edgeProperty.setShuffleSort(); - } else { - edgeProperty.setMRShuffle(); - } - } - - // simple distribute-by goes here - if (edgeProperty.isShuffleNone()) { - if (!useSparkGroupBy) { - LOG.info("hive.spark.use.groupby.shuffle is off. Use repartition shuffle instead."); - edgeProperty.setMRShuffle(); - } else { - edgeProperty.setShuffleGroup(); - } - } - - - return edgeProperty; - } - - /** - * Test if we need partition level order for group by query. - * GBY needs partition level order when distinct is present. Therefore, if the sorting - * keys, partitioning keys and grouping keys are the same, we ignore the sort and use - * GroupByShuffler to shuffle the data. In this case a group-by transformation should be - * sufficient to produce the correct results, i.e. data is properly grouped by the keys - * but keys are not guaranteed to be sorted. - */ - private static boolean groupByNeedParLevelOrder(ReduceSinkOperator reduceSinkOperator) { - // whether we have to enforce sort anyway, e.g. in case of RS deduplication - if (reduceSinkOperator.getConf().isDeduplicated()) { - return true; - } - List> children = reduceSinkOperator.getChildOperators(); - if (children != null && children.size() == 1 - && children.get(0) instanceof GroupByOperator) { - GroupByOperator child = (GroupByOperator) children.get(0); - if (isSame(reduceSinkOperator.getConf().getKeyCols(), - reduceSinkOperator.getConf().getPartitionCols()) - && reduceSinkOperator.getConf().getKeyCols().size() == child.getConf().getKeys().size()) { - return false; - } - } - return true; - } - - /** - * Test if two lists of ExprNodeDesc are semantically same. - */ - private static boolean isSame(List list1, List list2) { - if (list1 != list2) { - if (list1 != null && list2 != null) { - if (list1.size() != list2.size()) { - return false; - } - for (int i = 0; i < list1.size(); i++) { - if (!list1.get(i).isSame(list2.get(i))) { - return false; - } - } - } else { - return false; - } - } - return true; - } - - @SuppressWarnings("unchecked") - public static T getChildOperator(Operator root, Class klazz) throws SemanticException { - if (root == null) return null; - - HashSet> visited = new HashSet>(); - Stack> stack = new Stack>(); - stack.push(root); - visited.add(root); - - while (!stack.isEmpty()) { - Operator op = stack.pop(); - if (klazz.isInstance(op)) { - return (T) op; - } - List> childOperators = op.getChildOperators(); - for (Operator childOp : childOperators) { - if (!visited.contains(childOp)) { - stack.push(childOp); - visited.add(childOp); - } - } - } - - return null; - } - - /** - * Fill MapWork with 'local' work and bucket information for SMB Join. - * @param context context, containing references to MapWorks and their SMB information. - * @throws SemanticException - */ - public void annotateMapWork(GenSparkProcContext context) throws SemanticException { - for (SMBMapJoinOperator smbMapJoinOp : context.smbMapJoinCtxMap.keySet()) { - //initialize mapwork with smbMapJoin information. - SparkSMBMapJoinInfo smbMapJoinInfo = context.smbMapJoinCtxMap.get(smbMapJoinOp); - MapWork work = smbMapJoinInfo.mapWork; - SparkSortMergeJoinFactory.annotateMapWork(context, work, smbMapJoinOp, - (TableScanOperator) smbMapJoinInfo.bigTableRootOp, false); - for (Operator smallTableRootOp : smbMapJoinInfo.smallTableRootOps) { - SparkSortMergeJoinFactory.annotateMapWork(context, work, smbMapJoinOp, - (TableScanOperator) smallTableRootOp, true); - } - } - } - - public synchronized int getNextSeqNumber() { - return ++sequenceNumber; - } - - // test if we need group-by shuffle - private static boolean hasGBYOperator(ReduceSinkOperator rs) { - if (rs.getChildOperators().size() == 1) { - if (rs.getChildOperators().get(0) instanceof GroupByOperator) { - return true; - } else if (rs.getChildOperators().get(0) instanceof ForwardOperator) { - for (Operator grandChild : rs.getChildOperators().get(0).getChildOperators()) { - if (!(grandChild instanceof GroupByOperator)) { - return false; - } - } - return true; - } - } - return false; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java deleted file mode 100644 index 04b120200a3..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java +++ /dev/null @@ -1,290 +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.hadoop.hive.ql.parse.spark; - -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Stack; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.ObjectPair; -import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorFactory; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkSortMergeJoinFactory; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc; -import org.apache.hadoop.hive.ql.plan.ReduceWork; -import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty; -import org.apache.hadoop.hive.ql.plan.SparkWork; - -import com.google.common.base.Preconditions; - -/** - * GenSparkWork separates the operator tree into spark tasks. - * It is called once per leaf operator (operator that forces a new execution unit.) - * and break the operators into work and tasks along the way. - * - * Cloned from GenTezWork. - */ -public class GenSparkWork implements NodeProcessor { - static final private Logger LOG = LoggerFactory.getLogger(GenSparkWork.class.getName()); - - // instance of shared utils - private GenSparkUtils utils = null; - - /** - * Constructor takes utils as parameter to facilitate testing - */ - public GenSparkWork(GenSparkUtils utils) { - this.utils = utils; - } - - @Override - public Object process(Node nd, Stack stack, - NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procContext; - - Preconditions.checkArgument(context != null, - "AssertionError: expected context to be not null"); - Preconditions.checkArgument(context.currentTask != null, - "AssertionError: expected context.currentTask to be not null"); - Preconditions.checkArgument(context.currentRootOperator != null, - "AssertionError: expected context.currentRootOperator to be not null"); - - // Operator is a file sink or reduce sink. Something that forces a new vertex. - @SuppressWarnings("unchecked") - Operator operator = (Operator) nd; - - // root is the start of the operator pipeline we're currently - // packing into a vertex, typically a table scan, union or join - Operator root = context.currentRootOperator; - - LOG.debug("Root operator: " + root); - LOG.debug("Leaf operator: " + operator); - - SparkWork sparkWork = context.currentTask.getWork(); - SMBMapJoinOperator smbOp = GenSparkUtils.getChildOperator(root, SMBMapJoinOperator.class); - - // Right now the work graph is pretty simple. If there is no - // Preceding work we have a root and will generate a map - // vertex. If there is a preceding work we will generate - // a reduce vertex - BaseWork work; - if (context.rootToWorkMap.containsKey(root)) { - // having seen the root operator before means there was a branch in the - // operator graph. There's typically two reasons for that: a) mux/demux - // b) multi insert. Mux/Demux will hit the same leaf again, multi insert - // will result into a vertex with multiple FS or RS operators. - - // At this point we don't have to do anything special in this case. Just - // run through the regular paces w/o creating a new task. - work = context.rootToWorkMap.get(root); - } else { - // create a new vertex - if (context.preceedingWork == null) { - if (smbOp == null) { - work = utils.createMapWork(context, root, sparkWork, null); - } else { - //save work to be initialized later with SMB information. - work = utils.createMapWork(context, root, sparkWork, null, true); - context.smbMapJoinCtxMap.get(smbOp).mapWork = (MapWork) work; - } - } else { - work = utils.createReduceWork(context, root, sparkWork); - } - context.rootToWorkMap.put(root, work); - } - - if (!context.childToWorkMap.containsKey(operator)) { - List workItems = new LinkedList(); - workItems.add(work); - context.childToWorkMap.put(operator, workItems); - } else { - context.childToWorkMap.get(operator).add(work); - } - - // remember which mapjoin operator links with which work - if (!context.currentMapJoinOperators.isEmpty()) { - for (MapJoinOperator mj: context.currentMapJoinOperators) { - LOG.debug("Processing map join: " + mj); - // remember the mapping in case we scan another branch of the mapjoin later - if (!context.mapJoinWorkMap.containsKey(mj)) { - List workItems = new LinkedList(); - workItems.add(work); - context.mapJoinWorkMap.put(mj, workItems); - } else { - context.mapJoinWorkMap.get(mj).add(work); - } - - /* - * this happens in case of map join operations. - * The tree looks like this: - * - * RS <--- we are here perhaps - * | - * MapJoin - * / \ - * RS TS - * / - * TS - * - * If we are at the RS pointed above, and we may have already visited the - * RS following the TS, we have already generated work for the TS-RS. - * We need to hook the current work to this generated work. - */ - if (context.linkOpWithWorkMap.containsKey(mj)) { - Map linkWorkMap = context.linkOpWithWorkMap.get(mj); - if (linkWorkMap != null) { - if (context.linkChildOpWithDummyOp.containsKey(mj)) { - for (Operator dummy: context.linkChildOpWithDummyOp.get(mj)) { - work.addDummyOp((HashTableDummyOperator) dummy); - } - } - for (Entry parentWorkMap : linkWorkMap.entrySet()) { - BaseWork parentWork = parentWorkMap.getKey(); - LOG.debug("connecting " + parentWork.getName() + " with " + work.getName()); - SparkEdgeProperty edgeProp = parentWorkMap.getValue(); - sparkWork.connect(parentWork, work, edgeProp); - - // need to set up output name for reduce sink now that we know the name - // of the downstream work - for (ReduceSinkOperator r : context.linkWorkWithReduceSinkMap.get(parentWork)) { - if (r.getConf().getOutputName() != null) { - LOG.debug("Cloning reduce sink for multi-child broadcast edge"); - // we've already set this one up. Need to clone for the next work. - r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild( - r.getCompilationOpContext(), (ReduceSinkDesc)r.getConf().clone(), - r.getParentOperators()); - } - r.getConf().setOutputName(work.getName()); - } - } - } - } - } - // clear out the set. we don't need it anymore. - context.currentMapJoinOperators.clear(); - } - - // Here we are disconnecting root with its parents. However, we need to save - // a few information, since in future we may reach the parent operators via a - // different path, and we may need to connect parent works with the work associated - // with this root operator. - if (root.getNumParent() > 0) { - Preconditions.checkArgument(work instanceof ReduceWork, - "AssertionError: expected work to be a ReduceWork, but was " + work.getClass().getName()); - ReduceWork reduceWork = (ReduceWork) work; - for (Operator parent : new ArrayList>(root.getParentOperators())) { - Preconditions.checkArgument(parent instanceof ReduceSinkOperator, - "AssertionError: expected operator to be a ReduceSinkOperator, but was " - + parent.getClass().getName()); - ReduceSinkOperator rsOp = (ReduceSinkOperator) parent; - SparkEdgeProperty edgeProp = GenSparkUtils.getEdgeProperty(context.conf, rsOp, reduceWork); - - rsOp.getConf().setOutputName(reduceWork.getName()); - GenMapRedUtils.setKeyAndValueDesc(reduceWork, rsOp); - - context.leafOpToFollowingWorkInfo.put(rsOp, ObjectPair.create(edgeProp, reduceWork)); - LOG.debug("Removing " + parent + " as parent from " + root); - root.removeParent(parent); - } - } - - // If `currentUnionOperators` is not empty, it means we are creating BaseWork whose operator tree - // contains union operators. In this case, we need to save these BaseWorks, and remove - // the union operators from the operator tree later. - if (!context.currentUnionOperators.isEmpty()) { - context.currentUnionOperators.clear(); - context.workWithUnionOperators.add(work); - } - - // We're scanning a tree from roots to leaf (this is not technically - // correct, demux and mux operators might form a diamond shape, but - // we will only scan one path and ignore the others, because the - // diamond shape is always contained in a single vertex). The scan - // is depth first and because we remove parents when we pack a pipeline - // into a vertex we will never visit any node twice. But because of that - // we might have a situation where we need to connect 'work' that comes after - // the 'work' we're currently looking at. - // - // Also note: the concept of leaf and root is reversed in hive for historical - // reasons. Roots are data sources, leaves are data sinks. I know. - if (context.leafOpToFollowingWorkInfo.containsKey(operator)) { - ObjectPair childWorkInfo = context. - leafOpToFollowingWorkInfo.get(operator); - SparkEdgeProperty edgeProp = childWorkInfo.getFirst(); - ReduceWork childWork = childWorkInfo.getSecond(); - - LOG.debug("Second pass. Leaf operator: " + operator + " has common downstream work:" + childWork); - - // We may have already connected `work` with `childWork`, in case, for example, lateral view: - // TS - // | - // ... - // | - // LVF - // | \ - // SEL SEL - // | | - // LVJ-UDTF - // | - // SEL - // | - // RS - // Here, RS can be reached from TS via two different paths. If there is any child work after RS, - // we don't want to connect them with the work associated with TS more than once. - if (sparkWork.getEdgeProperty(work, childWork) == null) { - sparkWork.connect(work, childWork, edgeProp); - } else { - LOG.debug("work " + work.getName() + " is already connected to " + childWork.getName() + " before"); - } - } else { - LOG.debug("First pass. Leaf operator: " + operator); - } - - // No children means we're at the bottom. If there are more operators to scan - // the next item will be a new root. - if (!operator.getChildOperators().isEmpty()) { - Preconditions.checkArgument(operator.getChildOperators().size() == 1, - "AssertionError: expected operator.getChildOperators().size() to be 1, but was " - + operator.getChildOperators().size()); - context.parentOfRoot = operator; - context.currentRootOperator = operator.getChildOperators().get(0); - context.preceedingWork = work; - } - - return null; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java deleted file mode 100644 index ae614633f36..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWorkWalker.java +++ /dev/null @@ -1,114 +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.hadoop.hive.ql.parse.spark; - -import java.util.Collection; -import java.util.HashMap; -import java.util.List; - -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; - -/** - * Walks the operator tree in DFS fashion. - * - * Cloned from GenTezWorkWarlker. - */ -public class GenSparkWorkWalker extends DefaultGraphWalker { - private final GenSparkProcContext ctx; - - /** - * constructor of the walker - the dispatcher is passed. - * - * @param disp the dispatcher to be called for each node visited - * @param ctx the context where we'll set the current root operator - * - */ - public GenSparkWorkWalker(Dispatcher disp, GenSparkProcContext ctx) { - super(disp); - this.ctx = ctx; - } - - @SuppressWarnings("unchecked") - private void setRoot(Node nd) { - ctx.currentRootOperator = (Operator) nd; - ctx.preceedingWork = null; - ctx.parentOfRoot = null; - } - - /** - * starting point for walking. - * - * @throws SemanticException - */ - @Override - public void startWalking(Collection startNodes, HashMap nodeOutput) - throws SemanticException { - toWalk.addAll(startNodes); - while (toWalk.size() > 0) { - Node nd = toWalk.remove(0); - setRoot(nd); - walk(nd); - if (nodeOutput != null) { - nodeOutput.put(nd, retMap.get(nd)); - } - } - } - - /** - * Walk the given operator. - * - * @param nd operator being walked - */ - @Override - protected void walk(Node nd) throws SemanticException { - List children = nd.getChildren(); - - // maintain the stack of operators encountered - opStack.push(nd); - Boolean skip = dispatchAndReturn(nd, opStack); - - // save some positional state - Operator currentRoot = ctx.currentRootOperator; - Operator parentOfRoot = ctx.parentOfRoot; - BaseWork preceedingWork = ctx.preceedingWork; - - if (skip == null || !skip) { - // move all the children to the front of queue - for (Node ch : children) { - - // and restore the state before walking each child - ctx.currentRootOperator = currentRoot; - ctx.parentOfRoot = parentOfRoot; - ctx.preceedingWork = preceedingWork; - - walk(ch); - } - } - - // done with this operator - opStack.pop(); - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java deleted file mode 100644 index 93deee1d4d7..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/OptimizeSparkProcContext.java +++ /dev/null @@ -1,80 +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.hadoop.hive.ql.parse.spark; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.hooks.ReadEntity; -import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; - -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** - * OptimizeSparkProcContext. OptimizeSparkProcContext maintains information - * about the current operator plan as we walk the operator tree - * to do some additional optimizations on it. clone from OptimizeTezProcContext. - * - */ -public class OptimizeSparkProcContext implements NodeProcessorCtx { - - private final ParseContext parseContext; - private final HiveConf conf; - private final Set inputs; - private final Set outputs; - private final Set visitedReduceSinks = new HashSet(); - private final Map mjOpSizes = new HashMap(); - - public OptimizeSparkProcContext(HiveConf conf, ParseContext parseContext, - Set inputs, Set outputs) { - this.conf = conf; - this.parseContext = parseContext; - this.inputs = inputs; - this.outputs = outputs; - } - - public ParseContext getParseContext() { - return parseContext; - } - - public HiveConf getConf() { - return conf; - } - - public Set getInputs() { - return inputs; - } - - public Set getOutputs() { - return outputs; - } - - public Set getVisitedReduceSinks() { - return visitedReduceSinks; - } - - public Map getMjOpSizes() { - return mjOpSizes; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java deleted file mode 100644 index 0a76ffa28bd..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ /dev/null @@ -1,619 +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.hadoop.hive.ql.parse.spark; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.exec.FilterOperator; -import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.UnionOperator; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.hooks.ReadEntity; -import org.apache.hadoop.hive.ql.hooks.WriteEntity; -import org.apache.hadoop.hive.ql.lib.CompositeProcessor; -import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; -import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; -import org.apache.hadoop.hive.ql.lib.Dispatcher; -import org.apache.hadoop.hive.ql.lib.ForwardWalker; -import org.apache.hadoop.hive.ql.lib.GraphWalker; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.lib.PreOrderWalker; -import org.apache.hadoop.hive.ql.lib.Rule; -import org.apache.hadoop.hive.ql.lib.RuleRegExp; -import org.apache.hadoop.hive.ql.lib.TypeRule; -import org.apache.hadoop.hive.ql.log.PerfLogger; -import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate; -import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcCtx; -import org.apache.hadoop.hive.ql.optimizer.DynamicPartitionPruningOptimization; -import org.apache.hadoop.hive.ql.optimizer.SparkRemoveDynamicPruning; -import org.apache.hadoop.hive.ql.optimizer.metainfo.annotation.AnnotateWithOpTraits; -import org.apache.hadoop.hive.ql.optimizer.physical.AnnotateRunTimeStatsOptimizer; -import org.apache.hadoop.hive.ql.optimizer.physical.MetadataOnlyOptimizer; -import org.apache.hadoop.hive.ql.optimizer.physical.NullScanOptimizer; -import org.apache.hadoop.hive.ql.optimizer.physical.PhysicalContext; -import org.apache.hadoop.hive.ql.optimizer.physical.SparkCrossProductCheck; -import org.apache.hadoop.hive.ql.optimizer.physical.SparkDynamicPartitionPruningResolver; -import org.apache.hadoop.hive.ql.optimizer.physical.SparkMapJoinResolver; -import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger; -import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer; -import org.apache.hadoop.hive.ql.optimizer.spark.CombineEquivalentWorkResolver; -import org.apache.hadoop.hive.ql.optimizer.spark.SetSparkReducerParallelism; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkJoinHintOptimizer; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkJoinOptimizer; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkReduceSinkMapJoinProc; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkSkewJoinResolver; -import org.apache.hadoop.hive.ql.optimizer.spark.SplitSparkWorkResolver; -import org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics; -import org.apache.hadoop.hive.ql.parse.GlobalLimitCtx; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.parse.TaskCompiler; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.MoveWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.session.SessionState; - -/** - * SparkCompiler translates the operator plan into SparkTasks. - * - * Cloned from TezCompiler. - */ -public class SparkCompiler extends TaskCompiler { - private static final String CLASS_NAME = SparkCompiler.class.getName(); - private static final PerfLogger PERF_LOGGER = SessionState.getPerfLogger(); - - public SparkCompiler() { - } - - @Override - protected void optimizeOperatorPlan(ParseContext pCtx, Set inputs, - Set outputs) throws SemanticException { - PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); - - OptimizeSparkProcContext procCtx = new OptimizeSparkProcContext(conf, pCtx, inputs, outputs); - - // Run Spark Dynamic Partition Pruning - runDynamicPartitionPruning(procCtx); - - // Annotation OP tree with statistics - runStatsAnnotation(procCtx); - - // Set reducer parallelism - runSetReducerParallelism(procCtx); - - // Run Join releated optimizations - runJoinOptimizations(procCtx); - - if(conf.isSparkDPPAny()){ - // Remove DPP based on expected size of the output data - runRemoveDynamicPruning(procCtx); - - // Remove cyclic dependencies for DPP - runCycleAnalysisForPartitionPruning(procCtx); - - // Remove nested DPPs - SparkUtilities.removeNestedDPP(procCtx); - } - - // Re-run constant propagation so we fold any new constants introduced by the operator optimizers - // Specifically necessary for DPP because we might have created lots of "and true and true" conditions - if (procCtx.getConf().getBoolVar(HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) { - new ConstantPropagate(ConstantPropagateProcCtx.ConstantPropagateOption.SHORTCUT).transform(pCtx); - } - - PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE); - } - - private void runRemoveDynamicPruning(OptimizeSparkProcContext procCtx) throws SemanticException { - ParseContext pCtx = procCtx.getParseContext(); - Map opRules = new LinkedHashMap(); - - opRules.put(new RuleRegExp("Disabling Dynamic Partition Pruning", - SparkPartitionPruningSinkOperator.getOperatorName() + "%"), - new SparkRemoveDynamicPruning()); - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new DefaultGraphWalker(disp); - - // Create a list of topop nodes - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); - ogw.startWalking(topNodes, null); - } - - private void runCycleAnalysisForPartitionPruning(OptimizeSparkProcContext procCtx) { - - boolean cycleFree = false; - while (!cycleFree) { - cycleFree = true; - Set>> components = getComponents(procCtx); - for (Set> component : components) { - if (LOG.isDebugEnabled()) { - LOG.debug("Component: "); - for (Operator co : component) { - LOG.debug("Operator: " + co.getName() + ", " + co.getIdentifier()); - } - } - if (component.size() != 1) { - LOG.info("Found cycle in operator plan..."); - cycleFree = false; - removeDPPOperator(component, procCtx); - break; - } - } - LOG.info("Cycle free: " + cycleFree); - } - } - - private void removeDPPOperator(Set> component, OptimizeSparkProcContext context) { - SparkPartitionPruningSinkOperator toRemove = null; - for (Operator o : component) { - if (o instanceof SparkPartitionPruningSinkOperator) { - // we want to remove the DPP with bigger data size - if (toRemove == null - || o.getConf().getStatistics().getDataSize() > toRemove.getConf().getStatistics() - .getDataSize()) { - toRemove = (SparkPartitionPruningSinkOperator) o; - } - } - } - - if (toRemove == null) { - return; - } - - OperatorUtils.removeBranch(toRemove); - // at this point we've found the fork in the op pipeline that has the pruning as a child plan. - LOG.info("Disabling dynamic pruning for: " - + toRemove.getConf().getTableScanNames() + ". Needed to break cyclic dependency"); - } - - // Tarjan's algo - private Set>> getComponents(OptimizeSparkProcContext procCtx) { - AtomicInteger index = new AtomicInteger(); - Map, Integer> indexes = new HashMap, Integer>(); - Map, Integer> lowLinks = new HashMap, Integer>(); - Stack> nodes = new Stack>(); - Set>> components = new HashSet>>(); - - for (Operator o : procCtx.getParseContext().getTopOps().values()) { - if (!indexes.containsKey(o)) { - connect(o, index, nodes, indexes, lowLinks, components); - } - } - return components; - } - - private void connect(Operator o, AtomicInteger index, Stack> nodes, - Map, Integer> indexes, Map, Integer> lowLinks, - Set>> components) { - - indexes.put(o, index.get()); - lowLinks.put(o, index.get()); - index.incrementAndGet(); - nodes.push(o); - - List> children; - if (o instanceof SparkPartitionPruningSinkOperator) { - children = new ArrayList<>(); - children.addAll(o.getChildOperators()); - SparkPartitionPruningSinkDesc dppDesc = ((SparkPartitionPruningSinkOperator) o).getConf(); - for (SparkPartitionPruningSinkDesc.DPPTargetInfo targetInfo : dppDesc.getTargetInfos()) { - TableScanOperator ts = targetInfo.tableScan; - LOG.debug("Adding special edge: " + o.getName() + " --> " + ts.toString()); - children.add(ts); - } - } else { - children = o.getChildOperators(); - } - - for (Operator child : children) { - if (!indexes.containsKey(child)) { - connect(child, index, nodes, indexes, lowLinks, components); - lowLinks.put(o, Math.min(lowLinks.get(o), lowLinks.get(child))); - } else if (nodes.contains(child)) { - lowLinks.put(o, Math.min(lowLinks.get(o), indexes.get(child))); - } - } - - if (lowLinks.get(o).equals(indexes.get(o))) { - Set> component = new HashSet>(); - components.add(component); - Operator current; - do { - current = nodes.pop(); - component.add(current); - } while (current != o); - } - } - - private void runStatsAnnotation(OptimizeSparkProcContext procCtx) throws SemanticException { - new AnnotateWithStatistics().transform(procCtx.getParseContext()); - new AnnotateWithOpTraits().transform(procCtx.getParseContext()); - } - - private void runDynamicPartitionPruning(OptimizeSparkProcContext procCtx) - throws SemanticException { - if (!conf.isSparkDPPAny()) { - return; - } - - ParseContext parseContext = procCtx.getParseContext(); - Map opRules = new LinkedHashMap(); - opRules.put( - new RuleRegExp(new String("Dynamic Partition Pruning"), - FilterOperator.getOperatorName() + "%"), - new DynamicPartitionPruningOptimization()); - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new ForwardWalker(disp); - - List topNodes = new ArrayList(); - topNodes.addAll(parseContext.getTopOps().values()); - ogw.startWalking(topNodes, null); - } - - private void runSetReducerParallelism(OptimizeSparkProcContext procCtx) throws SemanticException { - ParseContext pCtx = procCtx.getParseContext(); - Map opRules = new LinkedHashMap(); - opRules.put(new RuleRegExp("Set parallelism - ReduceSink", - ReduceSinkOperator.getOperatorName() + "%"), - new SetSparkReducerParallelism(pCtx.getConf())); - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new PreOrderWalker(disp); - - // Create a list of topop nodes - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); - ogw.startWalking(topNodes, null); - } - - private void runJoinOptimizations(OptimizeSparkProcContext procCtx) throws SemanticException { - ParseContext pCtx = procCtx.getParseContext(); - Map opRules = new LinkedHashMap(); - - opRules.put(new TypeRule(JoinOperator.class), new SparkJoinOptimizer(pCtx)); - - opRules.put(new TypeRule(MapJoinOperator.class), new SparkJoinHintOptimizer(pCtx)); - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new DefaultGraphWalker(disp); - - // Create a list of topop nodes - ArrayList topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); - ogw.startWalking(topNodes, null); - } - - /** - * TODO: need to turn on rules that's commented out and add more if necessary. - */ - @Override - protected void generateTaskTree(List> rootTasks, ParseContext pCtx, - List> mvTask, Set inputs, Set outputs) - throws SemanticException { - PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); - - GenSparkUtils utils = GenSparkUtils.getUtils(); - utils.resetSequenceNumber(); - - ParseContext tempParseContext = getParseContext(pCtx, rootTasks); - GenSparkProcContext procCtx = new GenSparkProcContext( - conf, tempParseContext, mvTask, rootTasks, inputs, outputs, pCtx.getTopOps()); - - // -------------------------------- First Pass ---------------------------------- // - // Identify SparkPartitionPruningSinkOperators, and break OP tree if necessary - - Map opRules = new LinkedHashMap(); - opRules.put(new RuleRegExp("Clone OP tree for PartitionPruningSink", - SparkPartitionPruningSinkOperator.getOperatorName() + "%"), - new SplitOpTreeForDPP()); - - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx); - - List topNodes = new ArrayList(); - topNodes.addAll(pCtx.getTopOps().values()); - ogw.startWalking(topNodes, null); - - // -------------------------------- Second Pass ---------------------------------- // - // Process operator tree in two steps: first we process the extra op trees generated - // in the first pass. Then we process the main op tree, and the result task will depend - // on the task generated in the first pass. - topNodes.clear(); - topNodes.addAll(procCtx.topOps.values()); - generateTaskTreeHelper(procCtx, topNodes); - - // If this set is not empty, it means we need to generate a separate task for collecting - // the partitions used. - if (!procCtx.clonedPruningTableScanSet.isEmpty()) { - SparkTask pruningTask = SparkUtilities.createSparkTask(conf); - SparkTask mainTask = procCtx.currentTask; - pruningTask.addDependentTask(procCtx.currentTask); - procCtx.rootTasks.remove(procCtx.currentTask); - procCtx.rootTasks.add(pruningTask); - procCtx.currentTask = pruningTask; - - topNodes.clear(); - topNodes.addAll(procCtx.clonedPruningTableScanSet); - generateTaskTreeHelper(procCtx, topNodes); - - procCtx.currentTask = mainTask; - } - - // -------------------------------- Post Pass ---------------------------------- // - - // we need to clone some operator plans and remove union operators still - for (BaseWork w : procCtx.workWithUnionOperators) { - GenSparkUtils.getUtils().removeUnionOperators(procCtx, w); - } - - // we need to fill MapWork with 'local' work and bucket information for SMB Join. - GenSparkUtils.getUtils().annotateMapWork(procCtx); - - // finally make sure the file sink operators are set up right - for (FileSinkOperator fileSink : procCtx.fileSinkSet) { - GenSparkUtils.getUtils().processFileSink(procCtx, fileSink); - } - - // Process partition pruning sinks - for (Operator prunerSink : procCtx.pruningSinkSet) { - utils.processPartitionPruningSink(procCtx, (SparkPartitionPruningSinkOperator) prunerSink); - } - - PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_GENERATE_TASK_TREE); - } - - private void generateTaskTreeHelper(GenSparkProcContext procCtx, List topNodes) - throws SemanticException { - // create a walker which walks the tree in a DFS manner while maintaining - // the operator stack. The dispatcher generates the plan from the operator tree - Map opRules = new LinkedHashMap(); - GenSparkWork genSparkWork = new GenSparkWork(GenSparkUtils.getUtils()); - - opRules.put(new RuleRegExp("Split Work - ReduceSink", - ReduceSinkOperator.getOperatorName() + "%"), genSparkWork); - - opRules.put(new RuleRegExp("Split Work - SparkPartitionPruningSink", - SparkPartitionPruningSinkOperator.getOperatorName() + "%"), genSparkWork); - - opRules.put(new TypeRule(MapJoinOperator.class), new SparkReduceSinkMapJoinProc()); - - opRules.put(new RuleRegExp("Split Work + Move/Merge - FileSink", - FileSinkOperator.getOperatorName() + "%"), - new CompositeProcessor(new SparkFileSinkProcessor(), genSparkWork)); - - opRules.put(new RuleRegExp("Handle Analyze Command", - TableScanOperator.getOperatorName() + "%"), - new SparkProcessAnalyzeTable(GenSparkUtils.getUtils())); - - opRules.put(new RuleRegExp("Remember union", UnionOperator.getOperatorName() + "%"), - new NodeProcessor() { - @Override - public Object process(Node n, Stack s, - NodeProcessorCtx procCtx, Object... os) throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procCtx; - UnionOperator union = (UnionOperator) n; - - // simply need to remember that we've seen a union. - context.currentUnionOperators.add(union); - return null; - } - } - ); - - /** - * SMB join case: (Big) (Small) (Small) - * TS TS TS - * \ | / - * \ DS DS - * \ | / - * SMBJoinOP - * - * Some of the other processors are expecting only one traversal beyond SMBJoinOp. - * We need to traverse from the big-table path only, and stop traversing on the - * small-table path once we reach SMBJoinOp. - * Also add some SMB join information to the context, so we can properly annotate - * the MapWork later on. - */ - opRules.put(new TypeRule(SMBMapJoinOperator.class), - new NodeProcessor() { - @Override - public Object process(Node currNode, Stack stack, - NodeProcessorCtx procCtx, Object... os) throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procCtx; - SMBMapJoinOperator currSmbNode = (SMBMapJoinOperator) currNode; - SparkSMBMapJoinInfo smbMapJoinCtx = context.smbMapJoinCtxMap.get(currSmbNode); - if (smbMapJoinCtx == null) { - smbMapJoinCtx = new SparkSMBMapJoinInfo(); - context.smbMapJoinCtxMap.put(currSmbNode, smbMapJoinCtx); - } - - for (Node stackNode : stack) { - if (stackNode instanceof DummyStoreOperator) { - //If coming from small-table side, do some book-keeping, and skip traversal. - smbMapJoinCtx.smallTableRootOps.add(context.currentRootOperator); - return true; - } - } - //If coming from big-table side, do some book-keeping, and continue traversal - smbMapJoinCtx.bigTableRootOp = context.currentRootOperator; - return false; - } - } - ); - - // The dispatcher fires the processor corresponding to the closest matching - // rule and passes the context along - Dispatcher disp = new DefaultRuleDispatcher(null, opRules, procCtx); - GraphWalker ogw = new GenSparkWorkWalker(disp, procCtx); - ogw.startWalking(topNodes, null); - } - - @Override - protected void setInputFormat(Task task) { - if (task instanceof SparkTask) { - SparkWork work = ((SparkTask)task).getWork(); - List all = work.getAllWork(); - for (BaseWork w: all) { - if (w instanceof MapWork) { - MapWork mapWork = (MapWork) w; - HashMap> opMap = mapWork.getAliasToWork(); - if (!opMap.isEmpty()) { - for (Operator op : opMap.values()) { - setInputFormat(mapWork, op); - } - } - } - } - } else if (task instanceof ConditionalTask) { - List> listTasks - = ((ConditionalTask) task).getListTasks(); - for (Task tsk : listTasks) { - setInputFormat(tsk); - } - } - - if (task.getChildTasks() != null) { - for (Task childTask : task.getChildTasks()) { - setInputFormat(childTask); - } - } - } - - private void setInputFormat(MapWork work, Operator op) { - if (op.isUseBucketizedHiveInputFormat()) { - work.setUseBucketizedHiveInputFormat(true); - return; - } - - if (op.getChildOperators() != null) { - for (Operator childOp : op.getChildOperators()) { - setInputFormat(work, childOp); - } - } - } - - @Override - protected void decideExecMode(List> rootTasks, Context ctx, - GlobalLimitCtx globalLimitCtx) throws SemanticException { - // currently all Spark work is on the cluster - return; - } - - @Override - protected void optimizeTaskPlan(List> rootTasks, ParseContext pCtx, - Context ctx) throws SemanticException { - PERF_LOGGER.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); - PhysicalContext physicalCtx = new PhysicalContext(conf, pCtx, pCtx.getContext(), rootTasks, - pCtx.getFetchTask()); - - physicalCtx = new SplitSparkWorkResolver().resolve(physicalCtx); - - if (conf.getBoolVar(HiveConf.ConfVars.HIVESKEWJOIN)) { - (new SparkSkewJoinResolver()).resolve(physicalCtx); - } else { - LOG.debug("Skipping runtime skew join optimization"); - } - - physicalCtx = new SparkMapJoinResolver().resolve(physicalCtx); - - if (conf.isSparkDPPAny()) { - physicalCtx = new SparkDynamicPartitionPruningResolver().resolve(physicalCtx); - } - - if (conf.getBoolVar(HiveConf.ConfVars.HIVENULLSCANOPTIMIZE)) { - physicalCtx = new NullScanOptimizer().resolve(physicalCtx); - } else { - LOG.debug("Skipping null scan query optimization"); - } - - if (conf.getBoolVar(HiveConf.ConfVars.HIVEMETADATAONLYQUERIES)) { - physicalCtx = new MetadataOnlyOptimizer().resolve(physicalCtx); - } else { - LOG.debug("Skipping metadata only query optimization"); - } - - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CHECK_CROSS_PRODUCT)) { - physicalCtx = new SparkCrossProductCheck().resolve(physicalCtx); - } else { - LOG.debug("Skipping cross product analysis"); - } - - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED)) { - (new Vectorizer()).resolve(physicalCtx); - } else { - LOG.debug("Skipping vectorization"); - } - - if (!"none".equalsIgnoreCase(conf.getVar(HiveConf.ConfVars.HIVESTAGEIDREARRANGE))) { - (new StageIDsRearranger()).resolve(physicalCtx); - } else { - LOG.debug("Skipping stage id rearranger"); - } - - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_COMBINE_EQUIVALENT_WORK_OPTIMIZATION)) { - new CombineEquivalentWorkResolver().resolve(physicalCtx); - } else { - LOG.debug("Skipping combine equivalent work optimization"); - } - - if (physicalCtx.getContext().getExplainAnalyze() != null) { - new AnnotateRunTimeStatsOptimizer().resolve(physicalCtx); - } - - PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE); - return; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java deleted file mode 100644 index 441425082dd..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkFileSinkProcessor.java +++ /dev/null @@ -1,57 +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.hadoop.hive.ql.parse.spark; - -import java.util.Stack; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.FileSinkOperator; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.parse.SemanticException; - -/** - * FileSinkProcessor handles addition of merge, move and stats tasks for filesinks. - * Cloned from tez's FileSinkProcessor. - */ -public class SparkFileSinkProcessor implements NodeProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(SparkFileSinkProcessor.class.getName()); - - /* - * (non-Javadoc) - * we should ideally not modify the tree we traverse. - * However, since we need to walk the tree at any time when we modify the operator, - * we might as well do it here. - */ - @Override - public Object process(Node nd, Stack stack, - NodeProcessorCtx procCtx, Object... nodeOutputs) - throws SemanticException { - - GenSparkProcContext context = (GenSparkProcContext) procCtx; - FileSinkOperator fileSink = (FileSinkOperator) nd; - - // just remember it for later processing - context.fileSinkSet.add(fileSink); - return true; - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java deleted file mode 100644 index 1de7a45ab2a..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java +++ /dev/null @@ -1,298 +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.hadoop.hive.ql.parse.spark; - -import java.io.BufferedOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc; -import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.ql.plan.api.OperatorType; -import org.apache.hadoop.hive.serde2.Serializer; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.util.ReflectionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.annotations.VisibleForTesting; - -/** - * This operator gets partition info from the upstream operators, and write them - * to HDFS. This will later be read at the driver, and used for pruning the partitions - * for the big table side. - */ -public class SparkPartitionPruningSinkOperator extends Operator { - - @SuppressWarnings("deprecation") - protected transient Serializer serializer; - protected transient DataOutputBuffer buffer; - protected static final Logger LOG = LoggerFactory.getLogger(SparkPartitionPruningSinkOperator.class); - private static final AtomicLong SEQUENCE_NUM = new AtomicLong(0); - - private transient String uniqueId = null; - - /** Kryo ctor. */ - @VisibleForTesting - public SparkPartitionPruningSinkOperator() { - super(); - } - - public SparkPartitionPruningSinkOperator(CompilationOpContext ctx) { - super(ctx); - } - - @Override - @SuppressWarnings("deprecation") - public void initializeOp(Configuration hconf) throws HiveException { - super.initializeOp(hconf); - serializer = (Serializer) ReflectionUtils.newInstance( - conf.getTable().getDeserializerClass(), null); - buffer = new DataOutputBuffer(); - } - - @Override - public void process(Object row, int tag) throws HiveException { - ObjectInspector rowInspector = inputObjInspectors[0]; - try { - Writable writableRow = serializer.serialize(row, rowInspector); - writableRow.write(buffer); - } catch (Exception e) { - throw new HiveException(e); - } - } - - @Override - public void closeOp(boolean abort) throws HiveException { - if (!abort) { - try { - flushToFile(); - } catch (Exception e) { - throw new HiveException(e); - } - } - } - - /* This function determines whether sparkpruningsink is with mapjoin. This will be called - to check whether the tree should be split for dpp. For mapjoin it won't be. Also called - to determine whether dpp should be enabled for anything other than mapjoin. - */ - public boolean isWithMapjoin() { - Operator branchingOp = this.getBranchingOp(); - - // Check if this is a MapJoin. If so, do not split. - for (Operator childOp : branchingOp.getChildOperators()) { - if (childOp instanceof ReduceSinkOperator && - childOp.getChildOperators().get(0) instanceof MapJoinOperator) { - return true; - } - } - - return false; - } - - /* Locate the op where the branch starts. This function works only for the following pattern. - * TS1 TS2 - * | | - * FIL FIL - * | | - * | --------- - * RS | | | - * | RS SEL SEL - * | / | | - * | / GBY GBY - * JOIN | | - * | SPARKPRUNINGSINK - * | - * SPARKPRUNINGSINK - */ - public Operator getBranchingOp() { - Operator branchingOp = this; - - while (branchingOp != null) { - if (branchingOp.getNumChild() > 1) { - break; - } else { - branchingOp = branchingOp.getParentOperators().get(0); - } - } - - return branchingOp; - } - - private void flushToFile() throws IOException { - // write an intermediate file to the specified path - // the format of the path is: tmpPath/targetWorkId/sourceWorkId/randInt - Path path = conf.getPath(); - FileSystem fs = path.getFileSystem(this.getConfiguration()); - fs.mkdirs(path); - - while (true) { - path = new Path(path, String.valueOf(Utilities.randGen.nextInt())); - if (!fs.exists(path)) { - break; - } - } - - short numOfRepl = fs.getDefaultReplication(path); - - ObjectOutputStream out = null; - FSDataOutputStream fsout = null; - - try { - fsout = fs.create(path, numOfRepl); - out = new ObjectOutputStream(new BufferedOutputStream(fsout)); - out.writeInt(conf.getTargetInfos().size()); - for (SparkPartitionPruningSinkDesc.DPPTargetInfo info : conf.getTargetInfos()) { - out.writeUTF(info.columnName); - } - buffer.writeTo(out); - } catch (Exception e) { - try { - fs.delete(path, false); - } catch (Exception ex) { - LOG.warn("Exception happened while trying to clean partial file."); - } - throw e; - } finally { - if (out != null) { - LOG.info("Flushed to file: " + path); - out.close(); - } else if (fsout != null) { - fsout.close(); - } - } - } - - @Override - public OperatorType getType() { - return OperatorType.SPARKPRUNINGSINK; - } - - @Override - public String getName() { - return SparkPartitionPruningSinkOperator.getOperatorName(); - } - - public static String getOperatorName() { - return "SPARKPRUNINGSINK"; - } - - public synchronized String getUniqueId() { - if (uniqueId == null) { - uniqueId = getOperatorId() + "_" + SEQUENCE_NUM.getAndIncrement(); - } - return uniqueId; - } - - public synchronized void setUniqueId(String uniqueId) { - this.uniqueId = uniqueId; - } - - /** - * Add this DPP sink as a pruning source for the target MapWork. It means the DPP sink's output - * will be used to prune a certain partition in the MapWork. The MapWork's event source maps will - * be updated to remember the DPP sink's unique ID and corresponding target columns. - */ - public void addAsSourceEvent(MapWork mapWork, ExprNodeDesc partKey, String columnName, - String columnType) { - String sourceId = getUniqueId(); - SparkPartitionPruningSinkDesc conf = getConf(); - - // store table descriptor in map-targetWork - List tableDescs = mapWork.getEventSourceTableDescMap().computeIfAbsent(sourceId, - v -> new ArrayList<>()); - tableDescs.add(conf.getTable()); - - // store partition key expr in map-targetWork - List partKeys = mapWork.getEventSourcePartKeyExprMap().computeIfAbsent(sourceId, - v -> new ArrayList<>()); - partKeys.add(partKey); - - // store column name in map-targetWork - List columnNames = mapWork.getEventSourceColumnNameMap().computeIfAbsent(sourceId, - v -> new ArrayList<>()); - columnNames.add(columnName); - - List columnTypes = mapWork.getEventSourceColumnTypeMap().computeIfAbsent(sourceId, - v -> new ArrayList<>()); - columnTypes.add(columnType); - } - - /** - * Remove this DPP sink from the target MapWork's pruning source. The MapWork's event source maps - * will be updated to remove the association between the target column and the DPP sink's unique - * ID. If the DPP sink has no target columns after the removal, its unique ID is removed from the - * event source maps. - */ - public void removeFromSourceEvent(MapWork mapWork, ExprNodeDesc partKey, String columnName, - String columnType) { - String sourceId = getUniqueId(); - SparkPartitionPruningSinkDesc conf = getConf(); - - List tableDescs = mapWork.getEventSourceTableDescMap().get(sourceId); - if (tableDescs != null) { - tableDescs.remove(conf.getTable()); - if (tableDescs.isEmpty()) { - mapWork.getEventSourceTableDescMap().remove(sourceId); - } - } - - List partKeys = mapWork.getEventSourcePartKeyExprMap().get(sourceId); - if (partKeys != null) { - partKeys.remove(partKey); - if (partKeys.isEmpty()) { - mapWork.getEventSourcePartKeyExprMap().remove(sourceId); - } - } - - List columnNames = mapWork.getEventSourceColumnNameMap().get(sourceId); - if (columnNames != null) { - columnNames.remove(columnName); - if (columnNames.isEmpty()) { - mapWork.getEventSourceColumnNameMap().remove(sourceId); - } - } - - List columnTypes = mapWork.getEventSourceColumnTypeMap().get(sourceId); - if (columnTypes != null) { - columnTypes.remove(columnType); - if (columnTypes.isEmpty()) { - mapWork.getEventSourceColumnTypeMap().remove(sourceId); - } - } - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java deleted file mode 100644 index 28d4de7f7bb..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java +++ /dev/null @@ -1,159 +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.hadoop.hive.ql.parse.spark; - -import java.util.List; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.TaskFactory; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.metadata.Partition; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; -import org.apache.hadoop.hive.ql.parse.ParseContext; -import org.apache.hadoop.hive.ql.parse.PrunedPartitionList; -import org.apache.hadoop.hive.ql.parse.SemanticException; -import org.apache.hadoop.hive.ql.plan.StatsWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hadoop.hive.ql.plan.BasicStatsWork; -import org.apache.hadoop.mapred.InputFormat; - -import com.google.common.base.Preconditions; - -/** - * ProcessAnalyzeTable sets up work for the several variants of analyze table - * (normal, no scan) The plan at this point will be a single - * table scan operator. - * - * Cloned from Tez ProcessAnalyzeTable. - */ -public class SparkProcessAnalyzeTable implements NodeProcessor { - private static final Logger LOGGER = LoggerFactory.getLogger(SparkProcessAnalyzeTable.class.getName()); - - // shared plan utils for spark - private GenSparkUtils utils = null; - - /** - * Injecting the utils in the constructor facilitates testing. - */ - public SparkProcessAnalyzeTable(GenSparkUtils utils) { - this.utils = utils; - } - - @SuppressWarnings("unchecked") - @Override - public Object process(Node nd, Stack stack, - NodeProcessorCtx procContext, Object... nodeOutputs) throws SemanticException { - GenSparkProcContext context = (GenSparkProcContext) procContext; - - TableScanOperator tableScan = (TableScanOperator) nd; - - ParseContext parseContext = context.parseContext; - - Table table = tableScan.getConf().getTableMetadata(); - @SuppressWarnings("rawtypes") - Class inputFormat = table.getInputFormatClass(); - - if (parseContext.getQueryProperties().isAnalyzeCommand()) { - Preconditions.checkArgument(tableScan.getChildOperators() == null - || tableScan.getChildOperators().size() == 0, - "AssertionError: expected tableScan.getChildOperators() to be null, " - + "or tableScan.getChildOperators().size() to be 0"); - - String alias = null; - for (String a: parseContext.getTopOps().keySet()) { - if (tableScan == parseContext.getTopOps().get(a)) { - alias = a; - } - } - Preconditions.checkArgument(alias != null, "AssertionError: expected alias to be not null"); - - SparkWork sparkWork = context.currentTask.getWork(); - if (OrcInputFormat.class.isAssignableFrom(inputFormat) || - MapredParquetInputFormat.class.isAssignableFrom(inputFormat)) { - // For ORC & Parquet, all the following statements are the same - // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS - // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS noscan; - // There will not be any Spark job above this task - StatsWork statWork = new StatsWork(table, parseContext.getConf()); - statWork.setFooterScan(); - // If partition is specified, get pruned partition list - Set confirmedParts = GenMapRedUtils.getConfirmedPartitionsForScan(tableScan); - if (confirmedParts.size() > 0) { - List partCols = GenMapRedUtils.getPartitionColumns(tableScan); - PrunedPartitionList partList = new PrunedPartitionList(table, confirmedParts, partCols, false); - statWork.addInputPartitions(partList.getPartitions()); - } - Task snjTask = TaskFactory.get(statWork); - snjTask.setParentTasks(null); - context.rootTasks.remove(context.currentTask); - context.rootTasks.add(snjTask); - return true; - } else { - - // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS; - // The plan consists of a simple SparkTask followed by a StatsTask. - // The Spark task is just a simple TableScanOperator - - BasicStatsWork basicStatsWork = new BasicStatsWork(table.getTableSpec()); - basicStatsWork.setIsExplicitAnalyze(true); - basicStatsWork.setNoScanAnalyzeCommand(parseContext.getQueryProperties().isNoScanAnalyzeCommand()); - StatsWork columnStatsWork = new StatsWork(table, basicStatsWork, parseContext.getConf()); - columnStatsWork.collectStatsFromAggregator(tableScan.getConf()); - columnStatsWork.setSourceTask(context.currentTask); - Task statsTask = TaskFactory.get(columnStatsWork); - context.currentTask.addDependentTask(statsTask); - - // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS noscan; - // The plan consists of a StatsTask only. - if (parseContext.getQueryProperties().isNoScanAnalyzeCommand()) { - statsTask.setParentTasks(null); - context.rootTasks.remove(context.currentTask); - context.rootTasks.add(statsTask); - } - - // NOTE: here we should use the new partition predicate pushdown API to get a list of pruned list, - // and pass it to setTaskPlan as the last parameter - Set confirmedPartns = GenMapRedUtils.getConfirmedPartitionsForScan(tableScan); - PrunedPartitionList partitions = null; - if (confirmedPartns.size() > 0) { - List partCols = GenMapRedUtils.getPartitionColumns(tableScan); - partitions = new PrunedPartitionList(table, confirmedPartns, partCols, false); - } - - MapWork w = utils.createMapWork(context, tableScan, sparkWork, partitions); - w.setGatheringStats(true); - return true; - } - } - - return null; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkSMBMapJoinInfo.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkSMBMapJoinInfo.java deleted file mode 100644 index cac11e2ab0f..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkSMBMapJoinInfo.java +++ /dev/null @@ -1,33 +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.hadoop.hive.ql.parse.spark; - -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.plan.MapWork; - -import java.util.ArrayList; -import java.util.List; - -/** - * Data structure to keep track of SMBMapJoin operators during query compilation for Spark. - */ -public class SparkSMBMapJoinInfo { - Operator bigTableRootOp; - List> smallTableRootOps = new ArrayList>(); - MapWork mapWork; -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java deleted file mode 100644 index b9c2c1c33ba..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SplitOpTreeForDPP.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hive.ql.parse.spark; - -import java.util.ArrayList; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; -import java.util.Stack; - -import org.apache.hadoop.hive.ql.exec.Operator; -import org.apache.hadoop.hive.ql.exec.OperatorUtils; -import org.apache.hadoop.hive.ql.exec.SerializationUtilities; -import org.apache.hadoop.hive.ql.exec.TableScanOperator; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; -import org.apache.hadoop.hive.ql.lib.Node; -import org.apache.hadoop.hive.ql.lib.NodeProcessor; -import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; -import org.apache.hadoop.hive.ql.parse.SemanticException; - -import com.google.common.base.Preconditions; - - -/** - * This processor triggers on SparkPartitionPruningSinkOperator. For a operator tree like - * this: - * - * Original Tree: - * TS1 TS2 - * | | - * FIL FIL - * | | - * RS / \ \ - * | | \ \ - * | RS SEL SEL - * \ / | | - * JOIN GBY GBY - * | | - * | SPARKPRUNINGSINK - * | - * SPARKPRUNINGSINK - * - * It removes the branch containing SPARKPRUNINGSINK from the original operator tree, and splits it into - * two separate trees: - * Tree #1: Tree #2 - * TS1 TS2 TS2 - * | | | - * FIL FIL FIL - * | | |_____ - * RS SEL | \ - * | | SEL SEL - * | RS | | - * \ / GBY GBY - * JOIN | | - * | SPARKPRUNINGSINK - * SPARKPRUNINGSINK - - * For MapJoinOperator, this optimizer will not do anything - it should be executed within - * the same SparkTask. - */ -public class SplitOpTreeForDPP implements NodeProcessor { - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, - Object... nodeOutputs) throws SemanticException { - SparkPartitionPruningSinkOperator pruningSinkOp = (SparkPartitionPruningSinkOperator) nd; - GenSparkProcContext context = (GenSparkProcContext) procCtx; - - for (Operator op : context.pruningSinkSet) { - if (pruningSinkOp.getOperatorId().equals(op.getOperatorId())) { - return null; - } - } - - // If pruning sink operator is with map join, then pruning sink need not be split to a - // separate tree. Add the pruning sink operator to context and return - if (pruningSinkOp.isWithMapjoin()) { - context.pruningSinkSet.add(pruningSinkOp); - return null; - } - - List> roots = new LinkedList>(); - collectRoots(roots, pruningSinkOp); - - Operator branchingOp = pruningSinkOp.getBranchingOp(); - String marker = "SPARK_DPP_BRANCH_POINT_" + branchingOp.getOperatorId(); - branchingOp.setMarker(marker); - List> savedChildOps = branchingOp.getChildOperators(); - List> firstNodesOfPruningBranch = findFirstNodesOfPruningBranch(branchingOp); - branchingOp.setChildOperators(null); - - // Now clone the tree above selOp - List> newRoots = SerializationUtilities.cloneOperatorTree(roots); - for (int i = 0; i < roots.size(); i++) { - TableScanOperator newTs = (TableScanOperator) newRoots.get(i); - TableScanOperator oldTs = (TableScanOperator) roots.get(i); - newTs.getConf().setTableMetadata(oldTs.getConf().getTableMetadata()); - } - context.clonedPruningTableScanSet.addAll(newRoots); - - Operator newBranchingOp = null; - for (int i = 0; i < newRoots.size() && newBranchingOp == null; i++) { - newBranchingOp = OperatorUtils.findOperatorByMarker(newRoots.get(i), marker); - } - Preconditions.checkNotNull(newBranchingOp, - "Cannot find the branching operator in cloned tree."); - newBranchingOp.setChildOperators(firstNodesOfPruningBranch); - - // Restore broken links between operators, and remove the branch from the original tree - branchingOp.setChildOperators(savedChildOps); - for (Operator selOp : firstNodesOfPruningBranch) { - branchingOp.removeChild(selOp); - } - - Set> sinkSet = new LinkedHashSet<>(); - for (Operator sel : firstNodesOfPruningBranch) { - SparkUtilities.collectOp(sinkSet, sel, SparkPartitionPruningSinkOperator.class); - sel.setParentOperators(Utilities.makeList(newBranchingOp)); - } - context.pruningSinkSet.addAll(sinkSet); - return null; - } - - //find operators which are the children of specified filterOp and there are SparkPartitionPruningSink in these - //branches. - private List> findFirstNodesOfPruningBranch(Operator branchingOp) { - List> res = new ArrayList<>(); - for (Operator child : branchingOp.getChildOperators()) { - if (SparkUtilities.isDirectDPPBranch(child)) { - res.add(child); - } - } - return res; - } - - /** - * Recursively collect all roots (e.g., table scans) that can be reached via this op. - * @param result contains all roots can be reached via op - * @param op the op to examine. - */ - private void collectRoots(List> result, Operator op) { - if (op.getNumParent() == 0) { - result.add(op); - } else { - for (Operator parentOp : op.getParentOperators()) { - collectRoots(result, parentOp); - } - } - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java index e77fc3eac82..880cea59aa1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java @@ -142,8 +142,6 @@ public List> getTasks(HiveConf conf, Object objCtx) work = ((MapredWork) mrTask.getWork()).getMapWork(); } else if (mrTask.getWork() instanceof TezWork){ work = (MapWork) ((TezWork) mrTask.getWork()).getAllWork().get(0); - } else if (mrTask.getWork() instanceof SparkWork) { - work = (MapWork) ((SparkWork) mrTask.getWork()).getAllWork().get(0); } else { work = (MapWork) mrTask.getWork(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java deleted file mode 100644 index 86ca3e05d92..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java +++ /dev/null @@ -1,39 +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.hadoop.hive.ql.plan; - -import java.util.Map; -import java.util.Set; - -public class SparkBucketMapJoinContext extends BucketMapJoinContext { - private static final long serialVersionUID = 1L; - - private Map> posToAliasMap; - - public SparkBucketMapJoinContext(MapJoinDesc clone) { - super(clone); - } - - public void setPosToAliasMap(Map> posToAliasMap) { - this.posToAliasMap = posToAliasMap; - } - - public Map> getPosToAliasMap() { - return posToAliasMap; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java deleted file mode 100644 index c25adfe9021..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkEdgeProperty.java +++ /dev/null @@ -1,114 +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.hadoop.hive.ql.plan; - - -@Explain(displayName = "Edge Property") -public class SparkEdgeProperty { - public static final long SHUFFLE_NONE = 0; // No shuffle is needed. For union only. - public static final long SHUFFLE_GROUP = 1; // HashPartition shuffle, keys are not sorted in any way. - public static final long SHUFFLE_SORT = 2; // RangePartition shuffle, keys are total sorted. - public static final long MR_SHUFFLE_SORT = 4; // HashPartition shuffle, keys are sorted by partition. - - private long edgeType; - - private int numPartitions; - - public SparkEdgeProperty(long edgeType, int numPartitions) { - this.edgeType = edgeType; - this.numPartitions = numPartitions; - } - - public SparkEdgeProperty(long edgeType) { - this.edgeType = edgeType; - } - - public boolean isShuffleNone() { - return edgeType == SHUFFLE_NONE; - } - - public void setShuffleNone() { - edgeType = SHUFFLE_NONE; - } - - public boolean isShuffleGroup() { - return (edgeType & SHUFFLE_GROUP) != 0; - } - - public void setShuffleGroup() { - edgeType |= SHUFFLE_GROUP; - } - - public void setMRShuffle() { - edgeType |= MR_SHUFFLE_SORT; - } - - public boolean isMRShuffle() { - return (edgeType & MR_SHUFFLE_SORT) != 0; - } - - public void setShuffleSort() { - edgeType |= SHUFFLE_SORT; - } - - public boolean isShuffleSort() { - return (edgeType & SHUFFLE_SORT) != 0; - } - - public long getEdgeType() { - return edgeType; - } - - @Explain(displayName = "Shuffle Type") - public String getShuffleType() { - if (isShuffleNone()) { - return "NONE"; - } - - StringBuilder sb = new StringBuilder(); - if (isShuffleGroup()) { - sb.append("GROUP"); - } - - if (isMRShuffle()) { - if (sb.length() != 0) { - sb.append(" "); - } - sb.append("PARTITION-LEVEL SORT"); - } - - if (isShuffleSort()) { - if (sb.length() != 0) { - sb.append(" "); - } - sb.append("SORT"); - } - - return sb.toString(); - } - - public int getNumPartitions() { - return numPartitions; - } - - public void setNumPartitions(int numPartitions) { - this.numPartitions = numPartitions; - } -} - diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkHashTableSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkHashTableSinkDesc.java deleted file mode 100644 index c06f598a541..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkHashTableSinkDesc.java +++ /dev/null @@ -1,73 +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.hadoop.hive.ql.plan; - -import org.apache.hadoop.hive.ql.plan.Explain.Level; -import org.apache.hadoop.hive.ql.plan.Explain.Vectorization; - -/** - * Map Join operator Descriptor implementation. - * - */ -@Explain(displayName = "Spark HashTable Sink Operator") -public class SparkHashTableSinkDesc extends HashTableSinkDesc { - private static final long serialVersionUID = 1L; - - // The position of this table - private byte tag; - - public SparkHashTableSinkDesc() { - } - - public SparkHashTableSinkDesc(MapJoinDesc clone) { - super(clone); - } - - public byte getTag() { - return tag; - } - - public void setTag(byte tag) { - this.tag = tag; - } - - public class SparkHashTableSinkOperatorExplainVectorization extends OperatorExplainVectorization { - - private final HashTableSinkDesc filterDesc; - private final VectorSparkHashTableSinkDesc vectorHashTableSinkDesc; - - public SparkHashTableSinkOperatorExplainVectorization(HashTableSinkDesc filterDesc, - VectorSparkHashTableSinkDesc vectorSparkHashTableSinkDesc) { - // Native vectorization supported. - super(vectorSparkHashTableSinkDesc, true); - this.filterDesc = filterDesc; - this.vectorHashTableSinkDesc = vectorSparkHashTableSinkDesc; - } - } - - @Explain(vectorization = Vectorization.OPERATOR, displayName = "Spark Hash Table Sink Vectorization", explainLevels = { Level.DEFAULT, Level.EXTENDED }) - public SparkHashTableSinkOperatorExplainVectorization getHashTableSinkVectorization() { - VectorSparkHashTableSinkDesc vectorHashTableSinkDesc = (VectorSparkHashTableSinkDesc) getVectorDesc(); - if (vectorHashTableSinkDesc == null) { - return null; - } - return new SparkHashTableSinkOperatorExplainVectorization(this, vectorHashTableSinkDesc); - } - -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java deleted file mode 100644 index 3ed5cb22f69..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkWork.java +++ /dev/null @@ -1,437 +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.hadoop.hive.ql.plan; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.LinkedList; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.hive.ql.plan.Explain.Vectorization; -import org.apache.hadoop.hive.ql.plan.Explain.Level; - -import com.google.common.base.Preconditions; - -/** - * This class encapsulates all the work objects that can be executed - * in a single Spark job. Currently it's basically a tree with MapWork at the - * roots and and ReduceWork at all other nodes. - */ -@SuppressWarnings("serial") -@Explain(displayName = "Spark", explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED }, - vectorization = Vectorization.SUMMARY_PATH) -public class SparkWork extends AbstractOperatorDesc { - - private static final AtomicInteger counter = new AtomicInteger(1); - private final String dagName; - private final String queryId; - - private final Set roots = new LinkedHashSet(); - private final Set leaves = new LinkedHashSet<>(); - - protected final Map> workGraph = - new LinkedHashMap>(); - protected final Map> invertedWorkGraph = - new LinkedHashMap>(); - protected final Map, SparkEdgeProperty> edgeProperties = - new HashMap, SparkEdgeProperty>(); - - private Map> requiredCounterPrefix; - - private Map cloneToWork; - - public SparkWork(String queryId) { - this.queryId = queryId; - this.dagName = queryId + ":" + counter.getAndIncrement(); - cloneToWork = new HashMap(); - } - - @Explain(displayName = "DagName") - public String getName() { - return this.dagName; - } - - public String getQueryId() { - return this.queryId; - } - - /** - * @return a map of "vertex name" to BaseWork - */ - @Explain(displayName = "Vertices", explainLevels = { Explain.Level.USER, Explain.Level.DEFAULT, Explain.Level.EXTENDED }, - vectorization = Vectorization.SUMMARY_PATH) - public Map getWorkMap() { - Map result = new LinkedHashMap(); - for (BaseWork w: getAllWork()) { - result.put(w.getName(), w); - } - return result; - } - - /** - * @return a topologically sorted list of BaseWork - */ - public List getAllWork() { - - List result = new LinkedList(); - Set seen = new HashSet(); - - for (BaseWork leaf: leaves) { - // make sure all leaves are visited at least once - visit(leaf, seen, result); - } - - return result; - } - - public Collection getAllWorkUnsorted() { - return workGraph.keySet(); - } - - private void visit(BaseWork child, Set seen, List result) { - if (seen.contains(child)) { - // don't visit multiple times - return; - } - - seen.add(child); - - for (BaseWork parent: getParents(child)) { - if (!seen.contains(parent)) { - visit(parent, seen, result); - } - } - - result.add(child); - } - - /** - * Add all nodes in the collection without any connections. - */ - public void addAll(Collection c) { - for (BaseWork w: c) { - this.add(w); - } - } - - /** - * Add all nodes in the collection without any connections. - */ - public void addAll(BaseWork[] bws) { - for (BaseWork w: bws) { - this.add(w); - } - } - - /** - * Whether the specified BaseWork is a vertex in this graph - * @param w the BaseWork to check - * @return whether specified BaseWork is in this graph - */ - public boolean contains(BaseWork w) { - return workGraph.containsKey(w); - } - - /** - * add creates a new node in the graph without any connections - */ - public void add(BaseWork w) { - if (workGraph.containsKey(w)) { - return; - } - workGraph.put(w, new LinkedList()); - invertedWorkGraph.put(w, new LinkedList()); - roots.add(w); - leaves.add(w); - } - - /** - * disconnect removes an edge between a and b. Both a and - * b have to be in the graph. If there is no matching edge - * no change happens. - */ - public void disconnect(BaseWork a, BaseWork b) { - workGraph.get(a).remove(b); - invertedWorkGraph.get(b).remove(a); - if (getParents(b).isEmpty()) { - roots.add(b); - } - if (getChildren(a).isEmpty()) { - leaves.add(a); - } - edgeProperties.remove(new ImmutablePair(a, b)); - } - - /** - * getRoots returns all nodes that do not have a parent. - */ - public Set getRoots() { - return new LinkedHashSet(roots); - } - - /** - * getLeaves returns all nodes that do not have a child - */ - public Set getLeaves() { - return new LinkedHashSet(leaves); - } - - public void setRequiredCounterPrefix(Map> requiredCounterPrefix) { - this.requiredCounterPrefix = requiredCounterPrefix; - } - - public Map> getRequiredCounterPrefix() { - return requiredCounterPrefix; - } - - /** - * getParents returns all the nodes with edges leading into work - */ - public List getParents(BaseWork work) { - Preconditions.checkArgument(invertedWorkGraph.containsKey(work), - "AssertionError: expected invertedWorkGraph.containsKey(work) to be true"); - Preconditions.checkArgument(invertedWorkGraph.get(work) != null, - "AssertionError: expected invertedWorkGraph.get(work) to be not null"); - return new LinkedList(invertedWorkGraph.get(work)); - } - - /** - * getChildren returns all the nodes with edges leading out of work - */ - public List getChildren(BaseWork work) { - Preconditions.checkArgument(workGraph.containsKey(work), - "AssertionError: expected workGraph.containsKey(work) to be true"); - Preconditions.checkArgument(workGraph.get(work) != null, - "AssertionError: expected workGraph.get(work) to be not null"); - return new LinkedList(workGraph.get(work)); - } - - /** - * remove removes a node from the graph and removes all edges with - * work as start or end point. No change to the graph if the node - * doesn't exist. - */ - public void remove(BaseWork work) { - if (!workGraph.containsKey(work)) { - return; - } - - List children = getChildren(work); - List parents = getParents(work); - - for (BaseWork w: children) { - edgeProperties.remove(new ImmutablePair(work, w)); - invertedWorkGraph.get(w).remove(work); - if (invertedWorkGraph.get(w).size() == 0) { - roots.add(w); - } - } - - for (BaseWork w: parents) { - edgeProperties.remove(new ImmutablePair(w, work)); - workGraph.get(w).remove(work); - if (workGraph.get(w).size() == 0) { - leaves.add(w); - } - } - - roots.remove(work); - leaves.remove(work); - - workGraph.remove(work); - invertedWorkGraph.remove(work); - } - - /** - * returns the edge type connecting work a and b - */ - public SparkEdgeProperty getEdgeProperty(BaseWork a, BaseWork b) { - return edgeProperties.get(new ImmutablePair(a, b)); - } - - /** - * connect adds an edge between a and b. Both nodes have - * to be added prior to calling connect. - * @param - */ - public void connect(BaseWork a, BaseWork b, SparkEdgeProperty edgeProp) { - workGraph.get(a).add(b); - invertedWorkGraph.get(b).add(a); - roots.remove(b); - leaves.remove(a); - ImmutablePair workPair = new ImmutablePair(a, b); - edgeProperties.put(workPair, edgeProp); - } - - /* - * Dependency is a class used for explain - */ - public class Dependency implements Serializable, Comparable { - public BaseWork w; - public SparkEdgeProperty prop; - - @Explain(displayName = "Name") - public String getName() { - return w.getName(); - } - - @Explain(displayName = "Shuffle Type") - public String getShuffleType() { - return prop.getShuffleType(); - } - - @Explain(displayName = "Number of Partitions") - public String getNumPartitions() { - return Integer.toString(prop.getNumPartitions()); - } - - @Override - public int compareTo(Dependency o) { - int compare = getName().compareTo(o.getName()); - if (compare == 0) { - compare = getShuffleType().compareTo(o.getShuffleType()); - } - return compare; - } - } - - /** - * Task name is usually sorted by natural order, which is the same - * as the topological order in most cases. However, with Spark, some - * tasks may be converted, so have new names. The natural order may - * be different from the topological order. This class is to make - * sure all tasks to be sorted by topological order deterministically. - */ - private static class ComparableName implements Comparable { - private final Map dependencies; - private final String name; - - ComparableName(Map dependencies, String name) { - this.dependencies = dependencies; - this.name = name; - } - - /** - * Check if task n1 depends on task n2 - */ - boolean dependsOn(String n1, String n2) { - for (String p = dependencies.get(n1); p != null; p = dependencies.get(p)) { - if (p.equals(n2)) { - return true; - } - } - return false; - } - - /** - * Get the number of parents of task n - */ - int getDepth(String n) { - int depth = 0; - for (String p = dependencies.get(n); p != null; p = dependencies.get(p)) { - depth++; - } - return depth; - } - - @Override - public int compareTo(ComparableName o) { - if (dependsOn(name, o.name)) { - // this depends on o - return 1; - } - if (dependsOn(o.name, name)) { - // o depends on this - return -1; - } - // No dependency, check depth - int d1 = getDepth(name); - int d2 = getDepth(o.name); - if (d1 == d2) { - // Same depth, using natural order - return name.compareTo(o.name); - } - // Deep one is bigger, i.e. less to the top - return d1 > d2 ? 1 : -1; - } - - @Override - public String toString() { - return name; - } - } - - @Explain(displayName = "Edges", explainLevels = { Explain.Level.USER, Explain.Level.DEFAULT, Explain.Level.EXTENDED }, - vectorization = Vectorization.SUMMARY_PATH) - public Map> getDependencyMap() { - Map allDependencies = new HashMap(); - Map> result = - new LinkedHashMap>(); - for (BaseWork baseWork : getAllWork()) { - if (invertedWorkGraph.get(baseWork) != null && invertedWorkGraph.get(baseWork).size() > 0) { - List dependencies = new LinkedList(); - for (BaseWork d : invertedWorkGraph.get(baseWork)) { - allDependencies.put(baseWork.getName(), d.getName()); - Dependency dependency = new Dependency(); - dependency.w = d; - dependency.prop = getEdgeProperty(d, baseWork); - dependencies.add(dependency); - } - if (!dependencies.isEmpty()) { - Collections.sort(dependencies); - result.put(new ComparableName(allDependencies, - baseWork.getName()), dependencies); - } - } - } - return result; - } - - /** - * @return all reduce works of this spark work, in sorted order. - */ - public List getAllReduceWork() { - List result = new ArrayList(); - for (BaseWork work : getAllWork()) { - if (work instanceof ReduceWork) { - result.add((ReduceWork) work); - } - } - return result; - } - - public Map getCloneToWork() { - return cloneToWork; - } - - public void setCloneToWork(Map cloneToWork) { - this.cloneToWork = cloneToWork; - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java deleted file mode 100644 index e59b93f8e97..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkHashTableSinkDesc.java +++ /dev/null @@ -1,35 +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.hadoop.hive.ql.plan; - -/** - * VectorHashTableSinkDesc. - * - * Extra parameters beyond HashTableSinkDesc just for the VectorHashTableSinkOperator. - * - * We don't extend HashTableSinkDesc because the base OperatorDesc doesn't support - * clone and adding it is a lot work for little gain. - */ -public class VectorSparkHashTableSinkDesc extends AbstractVectorDesc { - - private static final long serialVersionUID = 1L; - - public VectorSparkHashTableSinkDesc() { - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkPartitionPruningSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkPartitionPruningSinkDesc.java deleted file mode 100644 index c1e8830c0d3..00000000000 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorSparkPartitionPruningSinkDesc.java +++ /dev/null @@ -1,35 +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.hadoop.hive.ql.plan; - -/** - * VectorHashTableSinkDesc. - * - * Extra parameters beyond HashTableSinkDesc just for the VectorHashTableSinkOperator. - * - * We don't extend SparkPartitionPruningSinkDesc because the base OperatorDesc doesn't support - * clone and adding it is a lot work for little gain. - */ -public class VectorSparkPartitionPruningSinkDesc extends AbstractVectorDesc { - - private static final long serialVersionUID = 1L; - - public VectorSparkPartitionPruningSinkDesc() { - } -} diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java index 75b35f0aea6..22c82518984 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java @@ -74,7 +74,6 @@ import org.apache.hadoop.hive.ql.exec.mr3.session.MR3Session; import org.apache.hadoop.hive.ql.exec.mr3.session.MR3SessionManager; import org.apache.hadoop.hive.ql.exec.mr3.session.MR3SessionManagerImpl; -import org.apache.hadoop.hive.ql.exec.spark.session.SparkSession; import org.apache.hadoop.hive.ql.exec.tez.TezSessionState; import org.apache.hadoop.hive.ql.history.HiveHistory; import org.apache.hadoop.hive.ql.history.HiveHistoryImpl; @@ -1781,10 +1780,6 @@ private void unCacheDataNucleusClassLoaders() { } } - public void closeSparkSession() { - // sparkSession is never used - } - public AuthorizationMode getAuthorizationMode(){ setupAuth(); if(authorizer != null){ @@ -1873,15 +1868,6 @@ public void setUserIpAddress(String userIpAddress) { this.userIpAddress = userIpAddress; } - public SparkSession getSparkSession() { - // sparkSession is never used - return null; - } - - public void setSparkSession(SparkSession sparkSession) { - // sparkSession is never used - } - public MR3Session getMr3Session() { return mr3Session; } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java index 244bc739438..d6404fb8cb8 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExplainTask.java @@ -247,89 +247,6 @@ public void testGetJSONLogicalPlanJsonShouldMatch() throws Exception { assertEquals(expected, result); } - @Test - public void testOutputMapJsonShouldMatch() throws Exception { - Map map = new LinkedHashMap<>(); - - // String - map.put("key-1", "value-1"); - - // SparkWork - map.put("spark-work", new SparkWork("spark-work")); - - // Empty list - List emptList = Collections.emptyList(); - map.put("empty-list", emptList); - - // List of TezWork.Dependency - List tezList1 = new ArrayList<>(Arrays.asList(new Object[] {mockTezWorkDependency()})); - map.put("tez-list-1", tezList1); - List tezList2 = new ArrayList<>( - Arrays.asList(new Object[] {mockTezWorkDependency(), mockTezWorkDependency()})); - map.put("tez-list-2", tezList2); - - // List of SparkWork.Dependency - List sparkList1 = new ArrayList<>( - Arrays.asList(new Object[]{mockSparkWorkDependency()})); - map.put("spark-list-1", sparkList1); - List sparkList2 = new ArrayList<>( - Arrays.asList(new Object[]{mockSparkWorkDependency(), mockSparkWorkDependency()})); - map.put("spark-list-2", sparkList2); - - // inner Map - Map innerMap = new LinkedHashMap<>(); - innerMap.put("inner-key-1", "inner-value-1"); - innerMap.put("inner-key-2", tezList1); - map.put("map-1", innerMap); - - JsonNode result = objectMapper.readTree( - uut.outputMap(map, false, null, false, true, 0).toString()); - JsonNode expected = objectMapper.readTree("{\"key-1\":\"value-1\",\"tez-list-2\":" + - "[{\"parent\":\"name\"}," + "{\"parent\":\"name\"}],\"tez-list-1\":" + - "{\"parent\":\"name\"},\"empty-list\":\"[]\",\"spark-list-2\":" + - "[{\"parent\":\"mock-name\"},{\"parent\":\"mock-name\"}]," + - "\"spark-list-1\":{\"parent\":" + - "\"mock-name\"}, \"map-1\":\"{inner-key-1=inner-value-1, " + - "inner-key-2=[mock-tez-dependency]}\",\"spark-work\":" + - "{\"Spark\":{\"DagName:\":\"spark-work:2\"}}}"); - - assertEquals(expected, result); - } - - @Test - public void testOutputPlanJsonShouldMatch() throws Exception { - // SparkWork - SparkWork work = new SparkWork("spark-work"); - - JsonNode result = objectMapper.readTree( - uut.outputPlan(work, null, false, true, 0, null).toString()); - JsonNode expected = objectMapper.readTree("{\"Spark\":{\"DagName:\":\"spark-work:1\"}}"); - assertEquals(expected, result); - - // Operator with single child - CollectOperator parentCollectOperator1 = new CollectOperator(); - CollectOperator child1 = new CollectOperator(); - parentCollectOperator1.setChildOperators(new ArrayList>( - Arrays.asList(new CollectOperator[] {child1}))); - parentCollectOperator1.setConf(new CollectDesc()); - - result = objectMapper.readTree( - uut.outputPlan(parentCollectOperator1, null, false, true, 0, null).toString()); - expected = objectMapper.readTree("{\"Collect\":{\"children\":{}}}"); - assertEquals(expected, result); - - // Operator with 2 children - CollectOperator parentCollectOperator2 = new CollectOperator(); - CollectOperator child2 = new CollectOperator(); - parentCollectOperator2.setChildOperators(new ArrayList>( - Arrays.asList(new CollectOperator[] {child1, child2}))); - parentCollectOperator2.setConf(new CollectDesc()); - result = objectMapper.readTree( - uut.outputPlan(parentCollectOperator2, null, false, true, 0, null).toString()); - expected = objectMapper.readTree("{\"Collect\":{\"children\":[{},{}]}}"); - assertEquals(expected, result); - } - @Test public void testCollectAuthRelatedEntitiesJsonShouldMatch() throws Exception { QueryState qs = mock(QueryState.class); @@ -378,13 +295,6 @@ private TezWork.Dependency mockTezWorkDependency() { return dep; } - private SparkWork.Dependency mockSparkWorkDependency() { - SparkWork.Dependency dep = mock(SparkWork.Dependency.class); - when(dep.getName()).thenReturn("mock-name"); - when(dep.toString()).thenReturn("mock-spark-dependency"); - return dep; - } - private ExplainWork mockExplainWork() { ExplainWork explainWork = mock(ExplainWork.class); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java index 530ff91ee17..94183f69aa6 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperatorNames.java @@ -29,9 +29,6 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSparkHashTableSinkOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSparkPartitionPruningSinkOperator; -import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; import static org.junit.Assert.assertEquals; import org.junit.Before; import org.junit.After; @@ -104,16 +101,6 @@ public void testOperatorNames() throws Exception { assertEquals(MapJoinOperator.getOperatorName(), new VectorMapJoinOuterFilteredOperator().getName()); - assertEquals(SparkHashTableSinkOperator.getOperatorName(), - new SparkHashTableSinkOperator().getName()); - assertEquals(SparkHashTableSinkOperator.getOperatorName(), - new VectorSparkHashTableSinkOperator().getName()); - - assertEquals(SparkPartitionPruningSinkOperator.getOperatorName(), - new SparkPartitionPruningSinkOperator().getName()); - assertEquals(SparkPartitionPruningSinkOperator.getOperatorName(), - new VectorSparkPartitionPruningSinkOperator().getName()); - } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java index 90eb45b9943..5bdf2b6d7e5 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java @@ -59,7 +59,6 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.apache.hadoop.hive.ql.exec.tez.TezTask; import org.apache.hadoop.hive.ql.io.*; import org.apache.hadoop.hive.ql.metadata.HiveException; @@ -888,15 +887,12 @@ public void testGetTasksHaveNoRepeats() { CountingWrappingTask mrTask = new CountingWrappingTask(new ExecDriver()); CountingWrappingTask tezTask = new CountingWrappingTask(new TezTask()); - CountingWrappingTask sparkTask = new CountingWrappingTask(new SparkTask()); // First check - we should not have repeats in results assertEquals("No repeated MRTasks from Utilities.getMRTasks", 1, Utilities.getMRTasks(getTestDiamondTaskGraph(mrTask)).size()); assertEquals("No repeated TezTasks from Utilities.getTezTasks", 1, Utilities.getTezTasks(getTestDiamondTaskGraph(tezTask)).size()); - assertEquals("No repeated TezTasks from Utilities.getSparkTasks", 1, - Utilities.getSparkTasks(getTestDiamondTaskGraph(sparkTask)).size()); // Second check - the tasks we looked for must not have been accessed more than // once as a result of the traversal (note that we actually wind up accessing @@ -905,7 +901,6 @@ public void testGetTasksHaveNoRepeats() { assertEquals("MRTasks should have been visited only once", 2, mrTask.getDepCallCount()); assertEquals("TezTasks should have been visited only once", 2, tezTask.getDepCallCount()); - assertEquals("SparkTasks should have been visited only once", 2, sparkTask.getDepCallCount()); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java index 0693d2458b8..c432bc76f3b 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapRedTask.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hive.common.metrics.common.Metrics; import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; -import org.apache.hadoop.hive.ql.exec.spark.SparkTask; import org.junit.Test; import org.mockito.Mockito; @@ -41,7 +40,6 @@ public void mrTask_updates_Metrics() throws IOException { verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_MR_TASKS); verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS); - verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS); } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java index bb3ba88aa7c..8e0a81a4567 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/mr/TestMapredLocalTask.java @@ -40,7 +40,6 @@ public void localMRTask_updates_Metrics() throws IOException { verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_MR_TASKS); verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS); - verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS); } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java deleted file mode 100644 index cd7f02f4189..00000000000 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestHiveKVResultCache.java +++ /dev/null @@ -1,350 +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.hadoop.hive.ql.exec.spark; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.LinkedBlockingQueue; - -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.io.BytesWritable; -import org.junit.Test; - -import scala.Tuple2; - -import com.clearspring.analytics.util.Preconditions; - -@SuppressWarnings({"unchecked", "rawtypes"}) -public class TestHiveKVResultCache { - @Test - public void testSimple() throws Exception { - // Create KV result cache object, add one (k,v) pair and retrieve them. - HiveKVResultCache cache = new HiveKVResultCache(); - - HiveKey key = new HiveKey("key".getBytes(), "key".hashCode()); - BytesWritable value = new BytesWritable("value".getBytes()); - cache.add(key, value); - - assertTrue("KV result cache should have at least one element", cache.hasNext()); - - Tuple2 row = cache.next(); - assertTrue("Incorrect key", row._1().equals(key)); - assertTrue("Incorrect value", row._2().equals(value)); - - assertTrue("Cache shouldn't have more records", !cache.hasNext()); - } - - @Test - public void testSpilling() throws Exception { - HiveKVResultCache cache = new HiveKVResultCache(); - - final int recordCount = HiveKVResultCache.IN_MEMORY_NUM_ROWS * 3; - - // Test using the same cache where first n rows are inserted then cache is cleared. - // Next reuse the same cache and insert another m rows and verify the cache stores correctly. - // This simulates reusing the same cache over and over again. - testSpillingHelper(cache, recordCount); - testSpillingHelper(cache, 1); - testSpillingHelper(cache, recordCount); - } - - /** Helper method which inserts numRecords and retrieves them from cache and verifies */ - private void testSpillingHelper(HiveKVResultCache cache, int numRecords) { - for(int i=0; i row = cache.next(); - assertTrue("Unexpected key at position: " + recordsSeen, - new String(row._1().getBytes()).equals(key)); - assertTrue("Unexpected value at position: " + recordsSeen, - new String(row._2().getBytes()).equals(value)); - - recordsSeen++; - } - - assertTrue("Retrieved record count doesn't match inserted record count", - numRecords == recordsSeen); - - cache.clear(); - } - - @Test - public void testResultList() throws Exception { - scanAndVerify(10000, 0, 0, "a", "b"); - scanAndVerify(10000, 511, 0, "a", "b"); - scanAndVerify(10000, 511 * 2, 0, "a", "b"); - scanAndVerify(10000, 511, 10, "a", "b"); - scanAndVerify(10000, 511 * 2, 10, "a", "b"); - scanAndVerify(10000, 512, 0, "a", "b"); - scanAndVerify(10000, 512 * 2, 0, "a", "b"); - scanAndVerify(10000, 512, 3, "a", "b"); - scanAndVerify(10000, 512 * 6, 10, "a", "b"); - scanAndVerify(10000, 512 * 7, 5, "a", "b"); - scanAndVerify(10000, 512 * 9, 19, "a", "b"); - scanAndVerify(10000, 1, 0, "a", "b"); - scanAndVerify(10000, 1, 1, "a", "b"); - } - - private static void scanAndVerify( - long rows, int threshold, int separate, String prefix1, String prefix2) { - ArrayList> output = - new ArrayList>((int)rows); - scanResultList(rows, threshold, separate, output, prefix1, prefix2); - assertEquals(rows, output.size()); - long primaryRows = rows * (100 - separate) / 100; - long separateRows = rows - primaryRows; - HashSet primaryRowKeys = new HashSet(); - HashSet separateRowKeys = new HashSet(); - for (Tuple2 item: output) { - String key = bytesWritableToString(item._1); - String value = bytesWritableToString(item._2); - String prefix = key.substring(0, key.indexOf('_')); - Long id = Long.valueOf(key.substring(5 + prefix.length())); - if (prefix.equals(prefix1)) { - assertTrue(id >= 0 && id < primaryRows); - primaryRowKeys.add(id); - } else { - assertEquals(prefix2, prefix); - assertTrue(id >= 0 && id < separateRows); - separateRowKeys.add(id); - } - assertEquals(prefix + "_value_" + id, value); - } - assertEquals(separateRows, separateRowKeys.size()); - assertEquals(primaryRows, primaryRowKeys.size()); - } - - /** - * Convert a BytesWritable to a string. - * Don't use {@link BytesWritable#copyBytes()} - * so as to be compatible with hadoop 1 - */ - private static String bytesWritableToString(BytesWritable bw) { - int size = bw.getLength(); - byte[] bytes = new byte[size]; - System.arraycopy(bw.getBytes(), 0, bytes, 0, size); - return new String(bytes); - } - - private static class MyHiveFunctionResultList extends HiveBaseFunctionResultList { - private static final long serialVersionUID = -1L; - - // Total rows to emit during the whole iteration, - // excluding the rows emitted by the separate thread. - private long primaryRows; - // Batch of rows to emit per processNextRecord() call. - private int thresholdRows; - // Rows to be emitted with a separate thread per processNextRecord() call. - private long separateRows; - // Thread to generate the separate rows beside the normal thread. - private Thread separateRowGenerator; - - // Counter for rows emitted - private long rowsEmitted; - private long separateRowsEmitted; - - // Prefix for primary row keys - private String prefix1; - // Prefix for separate row keys - private String prefix2; - - // A queue to notify separateRowGenerator to generate the next batch of rows. - private LinkedBlockingQueue queue; - - MyHiveFunctionResultList(Iterator inputIterator) { - super(inputIterator); - } - - void init(long rows, int threshold, int separate, String p1, String p2) { - Preconditions.checkArgument((threshold > 0 || separate == 0) - && separate < 100 && separate >= 0 && rows > 0); - primaryRows = rows * (100 - separate) / 100; - separateRows = rows - primaryRows; - thresholdRows = threshold; - prefix1 = p1; - prefix2 = p2; - if (separateRows > 0) { - separateRowGenerator = new Thread(new Runnable() { - @Override - public void run() { - try { - long separateBatchSize = thresholdRows * separateRows / primaryRows; - while (!queue.take().booleanValue()) { - for (int i = 0; i < separateBatchSize; i++) { - collect(prefix2, separateRowsEmitted++); - } - } - } catch (InterruptedException e) { - e.printStackTrace(); - } - for (; separateRowsEmitted < separateRows;) { - collect(prefix2, separateRowsEmitted++); - } - } - }); - queue = new LinkedBlockingQueue(); - separateRowGenerator.start(); - } - } - - public void collect(String prefix, long id) { - String k = prefix + "_key_" + id; - String v = prefix + "_value_" + id; - HiveKey key = new HiveKey(k.getBytes(), k.hashCode()); - BytesWritable value = new BytesWritable(v.getBytes()); - try { - collect(key, value); - } catch (IOException e) { - e.printStackTrace(); - } - } - - @Override - protected void processNextRecord(Object inputRecord) throws IOException { - for (int i = 0; i < thresholdRows; i++) { - collect(prefix1, rowsEmitted++); - } - if (separateRowGenerator != null) { - queue.add(Boolean.FALSE); - } - } - - @Override - protected boolean processingDone() { - return false; - } - - @Override - protected void closeRecordProcessor() { - for (; rowsEmitted < primaryRows;) { - collect(prefix1, rowsEmitted++); - } - if (separateRowGenerator != null) { - queue.add(Boolean.TRUE); - try { - separateRowGenerator.join(); - } catch (InterruptedException e) { - e.printStackTrace(); - } - } - } - } - - private static long scanResultList(long rows, int threshold, int separate, - List> output, String prefix1, String prefix2) { - final long iteratorCount = threshold == 0 ? 1 : rows * (100 - separate) / 100 / threshold; - MyHiveFunctionResultList resultList = new MyHiveFunctionResultList(new Iterator() { - // Input record iterator, not used - private int i = 0; - @Override - public boolean hasNext() { - return i++ < iteratorCount; - } - - @Override - public Object next() { - return Integer.valueOf(i); - } - - @Override - public void remove() { - } - }); - - resultList.init(rows, threshold, separate, prefix1, prefix2); - long startTime = System.currentTimeMillis(); - while (resultList.hasNext()) { - Object item = resultList.next(); - if (output != null) { - output.add((Tuple2)item); - } - } - long endTime = System.currentTimeMillis(); - return endTime - startTime; - } - - private static long[] scanResultList(long rows, int threshold, int extra) { - // 1. Simulate emitting all records in closeRecordProcessor(). - long t1 = scanResultList(rows, 0, 0, null, "a", "b"); - - // 2. Simulate emitting records in processNextRecord() with small memory usage limit. - long t2 = scanResultList(rows, threshold, 0, null, "c", "d"); - - // 3. Simulate emitting records in processNextRecord() with large memory usage limit. - long t3 = scanResultList(rows, threshold * 10, 0, null, "e", "f"); - - // 4. Same as 2. Also emit extra records from a separate thread. - long t4 = scanResultList(rows, threshold, extra, null, "g", "h"); - - // 5. Same as 3. Also emit extra records from a separate thread. - long t5 = scanResultList(rows, threshold * 10, extra, null, "i", "j"); - - return new long[] {t1, t2, t3, t4, t5}; - } - - public static void main(String[] args) throws Exception { - long rows = 1000000; // total rows to generate - int threshold = 512; // # of rows to cache at most - int extra = 5; // percentile of extra rows to generate by a different thread - - if (args.length > 0) { - rows = Long.parseLong(args[0]); - } - if (args.length > 1) { - threshold = Integer.parseInt(args[1]); - } - if (args.length > 2) { - extra = Integer.parseInt(args[2]); - } - - // Warm up couple times - for (int i = 0; i < 2; i++) { - scanResultList(rows, threshold, extra); - } - - int count = 5; - long[] t = new long[count]; - // Run count times and get average - for (int i = 0; i < count; i++) { - long[] tmp = scanResultList(rows, threshold, extra); - for (int k = 0; k < count; k++) { - t[k] += tmp[k]; - } - } - for (int i = 0; i < count; i++) { - t[i] /= count; - } - - System.out.println(t[0] + "\t" + t[1] + "\t" + t[2] - + "\t" + t[3] + "\t" + t[4]); - } -} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java deleted file mode 100644 index 75b4151118a..00000000000 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/TestSparkTask.java +++ /dev/null @@ -1,125 +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.hadoop.hive.ql.exec.spark; - -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.hive.common.metrics.common.Metrics; -import org.apache.hadoop.hive.common.metrics.common.MetricsConstant; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.spark.Statistic.SparkStatisticsBuilder; -import org.apache.hadoop.hive.ql.exec.spark.status.RemoteSparkJobMonitor; -import org.apache.hadoop.hive.ql.exec.spark.status.impl.RemoteSparkJobStatus; -import org.apache.hadoop.hive.ql.plan.BaseWork; -import org.apache.hadoop.hive.ql.plan.MapWork; -import org.apache.hadoop.hive.ql.plan.SparkWork; -import org.apache.hive.spark.client.JobHandle.State; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; - -public class TestSparkTask { - - @Test - public void sparkTask_updates_Metrics() throws IOException { - - Metrics mockMetrics = Mockito.mock(Metrics.class); - - SparkTask sparkTask = new SparkTask(); - sparkTask.updateTaskMetrics(mockMetrics); - - verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS); - verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS); - verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_MR_TASKS); - } - - @Test - public void removeEmptySparkTask() { - SparkTask grandpa = new SparkTask(); - SparkWork grandpaWork = new SparkWork("grandpa"); - grandpaWork.add(new MapWork()); - grandpa.setWork(grandpaWork); - - SparkTask parent = new SparkTask(); - SparkWork parentWork = new SparkWork("parent"); - parentWork.add(new MapWork()); - parent.setWork(parentWork); - - SparkTask child1 = new SparkTask(); - SparkWork childWork1 = new SparkWork("child1"); - childWork1.add(new MapWork()); - child1.setWork(childWork1); - - - grandpa.addDependentTask(parent); - parent.addDependentTask(child1); - - Assert.assertEquals(grandpa.getChildTasks().size(), 1); - Assert.assertEquals(child1.getParentTasks().size(), 1); - if (isEmptySparkWork(parent.getWork())) { - SparkUtilities.removeEmptySparkTask(parent); - } - - Assert.assertEquals(grandpa.getChildTasks().size(), 0); - Assert.assertEquals(child1.getParentTasks().size(), 0); - } - - @Test - public void testRemoteSparkCancel() { - RemoteSparkJobStatus jobSts = Mockito.mock(RemoteSparkJobStatus.class); - when(jobSts.getRemoteJobState()).thenReturn(State.CANCELLED); - when(jobSts.isRemoteActive()).thenReturn(true); - HiveConf hiveConf = new HiveConf(); - RemoteSparkJobMonitor remoteSparkJobMonitor = new RemoteSparkJobMonitor(hiveConf, jobSts); - Assert.assertEquals(remoteSparkJobMonitor.startMonitor(), 3); - } - - @Test - public void testSparkStatisticsToString() { - SparkStatisticsBuilder statsBuilder = new SparkStatisticsBuilder(); - statsBuilder.add("TEST", "stat1", "1"); - statsBuilder.add("TEST", "stat2", "1"); - String statsString = SparkTask.sparkStatisticsToString(statsBuilder.build(), 10); - - Assert.assertTrue(statsString.contains("10")); - Assert.assertTrue(statsString.contains("TEST")); - Assert.assertTrue(statsString.contains("stat1")); - Assert.assertTrue(statsString.contains("stat2")); - Assert.assertTrue(statsString.contains("1")); - } - - private boolean isEmptySparkWork(SparkWork sparkWork) { - List allWorks = sparkWork.getAllWork(); - boolean allWorksIsEmtpy = true; - for (BaseWork work : allWorks) { - if (work.getAllOperators().size() > 0) { - allWorksIsEmtpy = false; - break; - } - } - return allWorksIsEmtpy; - } -} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java deleted file mode 100644 index fe95ce0a85e..00000000000 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/spark/session/TestSparkSessionManagerImpl.java +++ /dev/null @@ -1,259 +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.hadoop.hive.ql.exec.spark.session; - -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.util.StringUtils; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.List; -import java.util.Random; -import java.util.concurrent.TimeoutException; - -import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClient; -import org.apache.hadoop.hive.ql.exec.spark.HiveSparkClientFactory; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.spark.SparkConf; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -public class TestSparkSessionManagerImpl { - private static final Logger LOG = LoggerFactory.getLogger(TestSparkSessionManagerImpl.class); - - private SparkSessionManagerImpl sessionManagerHS2 = null; - private boolean anyFailedSessionThread; // updated only when a thread has failed. - - - /** Tests CLI scenario where we get a single session and use it multiple times. */ - @Test - public void testSingleSessionMultipleUse() throws Exception { - HiveConf conf = new HiveConf(); - conf.set("spark.master", "local"); - - SparkSessionManager sessionManager = SparkSessionManagerImpl.getInstance(); - SparkSession sparkSession1 = sessionManager.getSession(null, conf, true); - - assertTrue(sparkSession1.isOpen()); - - SparkSession sparkSession2 = sessionManager.getSession(sparkSession1, conf, true); - assertTrue(sparkSession1 == sparkSession2); // Same session object is expected. - - assertTrue(sparkSession2.isOpen()); - sessionManager.shutdown(); - sessionManager.closeSession(sparkSession1); - } - - /** - * Tests multi-user scenario (like HiveServer2) where each user gets a session - * and uses it multiple times. - */ - @Test - public void testMultiSessionMultipleUse() throws Exception { - sessionManagerHS2 = SparkSessionManagerImpl.getInstance(); - - // Shutdown existing session manager - sessionManagerHS2.shutdown(); - - HiveConf hiveConf = new HiveConf(); - hiveConf.set("spark.master", "local"); - - sessionManagerHS2.setup(hiveConf); - - List threadList = new ArrayList(); - for (int i = 0; i < 10; i++) { - Thread t = new Thread(new SessionThread(), "Session thread " + i); - t.start(); - threadList.add(t); - } - - for (Thread t : threadList) { - try { - t.join(); - } catch (InterruptedException e) { - String msg = "Interrupted while waiting for test session threads."; - LOG.error(msg, e); - fail(msg); - } - } - - assertFalse("At least one of the session threads failed. See the test output for details.", - anyFailedSessionThread); - - System.out.println("Ending SessionManagerHS2"); - sessionManagerHS2.shutdown(); - } - - /** - * Test HIVE-16395 - by default we force cloning of Configurations for Spark jobs - */ - @Test - public void testForceConfCloning() throws Exception { - HiveConf conf = new HiveConf(); - conf.set("spark.master", "local"); - String sparkCloneConfiguration = HiveSparkClientFactory.SPARK_CLONE_CONFIGURATION; - - // Clear the value of sparkCloneConfiguration - conf.unset(sparkCloneConfiguration); - assertNull( "Could not clear " + sparkCloneConfiguration + " in HiveConf", - conf.get(sparkCloneConfiguration)); - - // By default we should set sparkCloneConfiguration to true in the Spark config - checkSparkConf(conf, sparkCloneConfiguration, "true"); - - // User can override value for sparkCloneConfiguration in Hive config to false - conf.set(sparkCloneConfiguration, "false"); - checkSparkConf(conf, sparkCloneConfiguration, "false"); - - // User can override value of sparkCloneConfiguration in Hive config to true - conf.set(sparkCloneConfiguration, "true"); - checkSparkConf(conf, sparkCloneConfiguration, "true"); - } - - @Test - public void testGetHiveException() throws Exception { - HiveConf conf = new HiveConf(); - conf.set("spark.master", "local"); - SparkSessionManager ssm = SparkSessionManagerImpl.getInstance(); - SparkSessionImpl ss = (SparkSessionImpl) ssm.getSession( - null, conf, true); - - Throwable e; - - e = new TimeoutException(); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_TIMEOUT); - - e = new InterruptedException(); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_INTERRUPTED); - - e = new RuntimeException("\t diagnostics: Application application_1508358311878_3322732 " - + "failed 1 times due to ApplicationMaster for attempt " - + "appattempt_1508358311878_3322732_000001 timed out. Failing the application."); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_TIMEOUT); - - e = new RuntimeException("\t diagnostics: Application application_1508358311878_3330000 " - + "submitted by user hive to unknown queue: foo"); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_QUEUE, - "submitted by user hive to unknown queue: foo"); - - e = new RuntimeException("\t diagnostics: org.apache.hadoop.security.AccessControlException: " - + "Queue root.foo is STOPPED. Cannot accept submission of application: " - + "application_1508358311878_3369187"); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_QUEUE, - "Queue root.foo is STOPPED"); - - e = new RuntimeException("\t diagnostics: org.apache.hadoop.security.AccessControlException: " - + "Queue root.foo already has 10 applications, cannot accept submission of application: " - + "application_1508358311878_3384544"); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_QUEUE_FULL, - "Queue root.foo already has 10 applications"); - - e = new RuntimeException("Exception in thread \"\"main\"\" java.lang.IllegalArgumentException: " - + "Required executor memory (7168+10240 MB) is above the max threshold (16384 MB) of this " - + "cluster! Please check the values of 'yarn.scheduler.maximum-allocation-mb' and/or " - + "'yarn.nodemanager.resource.memory-mb'."); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST, - "Required executor memory (7168+10240 MB) is above the max threshold (16384 MB)"); - - e = new RuntimeException("Exception in thread \"\"main\"\" java.lang.IllegalArgumentException: " - + "requirement failed: initial executor number 5 must between min executor number10 " - + "and max executor number 50"); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_INVALID_RESOURCE_REQUEST, - "initial executor number 5 must between min executor number10 and max executor number 50"); - - // Other exceptions which defaults to SPARK_CREATE_CLIENT_ERROR - e = new Exception(); - checkHiveException(ss, e, ErrorMsg.SPARK_CREATE_CLIENT_ERROR); - } - - private void checkHiveException(SparkSessionImpl ss, Throwable e, ErrorMsg expectedErrMsg) { - checkHiveException(ss, e, expectedErrMsg, null); - } - - private void checkHiveException(SparkSessionImpl ss, Throwable e, - ErrorMsg expectedErrMsg, String expectedMatchedStr) { - HiveException he = ss.getHiveException(e); - assertEquals(expectedErrMsg, he.getCanonicalErrorMsg()); - if (expectedMatchedStr != null) { - assertEquals(expectedMatchedStr, ss.getMatchedString()); - } - } - - /** - * Force a Spark config to be generated and check that a config value has the expected value - * @param conf the Hive config to use as a base - * @param paramName the Spark config name to check - * @param expectedValue the expected value in the Spark config - */ - private void checkSparkConf(HiveConf conf, String paramName, String expectedValue) throws HiveException { - SparkSessionManager sessionManager = SparkSessionManagerImpl.getInstance(); - SparkSessionImpl sparkSessionImpl = (SparkSessionImpl) - sessionManager.getSession(null, conf, true); - assertTrue(sparkSessionImpl.isOpen()); - HiveSparkClient hiveSparkClient = sparkSessionImpl.getHiveSparkClient(); - SparkConf sparkConf = hiveSparkClient.getSparkConf(); - String cloneConfig = sparkConf.get(paramName); - sessionManager.closeSession(sparkSessionImpl); - assertEquals(expectedValue, cloneConfig); - sessionManager.shutdown(); - } - - /* Thread simulating a user session in HiveServer2. */ - public class SessionThread implements Runnable { - - - @Override - public void run() { - try { - Random random = new Random(Thread.currentThread().getId()); - String threadName = Thread.currentThread().getName(); - System.out.println(threadName + " started."); - HiveConf conf = new HiveConf(); - conf.set("spark.master", "local"); - - SparkSession prevSession = null; - SparkSession currentSession = null; - - for(int i = 0; i < 5; i++) { - currentSession = sessionManagerHS2.getSession(prevSession, conf, true); - assertTrue(prevSession == null || prevSession == currentSession); - assertTrue(currentSession.isOpen()); - System.out.println(String.format("%s got session (%d): %s", - threadName, i, currentSession.getSessionId())); - Thread.sleep((random.nextInt(3)+1) * 1000); - - sessionManagerHS2.returnSession(currentSession); - prevSession = currentSession; - } - sessionManagerHS2.closeSession(currentSession); - System.out.println(threadName + " ended."); - } catch (Throwable e) { - anyFailedSessionThread = true; - String msg = String.format("Error executing '%s'", Thread.currentThread().getName()); - LOG.error(msg, e); - fail(msg + " " + StringUtils.stringifyException(e)); - } - } - } -} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java index 4a6bc51dd89..04c999deaab 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/tez/TestTezTask.java @@ -310,7 +310,6 @@ public void tezTask_updates_Metrics() throws IOException { tezTask.updateTaskMetrics(mockMetrics); verify(mockMetrics, times(1)).incrementCounter(MetricsConstant.HIVE_TEZ_TASKS); - verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_SPARK_TASKS); verify(mockMetrics, never()).incrementCounter(MetricsConstant.HIVE_MR_TASKS); } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java index 1239ca58878..25a0eb474c3 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestExecutionEngineWorkConcurrency.java @@ -43,7 +43,7 @@ public final class TestExecutionEngineWorkConcurrency { @Parameterized.Parameters public static Collection data() { - return Arrays.asList(new Object[][]{{new TezDagIdProvider()}, {new SparkDagIdProvider()}}); + return Arrays.asList(new Object[][]{{new TezDagIdProvider()} }); } private final ExecutionEngineDagIdGenerator executionEngineDagIdGenerator; @@ -108,12 +108,4 @@ public String getDagId() { return new TezWork("query-id").getDagId(); } } - - private static final class SparkDagIdProvider implements ExecutionEngineDagIdGenerator { - - @Override - public String getDagId() { - return new SparkWork("query-id").getName(); - } - } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java index c6902d1c19d..31f6474c9e4 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java @@ -34,26 +34,4 @@ public class TestResetProcessor { - @Ignore - public void testResetClosesSparkSession() throws Exception { - SessionState mockSessionState = createMockSparkSessionState(); - new ResetProcessor().run(mockSessionState, ""); - verify(mockSessionState).closeSparkSession(); - } - - @Ignore - public void testResetExecutionEngineClosesSparkSession() throws Exception { - SessionState mockSessionState = createMockSparkSessionState(); - new ResetProcessor().run(mockSessionState, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname); - verify(mockSessionState).closeSparkSession(); - } - - private static SessionState createMockSparkSessionState() { - SessionState mockSessionState = mock(SessionState.class); - Map overriddenConfigurations = new HashMap<>(); - overriddenConfigurations.put(HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname, "spark"); - when(mockSessionState.getOverriddenConfigurations()).thenReturn(overriddenConfigurations); - when(mockSessionState.getConf()).thenReturn(new HiveConf()); - return mockSessionState; - } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUtils.java b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUtils.java index 4add29027d2..3a1fa955456 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUtils.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/stats/TestStatsUtils.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.plan.ColStatistics.Range; import org.apache.hadoop.hive.serde.serdeConstants; import org.junit.Test; -import org.spark_project.guava.collect.Sets; public class TestStatsUtils { @@ -75,30 +74,4 @@ private boolean rangeContains(Range range, Number f) { return m <= v && v <= M; } - @Test - public void testPrimitiveSizeEstimations() throws Exception { - HiveConf conf = new HiveConf(); - Set exclusions = Sets.newHashSet(); - exclusions.add(serdeConstants.VOID_TYPE_NAME); - exclusions.add(serdeConstants.LIST_TYPE_NAME); - exclusions.add(serdeConstants.MAP_TYPE_NAME); - exclusions.add(serdeConstants.STRUCT_TYPE_NAME); - exclusions.add(serdeConstants.UNION_TYPE_NAME); - Field[] serdeFields = serdeConstants.class.getFields(); - for (Field field : serdeFields) { - if (!Modifier.isStatic(field.getModifiers())) { - continue; - } - if (!field.getName().endsWith("_TYPE_NAME")) { - continue; - } - String typeName = (String) FieldUtils.readStaticField(field); - if (exclusions.contains(typeName)) { - continue; - } - long siz = StatsUtils.getSizeOfPrimitiveTypeArraysFromType(typeName, 3, conf); - assertNotEquals(field.toString(), 0, siz); - } - } - -} \ No newline at end of file +} diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFBinarySetFunctions.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFBinarySetFunctions.java index 3d33ea1b587..5040d133cbe 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFBinarySetFunctions.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/generic/TestGenericUDAFBinarySetFunctions.java @@ -38,7 +38,7 @@ import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; -import jersey.repackaged.com.google.common.collect.Lists; + @RunWith(Parameterized.class) public class TestGenericUDAFBinarySetFunctions { @@ -88,7 +88,9 @@ List run(List values) throws Exception { Object r1 = runComplete(values); Object r2 = runPartialFinal(values); Object r3 = runPartial2Final(values); - return Lists.newArrayList(r1, r2, r3); + List ret = new ArrayList<>(); + ret.add(r1); ret.add(r2); ret.add(r3); + return ret; } private Object runComplete(List values) throws SemanticException, HiveException { diff --git a/ql/src/test/queries/clientnegative/spark_job_max_tasks.q b/ql/src/test/queries/clientnegative/spark_job_max_tasks.q index a638f83cc2b..4e43613d01e 100644 --- a/ql/src/test/queries/clientnegative/spark_job_max_tasks.q +++ b/ql/src/test/queries/clientnegative/spark_job_max_tasks.q @@ -1,4 +1,3 @@ -set hive.spark.job.max.tasks=2; add file ../../data/scripts/sleep.py; diff --git a/ql/src/test/queries/clientnegative/spark_stage_max_tasks.q b/ql/src/test/queries/clientnegative/spark_stage_max_tasks.q index fd43b67cb65..4e43613d01e 100644 --- a/ql/src/test/queries/clientnegative/spark_stage_max_tasks.q +++ b/ql/src/test/queries/clientnegative/spark_stage_max_tasks.q @@ -1,4 +1,3 @@ -set hive.spark.stage.max.tasks=1; add file ../../data/scripts/sleep.py; diff --git a/ql/src/test/queries/clientpositive/autoColumnStats_6.q b/ql/src/test/queries/clientpositive/autoColumnStats_6.q index 45e5daa3ebe..2cacc6e88d7 100644 --- a/ql/src/test/queries/clientpositive/autoColumnStats_6.q +++ b/ql/src/test/queries/clientpositive/autoColumnStats_6.q @@ -4,7 +4,6 @@ set hive.explain.user=false; set hive.merge.orcfile.stripe.level=true; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; -set hive.merge.sparkfiles=true; DROP TABLE orcfile_merge2a; diff --git a/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q b/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q index 2ccc6f8dd4c..663519f1c8f 100644 --- a/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q +++ b/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q @@ -4,7 +4,6 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; ; set hive.exec.reducers.max = 1; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; CREATE TABLE bucket2_1_n0(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS; diff --git a/ql/src/test/queries/clientpositive/merge1.q b/ql/src/test/queries/clientpositive/merge1.q index 84e3b7054fe..512ecc6116c 100644 --- a/ql/src/test/queries/clientpositive/merge1.q +++ b/ql/src/test/queries/clientpositive/merge1.q @@ -1,7 +1,6 @@ set hive.mapred.mode=nonstrict; set hive.explain.user=false; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- SORT_QUERY_RESULTS diff --git a/ql/src/test/queries/clientpositive/merge2.q b/ql/src/test/queries/clientpositive/merge2.q index 1a65c0d103f..c4289f1d77b 100644 --- a/ql/src/test/queries/clientpositive/merge2.q +++ b/ql/src/test/queries/clientpositive/merge2.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.explain.user=false; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; set mapred.min.split.size=256; set mapred.min.split.size.per.node=256; set mapred.min.split.size.per.rack=256; diff --git a/ql/src/test/queries/clientpositive/merge_empty.q b/ql/src/test/queries/clientpositive/merge_empty.q index 92946124de9..4bb77c3ec62 100644 --- a/ql/src/test/queries/clientpositive/merge_empty.q +++ b/ql/src/test/queries/clientpositive/merge_empty.q @@ -1,5 +1,4 @@ set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; set hive.auto.convert.join=false; set mapreduce.job.reduces=1000; diff --git a/ql/src/test/queries/clientpositive/mm_all.q b/ql/src/test/queries/clientpositive/mm_all.q index a524c29ef54..5f9fb13aaa9 100644 --- a/ql/src/test/queries/clientpositive/mm_all.q +++ b/ql/src/test/queries/clientpositive/mm_all.q @@ -52,7 +52,6 @@ drop table dp_mm; set hive.exec.dynamic.partition.mode=nonstrict; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; set hive.merge.tezfiles=false; create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc @@ -224,7 +223,6 @@ select * from multi0_1_mm order by key, key2; select * from multi0_2_mm order by key, key2; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; set hive.merge.tezfiles=true; from intermediate_n0 @@ -235,7 +233,6 @@ select * from multi0_1_mm order by key, key2; select * from multi0_2_mm order by key, key2; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; set hive.merge.tezfiles=false; drop table multi0_1_mm; diff --git a/ql/src/test/queries/clientpositive/orc_merge1.q b/ql/src/test/queries/clientpositive/orc_merge1.q index 7fb5d908e80..2800aae5e41 100644 --- a/ql/src/test/queries/clientpositive/orc_merge1.q +++ b/ql/src/test/queries/clientpositive/orc_merge1.q @@ -13,7 +13,6 @@ set tez.grouping.max-size=2000; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; -- SORT_QUERY_RESULTS @@ -43,7 +42,6 @@ dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/orcfile_merge1_n1/ds=1/part=0/; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- auto-merge slow way EXPLAIN INSERT OVERWRITE TABLE orcfile_merge1b_n1 PARTITION (ds='1', part) diff --git a/ql/src/test/queries/clientpositive/orc_merge2.q b/ql/src/test/queries/clientpositive/orc_merge2.q index 1ab532339ce..7e85f5ad8ae 100644 --- a/ql/src/test/queries/clientpositive/orc_merge2.q +++ b/ql/src/test/queries/clientpositive/orc_merge2.q @@ -4,7 +4,6 @@ set hive.explain.user=false; set hive.merge.orcfile.stripe.level=true; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; -set hive.merge.sparkfiles=true; DROP TABLE orcfile_merge2a_n0; diff --git a/ql/src/test/queries/clientpositive/orc_merge3.q b/ql/src/test/queries/clientpositive/orc_merge3.q index 3abfef0d614..d92fc1c47f2 100644 --- a/ql/src/test/queries/clientpositive/orc_merge3.q +++ b/ql/src/test/queries/clientpositive/orc_merge3.q @@ -2,7 +2,6 @@ set hive.vectorized.execution.enabled=false; set hive.mapred.mode=nonstrict; set hive.explain.user=false; set hive.merge.orcfile.stripe.level=true; -set hive.merge.sparkfiles=true; DROP TABLE orcfile_merge3a_n0; DROP TABLE orcfile_merge3b_n0; diff --git a/ql/src/test/queries/clientpositive/orc_merge4.q b/ql/src/test/queries/clientpositive/orc_merge4.q index ed0a2927a2b..a063eeddd50 100644 --- a/ql/src/test/queries/clientpositive/orc_merge4.q +++ b/ql/src/test/queries/clientpositive/orc_merge4.q @@ -11,14 +11,12 @@ CREATE TABLE orcfile_merge3a (key int, value string) CREATE TABLE orcfile_merge3b (key int, value string) STORED AS TEXTFILE; set hive.merge.mapfiles=false; -set hive.merge.sparkfiles=false; INSERT OVERWRITE TABLE orcfile_merge3a PARTITION (ds='1') SELECT * FROM src; dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/orcfile_merge3a/ds=1/; set hive.merge.mapfiles=true; -set hive.merge.sparkfiles=true; INSERT OVERWRITE TABLE orcfile_merge3a PARTITION (ds='1') SELECT * FROM src; diff --git a/ql/src/test/queries/clientpositive/orc_merge5.q b/ql/src/test/queries/clientpositive/orc_merge5.q index 4ae5ba66122..1b42978e72b 100644 --- a/ql/src/test/queries/clientpositive/orc_merge5.q +++ b/ql/src/test/queries/clientpositive/orc_merge5.q @@ -19,7 +19,6 @@ set hive.merge.mapredfiles=false; set hive.compute.splits.in.am=true; set tez.grouping.min-size=1000; set tez.grouping.max-size=50000; -set hive.merge.sparkfiles=false; -- 3 mappers explain insert overwrite table orc_merge5b_n0 select userid,string1,subtype,decimal1,ts from orc_merge5_n5 where userid<=13; @@ -34,7 +33,6 @@ set hive.merge.orcfile.stripe.level=true; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- 3 mappers explain insert overwrite table orc_merge5b_n0 select userid,string1,subtype,decimal1,ts from orc_merge5_n5 where userid<=13; @@ -49,7 +47,6 @@ set hive.merge.orcfile.stripe.level=false; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; insert overwrite table orc_merge5b_n0 select userid,string1,subtype,decimal1,ts from orc_merge5_n5 where userid<=13; analyze table orc_merge5b_n0 compute statistics noscan; diff --git a/ql/src/test/queries/clientpositive/orc_merge6.q b/ql/src/test/queries/clientpositive/orc_merge6.q index 1c7ab087562..6163772920b 100644 --- a/ql/src/test/queries/clientpositive/orc_merge6.q +++ b/ql/src/test/queries/clientpositive/orc_merge6.q @@ -20,7 +20,6 @@ set hive.merge.mapredfiles=false; set hive.compute.splits.in.am=true; set tez.grouping.min-size=1000; set tez.grouping.max-size=50000; -set hive.merge.sparkfiles=false; -- 3 mappers explain insert overwrite table orc_merge5a_n1 partition (year="2000",hour=24) select userid,string1,subtype,decimal1,ts from orc_merge5_n4 where userid<=13; @@ -39,7 +38,6 @@ set hive.merge.orcfile.stripe.level=true; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- 3 mappers explain insert overwrite table orc_merge5a_n1 partition (year="2000",hour=24) select userid,string1,subtype,decimal1,ts from orc_merge5_n4 where userid<=13; @@ -58,7 +56,6 @@ set hive.merge.orcfile.stripe.level=false; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; insert overwrite table orc_merge5a_n1 partition (year="2000",hour=24) select userid,string1,subtype,decimal1,ts from orc_merge5_n4 where userid<=13; insert overwrite table orc_merge5a_n1 partition (year="2001",hour=24) select userid,string1,subtype,decimal1,ts from orc_merge5_n4 where userid<=13; diff --git a/ql/src/test/queries/clientpositive/orc_merge7.q b/ql/src/test/queries/clientpositive/orc_merge7.q index 6504989038a..c0ed11e01b0 100644 --- a/ql/src/test/queries/clientpositive/orc_merge7.q +++ b/ql/src/test/queries/clientpositive/orc_merge7.q @@ -24,7 +24,6 @@ set tez.grouping.max-size=50000; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; set hive.optimize.sort.dynamic.partition=false; -set hive.merge.sparkfiles=false; -- 3 mappers explain insert overwrite table orc_merge5a_n0 partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5_n2; @@ -43,7 +42,6 @@ set hive.merge.orcfile.stripe.level=true; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- 3 mappers explain insert overwrite table orc_merge5a_n0 partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5_n2; @@ -62,7 +60,6 @@ set hive.merge.orcfile.stripe.level=false; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; insert overwrite table orc_merge5a_n0 partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5_n2; insert overwrite table orc_merge5a_n0 partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5_n2; diff --git a/ql/src/test/queries/clientpositive/orc_merge8.q b/ql/src/test/queries/clientpositive/orc_merge8.q index b6a4260b04f..6679a0f1846 100644 --- a/ql/src/test/queries/clientpositive/orc_merge8.q +++ b/ql/src/test/queries/clientpositive/orc_merge8.q @@ -32,7 +32,6 @@ set hive.merge.orcfile.stripe.level=false; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; insert overwrite table alltypes_orc_n1 select * from alltypes_n1; insert into table alltypes_orc_n1 select * from alltypes_n1; @@ -43,7 +42,6 @@ set hive.merge.orcfile.stripe.level=true; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; alter table alltypes_orc_n1 concatenate; diff --git a/ql/src/test/queries/clientpositive/orc_merge9.q b/ql/src/test/queries/clientpositive/orc_merge9.q index 3864d717a43..dfc2987bd3c 100644 --- a/ql/src/test/queries/clientpositive/orc_merge9.q +++ b/ql/src/test/queries/clientpositive/orc_merge9.q @@ -17,7 +17,6 @@ set hive.merge.orcfile.stripe.level=true; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; select count(*) from ts_merge; alter table ts_merge concatenate; diff --git a/ql/src/test/queries/clientpositive/orc_merge_diff_fs.q b/ql/src/test/queries/clientpositive/orc_merge_diff_fs.q index 1ccdd07a41a..1758112efd8 100644 --- a/ql/src/test/queries/clientpositive/orc_merge_diff_fs.q +++ b/ql/src/test/queries/clientpositive/orc_merge_diff_fs.q @@ -13,7 +13,6 @@ set tez.grouping.max-size=2000; set hive.merge.tezfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; set hive.metastore.warehouse.dir=pfile://${system:test.tmp.dir}/orc_merge_diff_fs; @@ -45,7 +44,6 @@ dfs -ls ${hiveconf:hive.metastore.warehouse.dir}/orcfile_merge1_n0/ds=1/part=0/; set hive.merge.tezfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; -set hive.merge.sparkfiles=true; -- auto-merge slow way EXPLAIN INSERT OVERWRITE TABLE orcfile_merge1b_n0 PARTITION (ds='1', part) diff --git a/ql/src/test/queries/clientpositive/orc_merge_incompat1.q b/ql/src/test/queries/clientpositive/orc_merge_incompat1.q index 2b768eafca8..ae86bc773a4 100644 --- a/ql/src/test/queries/clientpositive/orc_merge_incompat1.q +++ b/ql/src/test/queries/clientpositive/orc_merge_incompat1.q @@ -12,7 +12,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; set hive.merge.orcfile.stripe.level=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; -- 3 mappers explain insert overwrite table orc_merge5b select userid,string1,subtype,decimal1,ts from orc_merge5_n3 where userid<=13; diff --git a/ql/src/test/queries/clientpositive/orc_merge_incompat2.q b/ql/src/test/queries/clientpositive/orc_merge_incompat2.q index 6281c962fe8..87629bf057f 100644 --- a/ql/src/test/queries/clientpositive/orc_merge_incompat2.q +++ b/ql/src/test/queries/clientpositive/orc_merge_incompat2.q @@ -25,7 +25,6 @@ set tez.am.grouping.max-size=50000; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; set hive.optimize.sort.dynamic.partition=false; -set hive.merge.sparkfiles=false; explain insert overwrite table orc_merge5a partition (st) select userid,string1,subtype,decimal1,ts,subtype from orc_merge5; set hive.exec.orc.default.row.index.stride=1000; diff --git a/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q index 9d5571bea83..dc2f14129b0 100644 --- a/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q +++ b/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q @@ -6,7 +6,6 @@ set hive.optimize.union.remove=true; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; set mapred.input.dir.recursive=true; -- This is to test the union->selectstar->filesink and skewjoin optimization diff --git a/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q index 06ebfdcb53f..74cda42f00a 100644 --- a/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q +++ b/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q @@ -6,7 +6,6 @@ set hive.optimize.union.remove=true; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; -set hive.merge.sparkfiles=false; set mapred.input.dir.recursive=true; CREATE TABLE T1_n8(key STRING, val STRING) diff --git a/ql/src/test/queries/clientpositive/spark_combine_equivalent_work_2.q b/ql/src/test/queries/clientpositive/spark_combine_equivalent_work_2.q index 2ff786a448b..f0d4c71c172 100644 --- a/ql/src/test/queries/clientpositive/spark_combine_equivalent_work_2.q +++ b/ql/src/test/queries/clientpositive/spark_combine_equivalent_work_2.q @@ -19,7 +19,6 @@ FROM ( FROM (SELECT * FROM src LIMIT 1) s2 ) a; -set hive.combine.equivalent.work.optimization = false; explain SELECT * diff --git a/ql/src/test/queries/clientpositive/spark_constprog_dpp.q b/ql/src/test/queries/clientpositive/spark_constprog_dpp.q index 44c705e8491..668e58037cd 100644 --- a/ql/src/test/queries/clientpositive/spark_constprog_dpp.q +++ b/ql/src/test/queries/clientpositive/spark_constprog_dpp.q @@ -1,6 +1,5 @@ set hive.mapred.mode=nonstrict; set hive.optimize.constant.propagation=true; -set hive.spark.dynamic.partition.pruning=true; drop table if exists tb1_n0; create table tb1_n0 (id int); diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q index 7aa1a9bd48c..312ea377e0a 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning.q @@ -1,7 +1,6 @@ SET hive.vectorized.execution.enabled=false; set hive.optimize.ppd=true; set hive.ppd.remove.duplicatefilters=true; -set hive.spark.dynamic.partition.pruning=true; set hive.optimize.metadataonly=false; set hive.optimize.index.filter=true; set hive.strict.checks.cartesian.product=false; @@ -20,19 +19,15 @@ create table srcpart_double_hour_n1 as select (hr*2) as hr, hr as hour from srcp -- single column, single key -- join a partitioned table to a non-partitioned table, static filter on the non-partitioned table EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = '2008-04-08'; -- single column, single key, udf with typechange EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (day(srcpart.ds) = day(srcpart_date_n4.ds)) where srcpart_date_n4.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n4 on (day(srcpart.ds) = day(srcpart_date_n4.ds)) where srcpart_date_n4.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (day(srcpart.ds) = day(srcpart_date_n4.ds)) where srcpart_date_n4.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n4 on (day(srcpart.ds) = day(srcpart_date_n4.ds)) where srcpart_date_n4.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=true; -- multiple udfs and casts EXPLAIN select count(*) from srcpart join srcpart_date_n4 on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date_n4.ds) as string), "0") as bigint)) + 10) where srcpart_date_n4.`date` = '2008-04-08'; @@ -47,30 +42,24 @@ EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpa where srcpart_date_n4.`date` = '2008-04-08' and srcpart_hour_n1.hour = 11; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) join srcpart_hour_n1 on (srcpart.hr = srcpart_hour_n1.hr) where srcpart_date_n4.`date` = '2008-04-08' and srcpart_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) join srcpart_hour_n1 on (srcpart.hr = srcpart_hour_n1.hr) where srcpart_date_n4.`date` = '2008-04-08' and srcpart_hour_n1.hour = 11; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) join srcpart_hour_n1 on (srcpart.hr = srcpart_hour_n1.hr) where srcpart_date_n4.`date` = '2008-04-08' and srcpart_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; -- multiple columns single source -- filter partitioned table on both partitioned columns via join with non-partitioned table, filter non-partitioned table EXPLAIN select count(*) from srcpart join srcpart_date_hour_n1 on (srcpart.ds = srcpart_date_hour_n1.ds and srcpart.hr = srcpart_date_hour_n1.hr) where srcpart_date_hour_n1.`date` = '2008-04-08' and srcpart_date_hour_n1.hour = 11; select count(*) from srcpart join srcpart_date_hour_n1 on (srcpart.ds = srcpart_date_hour_n1.ds and srcpart.hr = srcpart_date_hour_n1.hr) where srcpart_date_hour_n1.`date` = '2008-04-08' and srcpart_date_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_date_hour_n1 on (srcpart.ds = srcpart_date_hour_n1.ds and srcpart.hr = srcpart_date_hour_n1.hr) where srcpart_date_hour_n1.`date` = '2008-04-08' and srcpart_date_hour_n1.hour = 11; select count(*) from srcpart join srcpart_date_hour_n1 on (srcpart.ds = srcpart_date_hour_n1.ds and srcpart.hr = srcpart_date_hour_n1.hr) where srcpart_date_hour_n1.`date` = '2008-04-08' and srcpart_date_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = '2008-04-08' and hr = 11; -- empty set -- join a partitioned table to a non-partitioned table, static filter on the non-partitioned table that doesn't filter out anything EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = 'I DONT EXIST'; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = 'I DONT EXIST'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = 'I DONT EXIST'; select count(*) from srcpart join srcpart_date_n4 on (srcpart.ds = srcpart_date_n4.ds) where srcpart_date_n4.`date` = 'I DONT EXIST'; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = 'I DONT EXIST'; -- expressions -- triggers DPP with various expressions - e.g. cast, multiplication, division @@ -78,16 +67,13 @@ EXPLAIN select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr = cast(srcpart_double_hour_n1.hr/2 as int)) where srcpart_double_hour_n1.hour = 11; EXPLAIN select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr*2 = srcpart_double_hour_n1.hr) where srcpart_double_hour_n1.hour = 11; select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr*2 = srcpart_double_hour_n1.hr) where srcpart_double_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr = cast(srcpart_double_hour_n1.hr/2 as int)) where srcpart_double_hour_n1.hour = 11; select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr = cast(srcpart_double_hour_n1.hr/2 as int)) where srcpart_double_hour_n1.hour = 11; EXPLAIN select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr*2 = srcpart_double_hour_n1.hr) where srcpart_double_hour_n1.hour = 11; select count(*) from srcpart join srcpart_double_hour_n1 on (srcpart.hr*2 = srcpart_double_hour_n1.hr) where srcpart_double_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where hr = 11; EXPLAIN select count(*) from srcpart join srcpart_double_hour_n1 on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour_n1.hr as string)) where srcpart_double_hour_n1.hour = 11; select count(*) from srcpart join srcpart_double_hour_n1 on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour_n1.hr as string)) where srcpart_double_hour_n1.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where cast(hr as string) = 11; -- parent is reduce tasks -- join a partitioned table to a non-partitioned table, where the non-partitioned table is a subquery, static filter on the subquery diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q index 89031c4b756..6308ac1b532 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_2.q @@ -1,7 +1,6 @@ SET hive.vectorized.execution.enabled=false; set hive.optimize.ppd=true; set hive.ppd.remove.duplicatefilters=true; -set hive.spark.dynamic.partition.pruning=true; set hive.optimize.metadataonly=false; set hive.optimize.index.filter=true; set hive.auto.convert.join=true; @@ -46,7 +45,6 @@ d1.label in ('foo', 'bar') GROUP BY d1.label ORDER BY d1.label; -set hive.spark.dynamic.partition.pruning.max.data.size=1; EXPLAIN SELECT d1.label, count(*), sum(agg.amount) FROM agg_01_n0 agg, @@ -88,7 +86,6 @@ dim_shops_n0 d1 WHERE agg.dim_shops_id = d1.id and agg.dim_shops_id = 1; -set hive.spark.dynamic.partition.pruning.max.data.size=1000000; EXPLAIN SELECT d1.label, count(*), sum(agg.amount) FROM agg_01_n0 agg, @@ -118,7 +115,6 @@ SELECT amount FROM agg_01_n0, dim_shops_n0 WHERE dim_shops_id = id AND label = ' UNION ALL SELECT amount FROM agg_01_n0, dim_shops_n0 WHERE dim_shops_id = id AND label = 'bar'; -set hive.spark.dynamic.partition.pruning.max.data.size=10000; -- Dynamic partition pruning will be removed as data size exceeds the limit; -- and for self join on partitioning column, it should not fail (HIVE-10559). explain diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_3.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_3.q index 4a245f92772..c1d22efbcff 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_3.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_3.q @@ -52,7 +52,6 @@ INSERT INTO TABLE partitioned_table5 PARTITION (part_col1 = 1, part_col2 = 1) VA INSERT INTO TABLE partitioned_table5 PARTITION (part_col1 = 2, part_col2 = 2) VALUES (1), (2), (3), (4), (5), (6); INSERT INTO TABLE partitioned_table5 PARTITION (part_col1 = 3, part_col2 = 3) VALUES (1), (2), (3), (4), (5), (6); -SET hive.spark.dynamic.partition.pruning.map.join.only=true; SET hive.strict.checks.cartesian.product=false; SET hive.auto.convert.join=true; SET hive.auto.convert.join.noconditionaltask.size=15; -- ensure the partitioned tables are treated as big tables diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q index baf231fccbd..23808102448 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q @@ -1,6 +1,4 @@ SET hive.vectorized.execution.enabled=false; -set hive.spark.dynamic.partition.pruning=true; -set hive.combine.equivalent.work.optimization=true; -- This qfile tests whether equivalent DPP sink works are combined. -- When combined, one DPP sink operator will have multiple target columns/works. @@ -32,14 +30,12 @@ union all (select part2_n1.key, part2_n1.value from part2_n1 join src on part2_n1.p=src.key); -- verify result -set hive.spark.dynamic.partition.pruning=false; select * from (select part1.key, part1.value from part1 join src on part1.p=src.key) a union all (select part2_n1.key, part2_n1.value from part2_n1 join src on part2_n1.p=src.key); -set hive.spark.dynamic.partition.pruning=true; -- dpp works should be combined explain @@ -54,14 +50,12 @@ union all (select part2_n1.key, part2_n1.value from part2_n1 join src on part2_n1.q=src.key); -- verify result -set hive.spark.dynamic.partition.pruning=false; select * from (select part1.key, part1.value from part1 join src on part1.p=src.key) a union all (select part2_n1.key, part2_n1.value from part2_n1 join src on part2_n1.q=src.key); -set hive.spark.dynamic.partition.pruning=true; -- target works are already combined explain @@ -106,7 +100,6 @@ union all (select part2_n1.key, part2_n1.value from part2_n1 join top on part2_n1.q=top.key); -- verify result -set hive.spark.dynamic.partition.pruning=false; with top as (select key from src order by key limit 200) @@ -115,7 +108,6 @@ select * from union all (select part2_n1.key, part2_n1.value from part2_n1 join top on part2_n1.q=top.key); -set hive.spark.dynamic.partition.pruning=true; -- dpp works should be combined explain @@ -134,7 +126,6 @@ union all (select part2_n1.key, part2_n1.value from part2_n1 join top on part2_n1.p=top.key and part2_n1.q=top.key); -- verify result -set hive.spark.dynamic.partition.pruning=false; with top as (select key, value from src order by key, value limit 200) @@ -143,7 +134,6 @@ select * from union all (select part2_n1.key, part2_n1.value from part2_n1 join top on part2_n1.p=top.key and part2_n1.q=top.key); -set hive.spark.dynamic.partition.pruning=true; -- dpp works shouldn't be combined explain diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_5.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_5.q index d1eb86600ee..a7c79b453e8 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_5.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_5.q @@ -1,5 +1,4 @@ SET hive.vectorized.execution.enabled=false; -set hive.spark.dynamic.partition.pruning=true; -- This qfile tests whether we can handle nested DPP sinks diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_6.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_6.q index 19284fd4d4a..39b3abb13da 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_6.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_6.q @@ -1,5 +1,4 @@ SET hive.vectorized.execution.enabled=false; -set hive.spark.dynamic.partition.pruning=true; -- SORT_QUERY_RESULTS diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_mapjoin_only.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_mapjoin_only.q index 9aab1a77198..dc556a209d1 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_mapjoin_only.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_mapjoin_only.q @@ -28,7 +28,6 @@ EXPLAIN select * -- both join parts of srcpart_date_n3 and srcpart2 scans will result in partition pruning sink -- scan with srcpart2 will get split resulting in additional spark jobs -- expectation: 3 spark jobs -set hive.spark.dynamic.partition.pruning=true; EXPLAIN select * from srcpart join srcpart_date_n3 on (srcpart.ds = srcpart_date_n3.ds) @@ -38,8 +37,6 @@ EXPLAIN select * -- Restrict dpp to be enabled only for map joins -- expectation: 2 spark jobs -set hive.spark.dynamic.partition.pruning.map.join.only=true; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN select * from srcpart join srcpart_date_n3 on (srcpart.ds = srcpart_date_n3.ds) diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q index 51278e9a579..74bddecccf8 100644 --- a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_recursive_mapjoin.q @@ -1,5 +1,4 @@ SET hive.vectorized.execution.enabled=false; -SET hive.spark.dynamic.partition.pruning=true; SET hive.auto.convert.join=true; SET hive.strict.checks.cartesian.product=false; diff --git a/ql/src/test/queries/clientpositive/spark_explain_groupbyshuffle.q b/ql/src/test/queries/clientpositive/spark_explain_groupbyshuffle.q index cd2cba11ebe..8aff55f9699 100644 --- a/ql/src/test/queries/clientpositive/spark_explain_groupbyshuffle.q +++ b/ql/src/test/queries/clientpositive/spark_explain_groupbyshuffle.q @@ -1,8 +1,6 @@ -set hive.spark.use.groupby.shuffle=true; explain select key, count(value) from src group by key; -set hive.spark.use.groupby.shuffle=false; explain select key, count(value) from src group by key; diff --git a/ql/src/test/queries/clientpositive/spark_explainuser_1.q b/ql/src/test/queries/clientpositive/spark_explainuser_1.q index 7c051b3a1ab..b4e071da283 100644 --- a/ql/src/test/queries/clientpositive/spark_explainuser_1.q +++ b/ql/src/test/queries/clientpositive/spark_explainuser_1.q @@ -2,7 +2,6 @@ set hive.vectorized.execution.enabled=false; set hive.strict.checks.bucketing=false; set hive.mapred.mode=nonstrict; -set hive.spark.explain.user=true; explain create table src_orc_merge_test_part_n0(key int, value string) partitioned by (ds string, ts string) stored as orc; create table src_orc_merge_test_part_n0(key int, value string) partitioned by (ds string, ts string) stored as orc; diff --git a/ql/src/test/queries/clientpositive/spark_opt_shuffle_serde.q b/ql/src/test/queries/clientpositive/spark_opt_shuffle_serde.q index 2c4691a08c9..c12d52f8f2f 100644 --- a/ql/src/test/queries/clientpositive/spark_opt_shuffle_serde.q +++ b/ql/src/test/queries/clientpositive/spark_opt_shuffle_serde.q @@ -1,7 +1,4 @@ -set hive.spark.optimize.shuffle.serde=true; -set hive.spark.use.groupby.shuffle=true; select key, count(*) from src group by key order by key limit 100; -set hive.spark.use.groupby.shuffle=false; select key, count(*) from src group by key order by key limit 100; diff --git a/ql/src/test/queries/clientpositive/spark_union_merge.q b/ql/src/test/queries/clientpositive/spark_union_merge.q index 31210788a97..c9175fae9b1 100644 --- a/ql/src/test/queries/clientpositive/spark_union_merge.q +++ b/ql/src/test/queries/clientpositive/spark_union_merge.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; -- union case: both subqueries are map jobs on same input, followed by filesink -- mostly copied from union.q -set hive.merge.sparkfiles=false; EXPLAIN EXTENDED FROM ( @@ -21,7 +20,6 @@ INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput. dfs -ls ${system:test.warehouse.dir}/union_merge.out; -set hive.merge.sparkfiles=true; EXPLAIN EXTENDED FROM ( diff --git a/ql/src/test/queries/clientpositive/spark_use_op_stats.q b/ql/src/test/queries/clientpositive/spark_use_op_stats.q index b559bc0ab87..06d9c23c94a 100644 --- a/ql/src/test/queries/clientpositive/spark_use_op_stats.q +++ b/ql/src/test/queries/clientpositive/spark_use_op_stats.q @@ -1,5 +1,4 @@ set hive.mapred.mode=nonstrict; -set hive.spark.use.op.stats=false; set hive.auto.convert.join=false; set hive.exec.reducers.bytes.per.reducer=500; diff --git a/ql/src/test/queries/clientpositive/spark_use_ts_stats_for_mapjoin.q b/ql/src/test/queries/clientpositive/spark_use_ts_stats_for_mapjoin.q index b1df562be2b..46b9e9bd02c 100644 --- a/ql/src/test/queries/clientpositive/spark_use_ts_stats_for_mapjoin.q +++ b/ql/src/test/queries/clientpositive/spark_use_ts_stats_for_mapjoin.q @@ -1,6 +1,5 @@ set hive.mapred.mode=nonstrict; set hive.auto.convert.join=true; -set hive.spark.use.ts.stats.for.mapjoin=true; set hive.auto.convert.join.noconditionaltask.size=4000; -- SORT_QUERY_RESULTS diff --git a/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q b/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q index 42d026ee4b2..dbcd0d2f74a 100644 --- a/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q +++ b/ql/src/test/queries/clientpositive/spark_vectorized_dynamic_partition_pruning.q @@ -1,6 +1,5 @@ set hive.optimize.ppd=true; set hive.ppd.remove.duplicatefilters=true; -set hive.spark.dynamic.partition.pruning=true; set hive.optimize.metadataonly=false; set hive.optimize.index.filter=true; set hive.vectorized.execution.enabled=true; @@ -19,19 +18,15 @@ create table srcpart_double_hour stored as orc as select (hr*2) as hr, hr as hou -- single column, single key EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = '2008-04-08'; -- single column, single key, udf with typechange EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (day(srcpart.ds) = day(srcpart_date_n0.ds)) where srcpart_date_n0.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n0 on (day(srcpart.ds) = day(srcpart_date_n0.ds)) where srcpart_date_n0.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (day(srcpart.ds) = day(srcpart_date_n0.ds)) where srcpart_date_n0.`date` = '2008-04-08'; select count(*) from srcpart join srcpart_date_n0 on (day(srcpart.ds) = day(srcpart_date_n0.ds)) where srcpart_date_n0.`date` = '2008-04-08'; -set hive.spark.dynamic.partition.pruning=true; -- multiple udfs and casts EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on abs(negative(cast(concat(cast(day(srcpart.ds) as string), "0") as bigint)) + 10) = abs(negative(cast(concat(cast(day(srcpart_date_n0.ds) as string), "0") as bigint)) + 10) where srcpart_date_n0.`date` = '2008-04-08'; @@ -46,30 +41,24 @@ EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 o where srcpart_date_n0.`date` = '2008-04-08' and srcpart_hour.hour = 11; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date_n0.`date` = '2008-04-08' and srcpart_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date_n0.`date` = '2008-04-08' and srcpart_hour.hour = 11; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) join srcpart_hour on (srcpart.hr = srcpart_hour.hr) where srcpart_date_n0.`date` = '2008-04-08' and srcpart_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where hr = 11 and ds = '2008-04-08'; -- multiple columns single source EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; select count(*) from srcpart join srcpart_date_hour on (srcpart.ds = srcpart_date_hour.ds and srcpart.hr = srcpart_date_hour.hr) where srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = '2008-04-08' and hr = 11; -- empty set EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = 'I DONT EXIST'; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = 'I DONT EXIST'; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = 'I DONT EXIST'; select count(*) from srcpart join srcpart_date_n0 on (srcpart.ds = srcpart_date_n0.ds) where srcpart_date_n0.`date` = 'I DONT EXIST'; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where ds = 'I DONT EXIST'; -- expressions @@ -77,16 +66,13 @@ EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_double_ho select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=false; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; select count(*) from srcpart join srcpart_double_hour on (srcpart.hr = cast(srcpart_double_hour.hr/2 as int)) where srcpart_double_hour.hour = 11; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; select count(*) from srcpart join srcpart_double_hour on (srcpart.hr*2 = srcpart_double_hour.hr) where srcpart_double_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where hr = 11; EXPLAIN VECTORIZATION DETAIL select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; select count(*) from srcpart join srcpart_double_hour on (cast(srcpart.hr*2 as string) = cast(srcpart_double_hour.hr as string)) where srcpart_double_hour.hour = 11; -set hive.spark.dynamic.partition.pruning=true; select count(*) from srcpart where cast(hr as string) = 11; diff --git a/ql/src/test/queries/clientpositive/union_remove_1.q b/ql/src/test/queries/clientpositive/union_remove_1.q index 0a69068acda..45ab07aa525 100644 --- a/ql/src/test/queries/clientpositive/union_remove_1.q +++ b/ql/src/test/queries/clientpositive/union_remove_1.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_10.q b/ql/src/test/queries/clientpositive/union_remove_10.q index 71a08926ddf..b2ea3bc2bfb 100644 --- a/ql/src/test/queries/clientpositive/union_remove_10.q +++ b/ql/src/test/queries/clientpositive/union_remove_10.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_11.q b/ql/src/test/queries/clientpositive/union_remove_11.q index bfd734dd2ad..b40d90a3b3d 100644 --- a/ql/src/test/queries/clientpositive/union_remove_11.q +++ b/ql/src/test/queries/clientpositive/union_remove_11.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_12.q b/ql/src/test/queries/clientpositive/union_remove_12.q index cdddc7173e1..363e9cb274e 100644 --- a/ql/src/test/queries/clientpositive/union_remove_12.q +++ b/ql/src/test/queries/clientpositive/union_remove_12.q @@ -3,7 +3,6 @@ set hive.stats.autogather=false; set hive.optimize.union.remove=true; set hive.auto.convert.join=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_13.q b/ql/src/test/queries/clientpositive/union_remove_13.q index dcf14e0c662..24bc20362dd 100644 --- a/ql/src/test/queries/clientpositive/union_remove_13.q +++ b/ql/src/test/queries/clientpositive/union_remove_13.q @@ -3,7 +3,6 @@ set hive.stats.autogather=false; set hive.optimize.union.remove=true; set hive.auto.convert.join=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_14.q b/ql/src/test/queries/clientpositive/union_remove_14.q index 04e2998957d..5deb95f47c3 100644 --- a/ql/src/test/queries/clientpositive/union_remove_14.q +++ b/ql/src/test/queries/clientpositive/union_remove_14.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.auto.convert.join=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_15.q b/ql/src/test/queries/clientpositive/union_remove_15.q index 096d330c1dc..6ddaf03d78a 100644 --- a/ql/src/test/queries/clientpositive/union_remove_15.q +++ b/ql/src/test/queries/clientpositive/union_remove_15.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; diff --git a/ql/src/test/queries/clientpositive/union_remove_16.q b/ql/src/test/queries/clientpositive/union_remove_16.q index 053528ed9bf..af66474f7a7 100644 --- a/ql/src/test/queries/clientpositive/union_remove_16.q +++ b/ql/src/test/queries/clientpositive/union_remove_16.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_17.q b/ql/src/test/queries/clientpositive/union_remove_17.q index eb9a0933ad6..8e5c7560dc9 100644 --- a/ql/src/test/queries/clientpositive/union_remove_17.q +++ b/ql/src/test/queries/clientpositive/union_remove_17.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; diff --git a/ql/src/test/queries/clientpositive/union_remove_18.q b/ql/src/test/queries/clientpositive/union_remove_18.q index 1c5e9218239..adb234ff2a4 100644 --- a/ql/src/test/queries/clientpositive/union_remove_18.q +++ b/ql/src/test/queries/clientpositive/union_remove_18.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; diff --git a/ql/src/test/queries/clientpositive/union_remove_19.q b/ql/src/test/queries/clientpositive/union_remove_19.q index 75285ee5dbd..585e3a51823 100644 --- a/ql/src/test/queries/clientpositive/union_remove_19.q +++ b/ql/src/test/queries/clientpositive/union_remove_19.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_2.q b/ql/src/test/queries/clientpositive/union_remove_2.q index c0e395ff635..9e15da401b2 100644 --- a/ql/src/test/queries/clientpositive/union_remove_2.q +++ b/ql/src/test/queries/clientpositive/union_remove_2.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_20.q b/ql/src/test/queries/clientpositive/union_remove_20.q index 5343f58d4d3..39c86bf36e3 100644 --- a/ql/src/test/queries/clientpositive/union_remove_20.q +++ b/ql/src/test/queries/clientpositive/union_remove_20.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_21.q b/ql/src/test/queries/clientpositive/union_remove_21.q index d51de6426f6..db3c02f314d 100644 --- a/ql/src/test/queries/clientpositive/union_remove_21.q +++ b/ql/src/test/queries/clientpositive/union_remove_21.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_22.q b/ql/src/test/queries/clientpositive/union_remove_22.q index 134e650abbf..9e9b19e2d8e 100644 --- a/ql/src/test/queries/clientpositive/union_remove_22.q +++ b/ql/src/test/queries/clientpositive/union_remove_22.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_23.q b/ql/src/test/queries/clientpositive/union_remove_23.q index 8ac2093e85f..06d1a2727cc 100644 --- a/ql/src/test/queries/clientpositive/union_remove_23.q +++ b/ql/src/test/queries/clientpositive/union_remove_23.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_24.q b/ql/src/test/queries/clientpositive/union_remove_24.q index ea3c12bb9cc..0e8cd8a5e31 100644 --- a/ql/src/test/queries/clientpositive/union_remove_24.q +++ b/ql/src/test/queries/clientpositive/union_remove_24.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_25.q b/ql/src/test/queries/clientpositive/union_remove_25.q index 0d52c59b11e..90b5cb8cfe7 100644 --- a/ql/src/test/queries/clientpositive/union_remove_25.q +++ b/ql/src/test/queries/clientpositive/union_remove_25.q @@ -3,7 +3,6 @@ set hive.stats.autogather=false; set hive.optimize.union.remove=true; set hive.exec.dynamic.partition=true; set hive.exec.dynamic.partition.mode=nonstrict; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_3.q b/ql/src/test/queries/clientpositive/union_remove_3.q index f2c8541608c..a07e2c79ea4 100644 --- a/ql/src/test/queries/clientpositive/union_remove_3.q +++ b/ql/src/test/queries/clientpositive/union_remove_3.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_4.q b/ql/src/test/queries/clientpositive/union_remove_4.q index 0b2b8185b27..31b2c43a527 100644 --- a/ql/src/test/queries/clientpositive/union_remove_4.q +++ b/ql/src/test/queries/clientpositive/union_remove_4.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_5.q b/ql/src/test/queries/clientpositive/union_remove_5.q index 7c787141efe..68166cf22b4 100644 --- a/ql/src/test/queries/clientpositive/union_remove_5.q +++ b/ql/src/test/queries/clientpositive/union_remove_5.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/ql/src/test/queries/clientpositive/union_remove_6.q b/ql/src/test/queries/clientpositive/union_remove_6.q index 9dd5fb302c3..fef64a8eb00 100644 --- a/ql/src/test/queries/clientpositive/union_remove_6.q +++ b/ql/src/test/queries/clientpositive/union_remove_6.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_7.q b/ql/src/test/queries/clientpositive/union_remove_7.q index 43a5fe18503..44c71a6ba16 100644 --- a/ql/src/test/queries/clientpositive/union_remove_7.q +++ b/ql/src/test/queries/clientpositive/union_remove_7.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_8.q b/ql/src/test/queries/clientpositive/union_remove_8.q index 05a5671f764..0ee5352b79c 100644 --- a/ql/src/test/queries/clientpositive/union_remove_8.q +++ b/ql/src/test/queries/clientpositive/union_remove_8.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=false; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set mapred.input.dir.recursive=true; diff --git a/ql/src/test/queries/clientpositive/union_remove_9.q b/ql/src/test/queries/clientpositive/union_remove_9.q index 475e52c49f4..2bc6d246466 100644 --- a/ql/src/test/queries/clientpositive/union_remove_9.q +++ b/ql/src/test/queries/clientpositive/union_remove_9.q @@ -2,7 +2,6 @@ set hive.mapred.mode=nonstrict; set hive.stats.autogather=false; set hive.optimize.union.remove=true; -set hive.merge.sparkfiles=true; set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=1; diff --git a/spark-client/pom.xml b/spark-client/pom.xml deleted file mode 100644 index 2a559402ad7..00000000000 --- a/spark-client/pom.xml +++ /dev/null @@ -1,205 +0,0 @@ - - - - - 4.0.0 - - - org.apache.hive - hive - 3.1.3 - - - org.apache.hive - hive-spark-client - jar - Hive Spark Remote Client - 3.1.3 - - - .. - true - - - - - com.esotericsoftware - kryo-shaded - - - com.google.guava - guava - - - io.netty - netty-all - - - com.fasterxml.jackson.module - jackson-module-scala_${scala.binary.version} - - - org.apache.hive - hive-common - ${project.version} - - - org.eclipse.jetty.aggregate - jetty-all - - - - - org.apache.spark - spark-core_${scala.binary.version} - ${spark.version} - - - com.esotericsoftware.kryo - kryo - - - org.glassfish.jersey.containers - * - - - org.glassfish.jersey.core - * - - - org.slf4j - slf4j-log4j12 - - - commmons-logging - commons-logging - - - com.fasterxml.jackson.core - jackson-databind - - - com.fasterxml.jackson.module - jackson-module-scala_${scala.binary.version} - - - - - junit - junit - test - - - org.mockito - mockito-all - test - - - com.sun.jersey - jersey-servlet - test - - - org.glassfish.jersey.containers - jersey-container-servlet - ${glassfish.jersey.version} - test - - - org.apache.hive - hive-service-rpc - ${project.version} - - - - - - sources - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - test-jar - - - - - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - - copy-guava-14 - test-compile - - copy - - - - - com.google.guava - guava - 14.0.1 - - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - ${project.build.directory}/dependency/guava-14.0.1.jar - - - com.google.guava:guava - - - true - ${spark.home} - - ${test.redirectToFile} - ${test.redirectToFile} - -Xmx4096m - - - - - - diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java b/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java deleted file mode 100644 index 4dbc4908bf2..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/BaseProtocol.java +++ /dev/null @@ -1,164 +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.hive.spark.client; - -import java.io.Serializable; - -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.client.rpc.RpcDispatcher; -import org.apache.hive.spark.counter.SparkCounters; - -import com.google.common.base.Throwables; - -abstract class BaseProtocol extends RpcDispatcher { - - protected static class CancelJob implements Serializable { - - final String id; - - CancelJob(String id) { - this.id = id; - } - - CancelJob() { - this(null); - } - - } - - protected static class EndSession implements Serializable { - - } - - protected static class Error implements Serializable { - - final String cause; - - Error(String cause) { - this.cause = cause; - } - - Error() { - this(null); - } - - } - - protected static class JobMetrics implements Serializable { - - final String jobId; - final int sparkJobId; - final int stageId; - final long taskId; - final Metrics metrics; - - JobMetrics(String jobId, int sparkJobId, int stageId, long taskId, Metrics metrics) { - this.jobId = jobId; - this.sparkJobId = sparkJobId; - this.stageId = stageId; - this.taskId = taskId; - this.metrics = metrics; - } - - JobMetrics() { - this(null, -1, -1, -1, null); - } - - } - - protected static class JobRequest implements Serializable { - - final String id; - final Job job; - - JobRequest(String id, Job job) { - this.id = id; - this.job = job; - } - - JobRequest() { - this(null, null); - } - - } - - protected static class JobResult implements Serializable { - - final String id; - final T result; - final String error; - final SparkCounters sparkCounters; - - JobResult(String id, T result, Throwable error, SparkCounters sparkCounters) { - this.id = id; - this.result = result; - this.error = error != null ? Throwables.getStackTraceAsString(error) : null; - this.sparkCounters = sparkCounters; - } - - JobResult() { - this(null, null, null, null); - } - - } - - protected static class JobStarted implements Serializable { - - final String id; - - JobStarted(String id) { - this.id = id; - } - - JobStarted() { - this(null); - } - - } - - /** - * Inform the client that a new spark job has been submitted for the client job. - */ - protected static class JobSubmitted implements Serializable { - final String clientJobId; - final int sparkJobId; - - JobSubmitted(String clientJobId, int sparkJobId) { - this.clientJobId = clientJobId; - this.sparkJobId = sparkJobId; - } - - JobSubmitted() { - this(null, -1); - } - } - - protected static class SyncJobRequest implements Serializable { - - final Job job; - - SyncJobRequest(Job job) { - this.job = job; - } - - SyncJobRequest() { - this(null); - } - - } -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/Job.java b/spark-client/src/main/java/org/apache/hive/spark/client/Job.java deleted file mode 100644 index eb075679fc5..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/Job.java +++ /dev/null @@ -1,32 +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.hive.spark.client; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -import java.io.Serializable; - -/** - * Interface for a Spark remote job. - */ -@InterfaceAudience.Private -public interface Job extends Serializable { - - T call(JobContext jc) throws Exception; - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java b/spark-client/src/main/java/org/apache/hive/spark/client/JobContext.java deleted file mode 100644 index c9c975bedcf..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/JobContext.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.hive.spark.client; - -import java.io.File; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; -import org.apache.hive.spark.counter.SparkCounters; - -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaSparkContext; - -/** - * Holds runtime information about the job execution context. - * - * An instance of this class is kept on the node hosting a remote Spark context and is made - * available to jobs being executed via RemoteSparkContext#submit(). - */ -@InterfaceAudience.Private -public interface JobContext { - - /** The shared SparkContext instance. */ - JavaSparkContext sc(); - - /** - * Monitor a job. This allows job-related information (such as metrics) to be communicated - * back to the client. - * - * @return The job (unmodified). - */ - JavaFutureAction monitor( - JavaFutureAction job, SparkCounters sparkCounters, Set cachedRDDIds); - - /** - * Return a map from client job Id to corresponding JavaFutureActions. - */ - Map>> getMonitoredJobs(); - - /** - * Return all added jar path and timestamp which added through AddJarJob. - */ - Map getAddedJars(); - - /** - * Returns a local tmp dir specific to the context - */ - File getLocalTmpDir(); - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java deleted file mode 100644 index b73bcd72235..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/JobContextImpl.java +++ /dev/null @@ -1,84 +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.hive.spark.client; - -import java.io.File; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.hive.spark.counter.SparkCounters; - -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaSparkContext; - -class JobContextImpl implements JobContext { - - private final JavaSparkContext sc; - private final ThreadLocal monitorCb; - private final Map>> monitoredJobs; - private final Map addedJars; - private final File localTmpDir; - - public JobContextImpl(JavaSparkContext sc, File localTmpDir) { - this.sc = sc; - this.monitorCb = new ThreadLocal(); - monitoredJobs = new ConcurrentHashMap>>(); - addedJars = new ConcurrentHashMap<>(); - this.localTmpDir = localTmpDir; - } - - - @Override - public JavaSparkContext sc() { - return sc; - } - - @Override - public JavaFutureAction monitor(JavaFutureAction job, - SparkCounters sparkCounters, Set cachedRDDIds) { - monitorCb.get().call(job, sparkCounters, cachedRDDIds); - return job; - } - - @Override - public Map>> getMonitoredJobs() { - return monitoredJobs; - } - - @Override - public Map getAddedJars() { - return addedJars; - } - - @Override - public File getLocalTmpDir() { - return localTmpDir; - } - - void setMonitorCb(MonitorCallback cb) { - monitorCb.set(cb); - } - - void stop() { - monitoredJobs.clear(); - sc.stop(); - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java b/spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java deleted file mode 100644 index dffd60c2fae..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/JobHandle.java +++ /dev/null @@ -1,104 +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.hive.spark.client; - -import java.io.Serializable; -import java.util.List; -import java.util.concurrent.Future; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -import org.apache.hive.spark.counter.SparkCounters; - -/** - * A handle to a submitted job. Allows for monitoring and controlling of the running remote job. - */ -@InterfaceAudience.Private -public interface JobHandle extends Future { - - /** - * The client job ID. This is unrelated to any Spark jobs that might be triggered by the - * submitted job. - */ - String getClientJobId(); - - /** - * A collection of metrics collected from the Spark jobs triggered by this job. - * - * To collect job metrics on the client, Spark jobs must be registered with JobContext::monitor() - * on the remote end. - */ - MetricsCollection getMetrics(); - - /** - * Get corresponding spark job IDs for this job. - */ - List getSparkJobIds(); - - /** - * Get the SparkCounters for this job. - */ - SparkCounters getSparkCounters(); - - /** - * Return the current state of the job. - */ - State getState(); - - /** - * Return the error if the job has failed. - */ - Throwable getError(); - - /** - * The current state of the submitted job. - */ - static enum State { - SENT, - QUEUED, - STARTED, - CANCELLED, - FAILED, - SUCCEEDED; - } - - /** - * A listener for monitoring the state of the job in the remote context. Callbacks are called - * when the corresponding state change occurs. - */ - static interface Listener { - - void onJobQueued(JobHandle job); - - void onJobStarted(JobHandle job); - - void onJobCancelled(JobHandle job); - - void onJobFailed(JobHandle job, Throwable cause); - - void onJobSucceeded(JobHandle job, T result); - - /** - * Called when a monitored Spark job is started on the remote context. This callback - * does not indicate a state change in the client job's status. - */ - void onSparkJobStarted(JobHandle job, int sparkJobId); - - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java deleted file mode 100644 index 2881252b0ec..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/JobHandleImpl.java +++ /dev/null @@ -1,244 +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.hive.spark.client; - -import java.io.Serializable; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import com.google.common.collect.ImmutableList; -import io.netty.util.concurrent.Promise; - -import org.apache.hive.spark.counter.SparkCounters; - -/** - * A handle to a submitted job. Allows for monitoring and controlling of the running remote job. - */ -class JobHandleImpl implements JobHandle { - - private final SparkClientImpl client; - private final String jobId; - private final MetricsCollection metrics; - private final Promise promise; - private final List sparkJobIds; - private final List> listeners; - private volatile State state; - private volatile SparkCounters sparkCounters; - - JobHandleImpl(SparkClientImpl client, Promise promise, String jobId, - List> listeners) { - this.client = client; - this.jobId = jobId; - this.promise = promise; - this.listeners = ImmutableList.copyOf(listeners); - this.metrics = new MetricsCollection(); - this.sparkJobIds = new CopyOnWriteArrayList(); - this.state = State.SENT; - this.sparkCounters = null; - - synchronized (this.listeners) { - for (Listener listener : this.listeners) { - initializeListener(listener); - } - } - } - - /** Requests a running job to be cancelled. */ - @Override - public boolean cancel(boolean mayInterrupt) { - if (changeState(State.CANCELLED)) { - client.cancel(jobId); - promise.cancel(mayInterrupt); - return true; - } - return false; - } - - @Override - public T get() throws ExecutionException, InterruptedException { - return promise.get(); - } - - @Override - public T get(long timeout, TimeUnit unit) - throws ExecutionException, InterruptedException, TimeoutException { - return promise.get(timeout, unit); - } - - @Override - public boolean isCancelled() { - return promise.isCancelled(); - } - - @Override - public boolean isDone() { - return promise.isDone(); - } - - /** - * The client job ID. This is unrelated to any Spark jobs that might be triggered by the - * submitted job. - */ - @Override - public String getClientJobId() { - return jobId; - } - - /** - * A collection of metrics collected from the Spark jobs triggered by this job. - * - * To collect job metrics on the client, Spark jobs must be registered with JobContext::monitor() - * on the remote end. - */ - @Override - public MetricsCollection getMetrics() { - return metrics; - } - - @Override - public List getSparkJobIds() { - return sparkJobIds; - } - - @Override - public SparkCounters getSparkCounters() { - return sparkCounters; - } - - @Override - public State getState() { - return state; - } - - @Override - public Throwable getError() { - return promise.cause(); - } - - public void setSparkCounters(SparkCounters sparkCounters) { - this.sparkCounters = sparkCounters; - } - - @SuppressWarnings("unchecked") - void setSuccess(Object result) { - // The synchronization here is not necessary, but tests depend on it. - synchronized (listeners) { - promise.setSuccess((T) result); - changeState(State.SUCCEEDED); - } - } - - void setFailure(Throwable error) { - // The synchronization here is not necessary, but tests depend on it. - synchronized (listeners) { - promise.setFailure(error); - changeState(State.FAILED); - } - } - - /** - * Changes the state of this job handle, making sure that illegal state transitions are ignored. - * Fires events appropriately. - * - * As a rule, state transitions can only occur if the current state is "higher" than the current - * state (i.e., has a higher ordinal number) and is not a "final" state. "Final" states are - * CANCELLED, FAILED and SUCCEEDED, defined here in the code as having an ordinal number higher - * than the CANCELLED enum constant. - */ - boolean changeState(State newState) { - synchronized (listeners) { - if (newState.ordinal() > state.ordinal() && state.ordinal() < State.CANCELLED.ordinal()) { - state = newState; - for (Listener listener : listeners) { - fireStateChange(newState, listener); - } - return true; - } - return false; - } - } - - void addSparkJobId(int sparkJobId) { - synchronized (listeners) { - sparkJobIds.add(sparkJobId); - for (Listener listener : listeners) { - listener.onSparkJobStarted(this, sparkJobId); - } - } - } - - private void initializeListener(Listener listener) { - // If current state is a final state, notify of Spark job IDs before notifying about the - // state transition. - if (state.ordinal() >= State.CANCELLED.ordinal()) { - for (Integer id : sparkJobIds) { - listener.onSparkJobStarted(this, id); - } - } - - fireStateChange(state, listener); - - // Otherwise, notify about Spark jobs after the state notification. - if (state.ordinal() < State.CANCELLED.ordinal()) { - for (Integer id : sparkJobIds) { - listener.onSparkJobStarted(this, id); - } - } - } - - private void fireStateChange(State newState, Listener listener) { - switch (newState) { - case SENT: - break; - case QUEUED: - listener.onJobQueued(this); - break; - case STARTED: - listener.onJobStarted(this); - break; - case CANCELLED: - listener.onJobCancelled(this); - break; - case FAILED: - listener.onJobFailed(this, promise.cause()); - break; - case SUCCEEDED: - try { - listener.onJobSucceeded(this, promise.get()); - } catch (Exception e) { - // Shouldn't really happen. - throw new IllegalStateException(e); - } - break; - default: - throw new IllegalStateException(); - } - } - - /** Last attempt at preventing stray jobs from accumulating in SparkClientImpl. */ - @Override - protected void finalize() { - if (!isDone()) { - cancel(true); - } - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java b/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java deleted file mode 100644 index 2f3c0262127..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/MetricsCollection.java +++ /dev/null @@ -1,288 +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.hive.spark.client; - -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; -import org.apache.hive.spark.client.metrics.InputMetrics; -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; -import org.apache.hive.spark.client.metrics.ShuffleWriteMetrics; - -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.base.Predicates; -import com.google.common.collect.Collections2; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - -/** - * Provides metrics collected for a submitted job. - * - * The collected metrics can be analysed at different levels of granularity: - * - Global (all Spark jobs triggered by client job) - * - Spark job - * - Stage - * - Task - * - * Only successful, non-speculative tasks are considered. Metrics are updated as tasks finish, - * so snapshots can be retrieved before the whole job completes. - */ -@InterfaceAudience.Private -public class MetricsCollection { - - private final List taskMetrics = Lists.newArrayList(); - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - - public Metrics getAllMetrics() { - return aggregate(Predicates.alwaysTrue()); - } - - public Set getJobIds() { - Function fun = new Function() { - @Override - public Integer apply(TaskInfo input) { - return input.jobId; - } - }; - return transform(Predicates.alwaysTrue(), fun); - } - - public Metrics getJobMetrics(int jobId) { - return aggregate(new JobFilter(jobId)); - } - - public Set getStageIds(int jobId) { - Function fun = new Function() { - @Override - public Integer apply(TaskInfo input) { - return input.stageId; - } - }; - return transform(new JobFilter(jobId), fun); - } - - public Metrics getStageMetrics(final int jobId, final int stageId) { - return aggregate(new StageFilter(jobId, stageId)); - } - - public Set getTaskIds(int jobId, int stageId) { - Function fun = new Function() { - @Override - public Long apply(TaskInfo input) { - return input.taskId; - } - }; - return transform(new StageFilter(jobId, stageId), fun); - } - - public Metrics getTaskMetrics(final int jobId, final int stageId, final long taskId) { - Predicate filter = new Predicate() { - @Override - public boolean apply(TaskInfo input) { - return jobId == input.jobId && stageId == input.stageId && taskId == input.taskId; - } - }; - lock.readLock().lock(); - try { - Iterator it = Collections2.filter(taskMetrics, filter).iterator(); - if (it.hasNext()) { - return it.next().metrics; - } else { - throw new NoSuchElementException("Task not found."); - } - } finally { - lock.readLock().unlock(); - } - } - - public void addMetrics(int jobId, int stageId, long taskId, Metrics metrics) { - lock.writeLock().lock(); - try { - taskMetrics.add(new TaskInfo(jobId, stageId, taskId, metrics)); - } finally { - lock.writeLock().unlock(); - } - } - - private Set transform(Predicate filter, Function fun) { - lock.readLock().lock(); - try { - Collection filtered = Collections2.filter(taskMetrics, filter); - return Sets.newHashSet(Collections2.transform(filtered, fun)); - } finally { - lock.readLock().unlock(); - } - } - - private Metrics aggregate(Predicate filter) { - lock.readLock().lock(); - try { - // Task metrics. - long executorDeserializeTime = 0L; - long executorDeserializeCpuTime = 0L; - long executorRunTime = 0L; - long executorCpuTime = 0L; - long resultSize = 0L; - long jvmGCTime = 0L; - long resultSerializationTime = 0L; - long memoryBytesSpilled = 0L; - long diskBytesSpilled = 0L; - long taskDurationTime = 0L; - - // Input metrics. - boolean hasInputMetrics = false; - long bytesRead = 0L; - - // Shuffle read metrics. - boolean hasShuffleReadMetrics = false; - int remoteBlocksFetched = 0; - int localBlocksFetched = 0; - long fetchWaitTime = 0L; - long remoteBytesRead = 0L; - - // Shuffle write metrics. - long shuffleBytesWritten = 0L; - long shuffleWriteTime = 0L; - - for (TaskInfo info : Collections2.filter(taskMetrics, filter)) { - Metrics m = info.metrics; - executorDeserializeTime += m.executorDeserializeTime; - executorDeserializeCpuTime += m.executorDeserializeCpuTime; - executorRunTime += m.executorRunTime; - executorCpuTime += m.executorCpuTime; - resultSize += m.resultSize; - jvmGCTime += m.jvmGCTime; - resultSerializationTime += m.resultSerializationTime; - memoryBytesSpilled += m.memoryBytesSpilled; - diskBytesSpilled += m.diskBytesSpilled; - taskDurationTime += m.taskDurationTime; - - if (m.inputMetrics != null) { - hasInputMetrics = true; - bytesRead += m.inputMetrics.bytesRead; - } - - if (m.shuffleReadMetrics != null) { - hasShuffleReadMetrics = true; - remoteBlocksFetched += m.shuffleReadMetrics.remoteBlocksFetched; - localBlocksFetched += m.shuffleReadMetrics.localBlocksFetched; - fetchWaitTime += m.shuffleReadMetrics.fetchWaitTime; - remoteBytesRead += m.shuffleReadMetrics.remoteBytesRead; - } - - if (m.shuffleWriteMetrics != null) { - shuffleBytesWritten += m.shuffleWriteMetrics.shuffleBytesWritten; - shuffleWriteTime += m.shuffleWriteMetrics.shuffleWriteTime; - } - } - - InputMetrics inputMetrics = null; - if (hasInputMetrics) { - inputMetrics = new InputMetrics(bytesRead); - } - - ShuffleReadMetrics shuffleReadMetrics = null; - if (hasShuffleReadMetrics) { - shuffleReadMetrics = new ShuffleReadMetrics( - remoteBlocksFetched, - localBlocksFetched, - fetchWaitTime, - remoteBytesRead); - } - - ShuffleWriteMetrics shuffleWriteMetrics = null; - if (hasShuffleReadMetrics) { - shuffleWriteMetrics = new ShuffleWriteMetrics( - shuffleBytesWritten, - shuffleWriteTime); - } - - return new Metrics( - executorDeserializeTime, - executorDeserializeCpuTime, - executorRunTime, - executorCpuTime, - resultSize, - jvmGCTime, - resultSerializationTime, - memoryBytesSpilled, - diskBytesSpilled, - taskDurationTime, - inputMetrics, - shuffleReadMetrics, - shuffleWriteMetrics); - } finally { - lock.readLock().unlock(); - } - } - - private static class TaskInfo { - final int jobId; - final int stageId; - final long taskId; - final Metrics metrics; - - TaskInfo(int jobId, int stageId, long taskId, Metrics metrics) { - this.jobId = jobId; - this.stageId = stageId; - this.taskId = taskId; - this.metrics = metrics; - } - - } - - private static class JobFilter implements Predicate { - - private final int jobId; - - JobFilter(int jobId) { - this.jobId = jobId; - } - - @Override - public boolean apply(TaskInfo input) { - return jobId == input.jobId; - } - - } - - private static class StageFilter implements Predicate { - - private final int jobId; - private final int stageId; - - StageFilter(int jobId, int stageId) { - this.jobId = jobId; - this.stageId = stageId; - } - - @Override - public boolean apply(TaskInfo input) { - return jobId == input.jobId && stageId == input.stageId; - } - - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java b/spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java deleted file mode 100644 index e1e899ef2af..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/MonitorCallback.java +++ /dev/null @@ -1,30 +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.hive.spark.client; - -import java.util.Set; - -import org.apache.hive.spark.counter.SparkCounters; - -import org.apache.spark.api.java.JavaFutureAction; - -interface MonitorCallback { - - void call(JavaFutureAction future, SparkCounters sparkCounters, Set cachedRDDIds); - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java b/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java deleted file mode 100644 index 255c30583a9..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/RemoteDriver.java +++ /dev/null @@ -1,550 +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.hive.spark.client; - -import com.google.common.base.Throwables; -import com.google.common.io.Files; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.nio.NioEventLoopGroup; - -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.net.InetAddress; -import java.net.URI; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.hive.common.classification.InterfaceAudience; -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.client.rpc.Rpc; -import org.apache.hive.spark.client.rpc.RpcConfiguration; -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.spark.SparkConf; -import org.apache.spark.SparkJobInfo; -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.scheduler.SparkListener; -import org.apache.spark.scheduler.SparkListenerJobEnd; -import org.apache.spark.scheduler.SparkListenerJobStart; -import org.apache.spark.scheduler.SparkListenerTaskEnd; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import scala.Tuple2; - -import com.google.common.base.Joiner; -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -/** - * Driver code for the Spark client library. - */ -@InterfaceAudience.Private -public class RemoteDriver { - - private static final Logger LOG = LoggerFactory.getLogger(RemoteDriver.class); - - private final Map> activeJobs; - private final Object jcLock; - private final Object shutdownLock; - private final ExecutorService executor; - private final NioEventLoopGroup egroup; - private final Rpc clientRpc; - private final DriverProtocol protocol; - // a local temp dir specific to this driver - private final File localTmpDir; - - // Used to queue up requests while the SparkContext is being created. - private final List> jobQueue = Lists.newLinkedList(); - - // jc is effectively final, but it has to be volatile since it's accessed by different - // threads while the constructor is running. - private volatile JobContextImpl jc; - private volatile boolean running; - - private RemoteDriver(String[] args) throws Exception { - this.activeJobs = Maps.newConcurrentMap(); - this.jcLock = new Object(); - this.shutdownLock = new Object(); - localTmpDir = Files.createTempDir(); - - addShutdownHook(); - - SparkConf conf = new SparkConf(); - String serverAddress = null; - int serverPort = -1; - for (int idx = 0; idx < args.length; idx += 2) { - String key = args[idx]; - if (key.equals("--remote-host")) { - serverAddress = getArg(args, idx); - } else if (key.equals("--remote-port")) { - serverPort = Integer.parseInt(getArg(args, idx)); - } else if (key.equals("--client-id")) { - conf.set(SparkClientFactory.CONF_CLIENT_ID, getArg(args, idx)); - } else if (key.equals("--secret")) { - conf.set(SparkClientFactory.CONF_KEY_SECRET, getArg(args, idx)); - } else if (key.equals("--conf")) { - String[] val = getArg(args, idx).split("[=]", 2); - conf.set(val[0], val[1]); - } else { - throw new IllegalArgumentException("Invalid command line: " - + Joiner.on(" ").join(args)); - } - } - - executor = Executors.newCachedThreadPool(); - - LOG.info("Connecting to: {}:{}", serverAddress, serverPort); - - Map mapConf = Maps.newHashMap(); - for (Tuple2 e : conf.getAll()) { - mapConf.put(e._1(), e._2()); - LOG.debug("Remote Driver configured with: " + e._1() + "=" + e._2()); - } - - String clientId = mapConf.get(SparkClientFactory.CONF_CLIENT_ID); - Preconditions.checkArgument(clientId != null, "No client ID provided."); - String secret = mapConf.get(SparkClientFactory.CONF_KEY_SECRET); - Preconditions.checkArgument(secret != null, "No secret provided."); - - int threadCount = new RpcConfiguration(mapConf).getRpcThreadCount(); - this.egroup = new NioEventLoopGroup( - threadCount, - new ThreadFactoryBuilder() - .setNameFormat("Driver-RPC-Handler-%d") - .setDaemon(true) - .build()); - this.protocol = new DriverProtocol(); - - // The RPC library takes care of timing out this. - this.clientRpc = Rpc.createClient(mapConf, egroup, serverAddress, serverPort, - clientId, secret, protocol).get(); - this.running = true; - - this.clientRpc.addListener(new Rpc.Listener() { - @Override - public void rpcClosed(Rpc rpc) { - LOG.warn("Shutting down driver because RPC channel was closed."); - shutdown(null); - } - }); - - try { - JavaSparkContext sc = new JavaSparkContext(conf); - sc.sc().addSparkListener(new ClientListener()); - synchronized (jcLock) { - jc = new JobContextImpl(sc, localTmpDir); - jcLock.notifyAll(); - } - } catch (Exception e) { - LOG.error("Failed to start SparkContext: " + e, e); - shutdown(e); - synchronized (jcLock) { - jcLock.notifyAll(); - } - throw e; - } - - synchronized (jcLock) { - for (Iterator> it = jobQueue.iterator(); it.hasNext();) { - it.next().submit(); - } - } - } - - private void addShutdownHook() { - Runtime.getRuntime().addShutdownHook(new Thread(() -> { - if (running) { - LOG.info("Received signal SIGTERM, attempting safe shutdown of Remote Spark Context"); - protocol.sendErrorMessage("Remote Spark Context was shutdown because it received a SIGTERM " + - "signal. Most likely due to a kill request via YARN."); - shutdown(null); - } - })); - } - - private void run() throws InterruptedException { - synchronized (shutdownLock) { - while (running) { - shutdownLock.wait(); - } - } - executor.shutdownNow(); - try { - FileUtils.deleteDirectory(localTmpDir); - } catch (IOException e) { - LOG.warn("Failed to delete local tmp dir: " + localTmpDir, e); - } - } - - private void submit(JobWrapper job) { - synchronized (jcLock) { - if (jc != null) { - job.submit(); - } else { - LOG.info("SparkContext not yet up, queueing job request."); - jobQueue.add(job); - } - } - } - - private synchronized void shutdown(Throwable error) { - if (running) { - if (error == null) { - LOG.info("Shutting down remote driver."); - } else { - LOG.error("Shutting down remote driver due to error: " + error, error); - } - running = false; - for (JobWrapper job : activeJobs.values()) { - cancelJob(job); - } - if (error != null) { - protocol.sendError(error); - } - if (jc != null) { - jc.stop(); - } - clientRpc.close(); - egroup.shutdownGracefully(); - synchronized (shutdownLock) { - shutdownLock.notifyAll(); - } - } - } - - private boolean cancelJob(JobWrapper job) { - boolean cancelled = false; - for (JavaFutureAction action : job.jobs) { - cancelled |= action.cancel(true); - } - return cancelled | (job.future != null && job.future.cancel(true)); - } - - private String getArg(String[] args, int keyIdx) { - int valIdx = keyIdx + 1; - if (args.length <= valIdx) { - throw new IllegalArgumentException("Invalid command line: " - + Joiner.on(" ").join(args)); - } - return args[valIdx]; - } - - private class DriverProtocol extends BaseProtocol { - - void sendError(Throwable error) { - LOG.debug("Send error to Client: {}", Throwables.getStackTraceAsString(error)); - clientRpc.call(new Error(Throwables.getStackTraceAsString(error))); - } - - void sendErrorMessage(String cause) { - LOG.debug("Send error to Client: {}", cause); - clientRpc.call(new Error(cause)); - } - - void jobFinished(String jobId, T result, - Throwable error, SparkCounters counters) { - LOG.debug("Send job({}) result to Client.", jobId); - clientRpc.call(new JobResult(jobId, result, error, counters)); - } - - void jobStarted(String jobId) { - clientRpc.call(new JobStarted(jobId)); - } - - void jobSubmitted(String jobId, int sparkJobId) { - LOG.debug("Send job({}/{}) submitted to Client.", jobId, sparkJobId); - clientRpc.call(new JobSubmitted(jobId, sparkJobId)); - } - - void sendMetrics(String jobId, int sparkJobId, int stageId, long taskId, Metrics metrics) { - LOG.debug("Send task({}/{}/{}/{}) metric to Client.", jobId, sparkJobId, stageId, taskId); - clientRpc.call(new JobMetrics(jobId, sparkJobId, stageId, taskId, metrics)); - } - - private void handle(ChannelHandlerContext ctx, CancelJob msg) { - JobWrapper job = activeJobs.get(msg.id); - if (job == null || !cancelJob(job)) { - LOG.info("Requested to cancel an already finished job."); - } - } - - private void handle(ChannelHandlerContext ctx, EndSession msg) { - LOG.debug("Shutting down due to EndSession request."); - shutdown(null); - } - - private void handle(ChannelHandlerContext ctx, JobRequest msg) { - LOG.info("Received job request {}", msg.id); - JobWrapper wrapper = new JobWrapper(msg); - activeJobs.put(msg.id, wrapper); - submit(wrapper); - } - - private Object handle(ChannelHandlerContext ctx, SyncJobRequest msg) throws Exception { - // In case the job context is not up yet, let's wait, since this is supposed to be a - // "synchronous" RPC. - if (jc == null) { - synchronized (jcLock) { - while (jc == null) { - jcLock.wait(); - if (!running) { - throw new IllegalStateException("Remote context is shutting down."); - } - } - } - } - - jc.setMonitorCb(new MonitorCallback() { - @Override - public void call(JavaFutureAction future, - SparkCounters sparkCounters, Set cachedRDDIds) { - throw new IllegalStateException( - "JobContext.monitor() is not available for synchronous jobs."); - } - }); - try { - return msg.job.call(jc); - } finally { - jc.setMonitorCb(null); - } - } - - } - - private class JobWrapper implements Callable { - - private final BaseProtocol.JobRequest req; - private final List> jobs; - private final AtomicInteger jobEndReceived; - private int completed; - private SparkCounters sparkCounters; - private Set cachedRDDIds; - private Integer sparkJobId; - - private Future future; - - JobWrapper(BaseProtocol.JobRequest req) { - this.req = req; - this.jobs = Lists.newArrayList(); - completed = 0; - jobEndReceived = new AtomicInteger(0); - this.sparkCounters = null; - this.cachedRDDIds = null; - this.sparkJobId = null; - } - - @Override - public Void call() throws Exception { - protocol.jobStarted(req.id); - - try { - jc.setMonitorCb(new MonitorCallback() { - @Override - public void call(JavaFutureAction future, - SparkCounters sparkCounters, Set cachedRDDIds) { - monitorJob(future, sparkCounters, cachedRDDIds); - } - }); - - T result = req.job.call(jc); - // In case the job is empty, there won't be JobStart/JobEnd events. The only way - // to know if the job has finished is to check the futures here ourselves. - for (JavaFutureAction future : jobs) { - future.get(); - completed++; - LOG.debug("Client job {}: {} of {} Spark jobs finished.", - req.id, completed, jobs.size()); - } - - // If the job is not empty (but runs fast), we have to wait until all the TaskEnd/JobEnd - // events are processed. Otherwise, task metrics may get lost. See HIVE-13525. - if (sparkJobId != null) { - SparkJobInfo sparkJobInfo = jc.sc().statusTracker().getJobInfo(sparkJobId); - if (sparkJobInfo != null && sparkJobInfo.stageIds() != null && - sparkJobInfo.stageIds().length > 0) { - synchronized (jobEndReceived) { - while (jobEndReceived.get() < jobs.size()) { - jobEndReceived.wait(); - } - } - } - } - - SparkCounters counters = null; - if (sparkCounters != null) { - counters = sparkCounters.snapshot(); - } - protocol.jobFinished(req.id, result, null, counters); - } catch (Throwable t) { - // Catch throwables in a best-effort to report job status back to the client. It's - // re-thrown so that the executor can destroy the affected thread (or the JVM can - // die or whatever would happen if the throwable bubbled up). - LOG.error("Failed to run job " + req.id, t); - protocol.jobFinished(req.id, null, t, - sparkCounters != null ? sparkCounters.snapshot() : null); - throw new ExecutionException(t); - } finally { - jc.setMonitorCb(null); - activeJobs.remove(req.id); - releaseCache(); - } - return null; - } - - void submit() { - this.future = executor.submit(this); - } - - void jobDone() { - synchronized (jobEndReceived) { - jobEndReceived.incrementAndGet(); - jobEndReceived.notifyAll(); - } - } - - /** - * Release cached RDDs as soon as the job is done. - * This is different from local Spark client so as - * to save a RPC call/trip, avoid passing cached RDD - * id information around. Otherwise, we can follow - * the local Spark client way to be consistent. - */ - void releaseCache() { - if (cachedRDDIds != null) { - for (Integer cachedRDDId: cachedRDDIds) { - jc.sc().sc().unpersistRDD(cachedRDDId, false); - } - } - } - - private void monitorJob(JavaFutureAction job, - SparkCounters sparkCounters, Set cachedRDDIds) { - jobs.add(job); - if (!jc.getMonitoredJobs().containsKey(req.id)) { - jc.getMonitoredJobs().put(req.id, new CopyOnWriteArrayList>()); - } - jc.getMonitoredJobs().get(req.id).add(job); - this.sparkCounters = sparkCounters; - this.cachedRDDIds = cachedRDDIds; - sparkJobId = job.jobIds().get(0); - protocol.jobSubmitted(req.id, sparkJobId); - } - - } - - private class ClientListener extends SparkListener { - - private final Map stageToJobId = Maps.newHashMap(); - - @Override - public void onJobStart(SparkListenerJobStart jobStart) { - synchronized (stageToJobId) { - for (int i = 0; i < jobStart.stageIds().length(); i++) { - stageToJobId.put((Integer) jobStart.stageIds().apply(i), jobStart.jobId()); - } - } - } - - @Override - public void onJobEnd(SparkListenerJobEnd jobEnd) { - synchronized (stageToJobId) { - for (Iterator> it = stageToJobId.entrySet().iterator(); - it.hasNext();) { - Map.Entry e = it.next(); - if (e.getValue() == jobEnd.jobId()) { - it.remove(); - } - } - } - - String clientId = getClientId(jobEnd.jobId()); - if (clientId != null) { - activeJobs.get(clientId).jobDone(); - } - } - - @Override - public void onTaskEnd(SparkListenerTaskEnd taskEnd) { - if (taskEnd.reason() instanceof org.apache.spark.Success$ - && !taskEnd.taskInfo().speculative()) { - Metrics metrics = new Metrics(taskEnd.taskMetrics(), taskEnd.taskInfo()); - Integer jobId; - synchronized (stageToJobId) { - jobId = stageToJobId.get(taskEnd.stageId()); - } - - // TODO: implement implicit AsyncRDDActions conversion instead of jc.monitor()? - // TODO: how to handle stage failures? - - String clientId = getClientId(jobId); - if (clientId != null) { - protocol.sendMetrics(clientId, jobId, taskEnd.stageId(), - taskEnd.taskInfo().taskId(), metrics); - } - } - } - - /** - * Returns the client job ID for the given Spark job ID. - * - * This will only work for jobs monitored via JobContext#monitor(). Other jobs won't be - * matched, and this method will return `None`. - */ - private String getClientId(Integer jobId) { - for (Map.Entry> e : activeJobs.entrySet()) { - for (JavaFutureAction future : e.getValue().jobs) { - if (future.jobIds().contains(jobId)) { - return e.getKey(); - } - } - } - return null; - } - - } - - public static void main(String[] args) throws Exception { - RemoteDriver rd = new RemoteDriver(args); - try { - rd.run(); - } catch (Exception e) { - // If the main thread throws an exception for some reason, propagate the exception to the - // client and initiate a safe shutdown - if (rd.running) { - rd.protocol.sendError(e); - rd.shutdown(null); - } - throw e; - } - } -} - diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java deleted file mode 100644 index 1922e412a1d..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClient.java +++ /dev/null @@ -1,113 +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.hive.spark.client; - -import java.io.Serializable; -import java.net.URI; -import java.util.List; -import java.util.concurrent.Future; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Defines the API for the Spark remote client. - */ -@InterfaceAudience.Private -public interface SparkClient extends Serializable { - - /** - * Submits a job for asynchronous execution. - * - * @param job The job to execute. - * @return A handle that be used to monitor the job. - */ - JobHandle submit(Job job); - - /** - * Submits a job for asynchronous execution. - * - * @param job The job to execute. - * @param listeners jobhandle listeners to invoke during the job processing - * @return A handle that be used to monitor the job. - */ - JobHandle submit(Job job, List> listeners); - - /** - * Asks the remote context to run a job immediately. - *

- * Normally, the remote context will queue jobs and execute them based on how many worker - * threads have been configured. This method will run the submitted job in the same thread - * processing the RPC message, so that queueing does not apply. - *

- *

- * It's recommended that this method only be used to run code that finishes quickly. This - * avoids interfering with the normal operation of the context. - *

- * Note: the JobContext#monitor() functionality is not available when using this method. - * - * @param job The job to execute. - * @return A future to monitor the result of the job. - */ - Future run(Job job); - - /** - * Stops the remote context. - * - * Any pending jobs will be cancelled, and the remote context will be torn down. - */ - void stop(); - - /** - * Adds a jar file to the running remote context. - * - * Note that the URL should be reachable by the Spark driver process. If running the driver - * in cluster mode, it may reside on a different host, meaning "file:" URLs have to exist - * on that node (and not on the client machine). - * - * @param uri The location of the jar file. - * @return A future that can be used to monitor the operation. - */ - Future addJar(URI uri); - - /** - * Adds a file to the running remote context. - * - * Note that the URL should be reachable by the Spark driver process. If running the driver - * in cluster mode, it may reside on a different host, meaning "file:" URLs have to exist - * on that node (and not on the client machine). - * - * @param uri The location of the file. - * @return A future that can be used to monitor the operation. - */ - Future addFile(URI uri); - - /** - * Get the count of executors. - */ - Future getExecutorCount(); - - /** - * Get default parallelism. For standalone mode, this can be used to get total number of cores. - */ - Future getDefaultParallelism(); - - /** - * Check if remote context is still active. - */ - boolean isActive(); -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java deleted file mode 100644 index fd9b72583ac..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientFactory.java +++ /dev/null @@ -1,91 +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.hive.spark.client; - -import java.io.IOException; -import java.io.PrintStream; -import java.util.Map; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hive.spark.client.rpc.RpcServer; -import org.apache.spark.SparkException; - -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; - -/** - * Factory for SparkClient instances. - */ -@InterfaceAudience.Private -public final class SparkClientFactory { - - /** Used by client and driver to share a client ID for establishing an RPC session. */ - static final String CONF_CLIENT_ID = "spark.client.authentication.client_id"; - - /** Used by client and driver to share a secret for establishing an RPC session. */ - static final String CONF_KEY_SECRET = "spark.client.authentication.secret"; - - private static volatile RpcServer server = null; - private static final Object serverLock = new Object(); - - /** - * Initializes the SparkClient library. Must be called before creating client instances. - * - * @param conf Map containing configuration parameters for the client library. - */ - public static void initialize(Map conf) throws IOException { - if (server == null) { - synchronized (serverLock) { - if (server == null) { - try { - server = new RpcServer(conf); - } catch (InterruptedException ie) { - throw Throwables.propagate(ie); - } - } - } - } - } - - /** Stops the SparkClient library. */ - public static void stop() { - if (server != null) { - synchronized (serverLock) { - if (server != null) { - server.close(); - server = null; - } - } - } - } - - /** - * Instantiates a new Spark client. - * - * @param sparkConf Configuration for the remote Spark application, contains spark.* properties. - * @param hiveConf Configuration for Hive, contains hive.* properties. - */ - public static SparkClient createClient(Map sparkConf, HiveConf hiveConf, - String sessionId) - throws IOException, SparkException { - Preconditions.checkState(server != null, "initialize() not called."); - return new SparkClientImpl(server, sparkConf, hiveConf, sessionId); - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java deleted file mode 100644 index 665ed928985..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientImpl.java +++ /dev/null @@ -1,692 +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.hive.spark.client; - -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; -import static org.apache.hive.spark.client.SparkClientUtilities.HIVE_KRYO_REG_NAME; - -import com.google.common.base.Charsets; -import com.google.common.base.Joiner; -import com.google.common.base.Strings; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.io.Resources; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; - -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.io.PrintStream; -import java.io.Serializable; -import java.io.Writer; -import java.net.URI; -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.UUID; -import java.util.concurrent.Future; -import java.util.concurrent.TimeoutException; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.common.log.LogRedirector; -import org.apache.hadoop.hive.conf.Constants; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.shims.Utils; -import org.apache.hadoop.security.SecurityUtil; -import org.apache.hive.spark.client.rpc.Rpc; -import org.apache.hive.spark.client.rpc.RpcConfiguration; -import org.apache.hive.spark.client.rpc.RpcServer; -import org.apache.spark.SparkContext; -import org.apache.spark.SparkException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class SparkClientImpl implements SparkClient { - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(SparkClientImpl.class); - - private static final long DEFAULT_SHUTDOWN_TIMEOUT = 10000; // In milliseconds - - private static final String OSX_TEST_OPTS = "SPARK_OSX_TEST_OPTS"; - private static final String SPARK_HOME_ENV = "SPARK_HOME"; - private static final String SPARK_HOME_KEY = "spark.home"; - private static final String DRIVER_OPTS_KEY = "spark.driver.extraJavaOptions"; - private static final String EXECUTOR_OPTS_KEY = "spark.executor.extraJavaOptions"; - private static final String DRIVER_EXTRA_CLASSPATH = "spark.driver.extraClassPath"; - private static final String EXECUTOR_EXTRA_CLASSPATH = "spark.executor.extraClassPath"; - - private final Map conf; - private final HiveConf hiveConf; - private final Thread driverThread; - private final Map> jobs; - private final Rpc driverRpc; - private final ClientProtocol protocol; - private volatile boolean isAlive; - - SparkClientImpl(RpcServer rpcServer, Map conf, HiveConf hiveConf, - String sessionid) throws IOException { - this.conf = conf; - this.hiveConf = hiveConf; - this.jobs = Maps.newConcurrentMap(); - - String secret = rpcServer.createSecret(); - this.driverThread = startDriver(rpcServer, sessionid, secret); - this.protocol = new ClientProtocol(); - - try { - // The RPC server will take care of timeouts here. - this.driverRpc = rpcServer.registerClient(sessionid, secret, protocol).get(); - } catch (Throwable e) { - String errorMsg = null; - if (e.getCause() instanceof TimeoutException) { - errorMsg = "Timed out waiting for client to connect.\nPossible reasons include network " + - "issues, errors in remote driver or the cluster has no available resources, etc." + - "\nPlease check YARN or Spark driver's logs for further information."; - } else if (e.getCause() instanceof InterruptedException) { - errorMsg = "Interruption occurred while waiting for client to connect.\nPossibly the Spark session is closed " + - "such as in case of query cancellation." + - "\nPlease refer to HiveServer2 logs for further information."; - } else { - errorMsg = "Error while waiting for client to connect."; - } - LOG.error(errorMsg, e); - driverThread.interrupt(); - try { - driverThread.join(); - } catch (InterruptedException ie) { - // Give up. - LOG.warn("Interrupted before driver thread was finished.", ie); - } - throw Throwables.propagate(e); - } - - driverRpc.addListener(new Rpc.Listener() { - @Override - public void rpcClosed(Rpc rpc) { - if (isAlive) { - LOG.warn("Client RPC channel closed unexpectedly."); - isAlive = false; - } - } - }); - isAlive = true; - } - - @Override - public JobHandle submit(Job job) { - return protocol.submit(job, Collections.>emptyList()); - } - - @Override - public JobHandle submit(Job job, List> listeners) { - return protocol.submit(job, listeners); - } - - @Override - public Future run(Job job) { - return protocol.run(job); - } - - @Override - public void stop() { - if (isAlive) { - isAlive = false; - try { - protocol.endSession(); - } catch (Exception e) { - LOG.warn("Exception while waiting for end session reply.", e); - } finally { - driverRpc.close(); - } - } - - long endTime = System.currentTimeMillis() + DEFAULT_SHUTDOWN_TIMEOUT; - try { - driverThread.join(DEFAULT_SHUTDOWN_TIMEOUT); - } catch (InterruptedException ie) { - LOG.debug("Interrupted before driver thread was finished."); - } - if (endTime - System.currentTimeMillis() <= 0) { - LOG.warn("Timed out shutting down remote driver, interrupting..."); - driverThread.interrupt(); - } - } - - @Override - public Future addJar(URI uri) { - return run(new AddJarJob(uri.toString())); - } - - @Override - public Future addFile(URI uri) { - return run(new AddFileJob(uri.toString())); - } - - @Override - public Future getExecutorCount() { - return run(new GetExecutorCountJob()); - } - - @Override - public Future getDefaultParallelism() { - return run(new GetDefaultParallelismJob()); - } - - @Override - public boolean isActive() { - return isAlive && driverRpc.isActive(); - } - - void cancel(String jobId) { - protocol.cancel(jobId); - } - - private Thread startDriver(final RpcServer rpcServer, final String clientId, final String secret) - throws IOException { - Runnable runnable; - final String serverAddress = rpcServer.getAddress(); - final String serverPort = String.valueOf(rpcServer.getPort()); - - // If a Spark installation is provided, use the spark-submit script. Otherwise, call the - // SparkSubmit class directly, which has some caveats (like having to provide a proper - // version of Guava on the classpath depending on the deploy mode). - String sparkHome = Strings.emptyToNull(conf.get(SPARK_HOME_KEY)); - if (sparkHome == null) { - sparkHome = Strings.emptyToNull(System.getenv(SPARK_HOME_ENV)); - } - if (sparkHome == null) { - sparkHome = Strings.emptyToNull(System.getProperty(SPARK_HOME_KEY)); - } - String sparkLogDir = conf.get("hive.spark.log.dir"); - if (sparkLogDir == null) { - if (sparkHome == null) { - sparkLogDir = "./target/"; - } else { - sparkLogDir = sparkHome + "/logs/"; - } - } - - String osxTestOpts = ""; - if (Strings.nullToEmpty(System.getProperty("os.name")).toLowerCase().contains("mac")) { - osxTestOpts = Strings.nullToEmpty(System.getenv(OSX_TEST_OPTS)); - } - - String driverJavaOpts = Joiner.on(" ").skipNulls().join( - "-Dhive.spark.log.dir=" + sparkLogDir, osxTestOpts, conf.get(DRIVER_OPTS_KEY)); - String executorJavaOpts = Joiner.on(" ").skipNulls().join( - "-Dhive.spark.log.dir=" + sparkLogDir, osxTestOpts, conf.get(EXECUTOR_OPTS_KEY)); - - // Create a file with all the job properties to be read by spark-submit. Change the - // file's permissions so that only the owner can read it. This avoid having the - // connection secret show up in the child process's command line. - File properties = File.createTempFile("spark-submit.", ".properties"); - if (!properties.setReadable(false) || !properties.setReadable(true, true)) { - throw new IOException("Cannot change permissions of job properties file."); - } - properties.deleteOnExit(); - - Properties allProps = new Properties(); - // first load the defaults from spark-defaults.conf if available - try { - URL sparkDefaultsUrl = Thread.currentThread().getContextClassLoader().getResource("spark-defaults.conf"); - if (sparkDefaultsUrl != null) { - LOG.info("Loading spark defaults: " + sparkDefaultsUrl); - allProps.load(new ByteArrayInputStream(Resources.toByteArray(sparkDefaultsUrl))); - } - } catch (Exception e) { - String msg = "Exception trying to load spark-defaults.conf: " + e; - throw new IOException(msg, e); - } - // then load the SparkClientImpl config - for (Map.Entry e : conf.entrySet()) { - allProps.put(e.getKey(), conf.get(e.getKey())); - } - allProps.put(SparkClientFactory.CONF_CLIENT_ID, clientId); - allProps.put(SparkClientFactory.CONF_KEY_SECRET, secret); - allProps.put(DRIVER_OPTS_KEY, driverJavaOpts); - allProps.put(EXECUTOR_OPTS_KEY, executorJavaOpts); - - String isTesting = conf.get("spark.testing"); - if (isTesting != null && isTesting.equalsIgnoreCase("true")) { - String hiveHadoopTestClasspath = Strings.nullToEmpty(System.getenv("HIVE_HADOOP_TEST_CLASSPATH")); - if (!hiveHadoopTestClasspath.isEmpty()) { - String extraDriverClasspath = Strings.nullToEmpty((String)allProps.get(DRIVER_EXTRA_CLASSPATH)); - if (extraDriverClasspath.isEmpty()) { - allProps.put(DRIVER_EXTRA_CLASSPATH, hiveHadoopTestClasspath); - } else { - extraDriverClasspath = extraDriverClasspath.endsWith(File.pathSeparator) ? extraDriverClasspath : extraDriverClasspath + File.pathSeparator; - allProps.put(DRIVER_EXTRA_CLASSPATH, extraDriverClasspath + hiveHadoopTestClasspath); - } - - String extraExecutorClasspath = Strings.nullToEmpty((String)allProps.get(EXECUTOR_EXTRA_CLASSPATH)); - if (extraExecutorClasspath.isEmpty()) { - allProps.put(EXECUTOR_EXTRA_CLASSPATH, hiveHadoopTestClasspath); - } else { - extraExecutorClasspath = extraExecutorClasspath.endsWith(File.pathSeparator) ? extraExecutorClasspath : extraExecutorClasspath + File.pathSeparator; - allProps.put(EXECUTOR_EXTRA_CLASSPATH, extraExecutorClasspath + hiveHadoopTestClasspath); - } - } - } - - Writer writer = new OutputStreamWriter(new FileOutputStream(properties), Charsets.UTF_8); - try { - allProps.store(writer, "Spark Context configuration"); - } finally { - writer.close(); - } - - // Define how to pass options to the child process. If launching in client (or local) - // mode, the driver options need to be passed directly on the command line. Otherwise, - // SparkSubmit will take care of that for us. - String master = conf.get("spark.master"); - Preconditions.checkArgument(master != null, "spark.master is not defined."); - String deployMode = conf.get("spark.submit.deployMode"); - - List argv = Lists.newLinkedList(); - - if (sparkHome != null) { - argv.add(new File(sparkHome, "bin/spark-submit").getAbsolutePath()); - } else { - LOG.info("No spark.home provided, calling SparkSubmit directly."); - argv.add(new File(System.getProperty("java.home"), "bin/java").getAbsolutePath()); - - if (master.startsWith("local") || master.startsWith("mesos") || - SparkClientUtilities.isYarnClientMode(master, deployMode) || - master.startsWith("spark")) { - String mem = conf.get("spark.driver.memory"); - if (mem != null) { - argv.add("-Xms" + mem); - argv.add("-Xmx" + mem); - } - - String cp = conf.get("spark.driver.extraClassPath"); - if (cp != null) { - argv.add("-classpath"); - argv.add(cp); - } - - String libPath = conf.get("spark.driver.extraLibPath"); - if (libPath != null) { - argv.add("-Djava.library.path=" + libPath); - } - - String extra = conf.get(DRIVER_OPTS_KEY); - if (extra != null) { - for (String opt : extra.split("[ ]")) { - if (!opt.trim().isEmpty()) { - argv.add(opt.trim()); - } - } - } - } - - argv.add("org.apache.spark.deploy.SparkSubmit"); - } - - if (SparkClientUtilities.isYarnClusterMode(master, deployMode)) { - String executorCores = conf.get("spark.executor.cores"); - if (executorCores != null) { - argv.add("--executor-cores"); - argv.add(executorCores); - } - - String executorMemory = conf.get("spark.executor.memory"); - if (executorMemory != null) { - argv.add("--executor-memory"); - argv.add(executorMemory); - } - - String numOfExecutors = conf.get("spark.executor.instances"); - if (numOfExecutors != null) { - argv.add("--num-executors"); - argv.add(numOfExecutors); - } - } - // The options --principal/--keypad do not work with --proxy-user in spark-submit.sh - // (see HIVE-15485, SPARK-5493, SPARK-19143), so Hive could only support doAs or - // delegation token renewal, but not both. Since doAs is a more common case, if both - // are needed, we choose to favor doAs. So when doAs is enabled, we use kinit command, - // otherwise, we pass the principal/keypad to spark to support the token renewal for - // long-running application. - if ("kerberos".equals(hiveConf.get(HADOOP_SECURITY_AUTHENTICATION))) { - String principal = SecurityUtil.getServerPrincipal(hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL), - "0.0.0.0"); - String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); - if (StringUtils.isNotBlank(principal) && StringUtils.isNotBlank(keyTabFile)) { - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { - List kinitArgv = Lists.newLinkedList(); - kinitArgv.add("kinit"); - kinitArgv.add(principal); - kinitArgv.add("-k"); - kinitArgv.add("-t"); - kinitArgv.add(keyTabFile + ";"); - kinitArgv.addAll(argv); - argv = kinitArgv; - } else { - // if doAs is not enabled, we pass the principal/keypad to spark-submit in order to - // support the possible delegation token renewal in Spark - argv.add("--principal"); - argv.add(principal); - argv.add("--keytab"); - argv.add(keyTabFile); - } - } - } - if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { - try { - String currentUser = Utils.getUGI().getShortUserName(); - // do not do impersonation in CLI mode - if (!currentUser.equals(System.getProperty("user.name"))) { - LOG.info("Attempting impersonation of " + currentUser); - argv.add("--proxy-user"); - argv.add(currentUser); - } - } catch (Exception e) { - String msg = "Cannot obtain username: " + e; - throw new IllegalStateException(msg, e); - } - } - - String regStr = conf.get("spark.kryo.registrator"); - if (HIVE_KRYO_REG_NAME.equals(regStr)) { - argv.add("--jars"); - argv.add(SparkClientUtilities.findKryoRegistratorJar(hiveConf)); - } - - argv.add("--properties-file"); - argv.add(properties.getAbsolutePath()); - argv.add("--class"); - argv.add(RemoteDriver.class.getName()); - - String jar = "spark-internal"; - if (SparkContext.jarOfClass(this.getClass()).isDefined()) { - jar = SparkContext.jarOfClass(this.getClass()).get(); - } - argv.add(jar); - - argv.add("--remote-host"); - argv.add(serverAddress); - argv.add("--remote-port"); - argv.add(serverPort); - - //hive.spark.* keys are passed down to the RemoteDriver via --conf, - //as --properties-file contains the spark.* keys that are meant for SparkConf object. - for (String hiveSparkConfKey : RpcConfiguration.HIVE_SPARK_RSC_CONFIGS) { - String value = RpcConfiguration.getValue(hiveConf, hiveSparkConfKey); - argv.add("--conf"); - argv.add(String.format("%s=%s", hiveSparkConfKey, value)); - } - - String cmd = Joiner.on(" ").join(argv); - LOG.info("Running client driver with argv: {}", cmd); - ProcessBuilder pb = new ProcessBuilder("sh", "-c", cmd); - - // Prevent hive configurations from being visible in Spark. - pb.environment().remove("HIVE_HOME"); - pb.environment().remove("HIVE_CONF_DIR"); - // Add credential provider password to the child process's environment - // In case of Spark the credential provider location is provided in the jobConf when the job is submitted - String password = getSparkJobCredentialProviderPassword(); - if(password != null) { - pb.environment().put(Constants.HADOOP_CREDENTIAL_PASSWORD_ENVVAR, password); - } - if (isTesting != null) { - pb.environment().put("SPARK_TESTING", isTesting); - } - - final Process child = pb.start(); - String threadName = Thread.currentThread().getName(); - final List childErrorLog = Collections.synchronizedList(new ArrayList()); - final LogRedirector.LogSourceCallback callback = () -> {return isAlive;}; - - LogRedirector.redirect("RemoteDriver-stdout-redir-" + threadName, - new LogRedirector(child.getInputStream(), LOG, callback)); - LogRedirector.redirect("RemoteDriver-stderr-redir-" + threadName, - new LogRedirector(child.getErrorStream(), LOG, childErrorLog, callback)); - - runnable = new Runnable() { - @Override - public void run() { - try { - int exitCode = child.waitFor(); - if (exitCode != 0) { - StringBuilder errStr = new StringBuilder(); - synchronized(childErrorLog) { - Iterator iter = childErrorLog.iterator(); - while(iter.hasNext()){ - errStr.append(iter.next()); - errStr.append('\n'); - } - } - - LOG.warn("Child process exited with code {}", exitCode); - rpcServer.cancelClient(clientId, - "Child process (spark-submit) exited before connecting back with error log " + errStr.toString()); - } - } catch (InterruptedException ie) { - LOG.warn("Thread waiting on the child process (spark-submit) is interrupted, killing the child process."); - rpcServer.cancelClient(clientId, "Thread waiting on the child porcess (spark-submit) is interrupted"); - Thread.interrupted(); - child.destroy(); - } catch (Exception e) { - String errMsg = "Exception while waiting for child process (spark-submit)"; - LOG.warn(errMsg, e); - rpcServer.cancelClient(clientId, errMsg); - } - } - }; - - Thread thread = new Thread(runnable); - thread.setDaemon(true); - thread.setName("Driver"); - thread.start(); - return thread; - } - - private String getSparkJobCredentialProviderPassword() { - if (conf.containsKey("spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD")) { - return conf.get("spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD"); - } else if (conf.containsKey("spark.executorEnv.HADOOP_CREDSTORE_PASSWORD")) { - return conf.get("spark.executorEnv.HADOOP_CREDSTORE_PASSWORD"); - } - return null; - } - - private class ClientProtocol extends BaseProtocol { - - JobHandleImpl submit(Job job, List> listeners) { - final String jobId = UUID.randomUUID().toString(); - final Promise promise = driverRpc.createPromise(); - final JobHandleImpl handle = - new JobHandleImpl(SparkClientImpl.this, promise, jobId, listeners); - jobs.put(jobId, handle); - - final io.netty.util.concurrent.Future rpc = driverRpc.call(new JobRequest(jobId, job)); - LOG.debug("Send JobRequest[{}].", jobId); - - // Link the RPC and the promise so that events from one are propagated to the other as - // needed. - rpc.addListener(new GenericFutureListener>() { - @Override - public void operationComplete(io.netty.util.concurrent.Future f) { - if (f.isSuccess()) { - // If the spark job finishes before this listener is called, the QUEUED status will not be set - handle.changeState(JobHandle.State.QUEUED); - } else if (!promise.isDone()) { - promise.setFailure(f.cause()); - } - } - }); - promise.addListener(new GenericFutureListener>() { - @Override - public void operationComplete(Promise p) { - if (jobId != null) { - jobs.remove(jobId); - } - if (p.isCancelled() && !rpc.isDone()) { - rpc.cancel(true); - } - } - }); - return handle; - } - - Future run(Job job) { - @SuppressWarnings("unchecked") - final io.netty.util.concurrent.Future rpc = (io.netty.util.concurrent.Future) - driverRpc.call(new SyncJobRequest(job), Serializable.class); - return rpc; - } - - void cancel(String jobId) { - driverRpc.call(new CancelJob(jobId)); - } - - Future endSession() { - return driverRpc.call(new EndSession()); - } - - private void handle(ChannelHandlerContext ctx, Error msg) { - LOG.warn("Error reported from remote driver: {}", msg.cause); - } - - private void handle(ChannelHandlerContext ctx, JobMetrics msg) { - JobHandleImpl handle = jobs.get(msg.jobId); - if (handle != null) { - handle.getMetrics().addMetrics(msg.sparkJobId, msg.stageId, msg.taskId, msg.metrics); - } else { - LOG.warn("Received metrics for unknown job {}", msg.jobId); - } - } - - private void handle(ChannelHandlerContext ctx, JobResult msg) { - JobHandleImpl handle = jobs.remove(msg.id); - if (handle != null) { - LOG.info("Received result for {}", msg.id); - handle.setSparkCounters(msg.sparkCounters); - Throwable error = msg.error != null ? new SparkException(msg.error) : null; - if (error == null) { - handle.setSuccess(msg.result); - } else { - handle.setFailure(error); - } - } else { - LOG.warn("Received result for unknown job {}", msg.id); - } - } - - private void handle(ChannelHandlerContext ctx, JobStarted msg) { - JobHandleImpl handle = jobs.get(msg.id); - if (handle != null) { - handle.changeState(JobHandle.State.STARTED); - } else { - LOG.warn("Received event for unknown job {}", msg.id); - } - } - - private void handle(ChannelHandlerContext ctx, JobSubmitted msg) { - JobHandleImpl handle = jobs.get(msg.clientJobId); - if (handle != null) { - LOG.info("Received spark job ID: {} for {}", msg.sparkJobId, msg.clientJobId); - handle.addSparkJobId(msg.sparkJobId); - } else { - LOG.warn("Received spark job ID: {} for unknown job {}", msg.sparkJobId, msg.clientJobId); - } - } - } - - private static class AddJarJob implements Job { - private static final long serialVersionUID = 1L; - - private final String path; - - AddJarJob() { - this(null); - } - - AddJarJob(String path) { - this.path = path; - } - - @Override - public Serializable call(JobContext jc) throws Exception { - jc.sc().addJar(path); - // Following remote job may refer to classes in this jar, and the remote job would be executed - // in a different thread, so we add this jar path to JobContext for further usage. - jc.getAddedJars().put(path, System.currentTimeMillis()); - return null; - } - - } - - private static class AddFileJob implements Job { - private static final long serialVersionUID = 1L; - - private final String path; - - AddFileJob() { - this(null); - } - - AddFileJob(String path) { - this.path = path; - } - - @Override - public Serializable call(JobContext jc) throws Exception { - jc.sc().addFile(path); - return null; - } - - } - - private static class GetExecutorCountJob implements Job { - private static final long serialVersionUID = 1L; - - @Override - public Integer call(JobContext jc) throws Exception { - // minus 1 here otherwise driver is also counted as an executor - int count = jc.sc().sc().getExecutorMemoryStatus().size() - 1; - return Integer.valueOf(count); - } - - } - - private static class GetDefaultParallelismJob implements Job { - private static final long serialVersionUID = 1L; - - @Override - public Integer call(JobContext jc) throws Exception { - return jc.sc().sc().defaultParallelism(); - } - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java deleted file mode 100644 index d3cb3dd7a1c..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java +++ /dev/null @@ -1,189 +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.hive.spark.client; - -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; - -import java.io.File; -import java.io.FileNotFoundException; -import java.net.MalformedURLException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.spark.SparkContext; -import org.apache.spark.util.MutableURLClassLoader; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import scala.Option; - -public class SparkClientUtilities { - protected static final transient Logger LOG = LoggerFactory.getLogger(SparkClientUtilities.class); - - private static final Map downloadedFiles = new ConcurrentHashMap<>(); - - public static final String HIVE_KRYO_REG_NAME = "org.apache.hive.spark.HiveKryoRegistrator"; - private static final String HIVE_KRYO_REG_JAR_NAME = "hive-kryo-registrator"; - - /** - * Add new elements to the classpath. - * - * @param newPaths Map of classpath elements and corresponding timestamp - * @return locally accessible files corresponding to the newPaths - */ - public static List addToClassPath(Map newPaths, Configuration conf, - File localTmpDir) throws Exception { - URLClassLoader loader = (URLClassLoader) Thread.currentThread().getContextClassLoader(); - List curPath = Lists.newArrayList(loader.getURLs()); - List localNewPaths = new ArrayList<>(); - - boolean newPathAdded = false; - for (Map.Entry entry : newPaths.entrySet()) { - URL newUrl = urlFromPathString(entry.getKey(), entry.getValue(), conf, localTmpDir); - localNewPaths.add(newUrl.toString()); - if (newUrl != null && !curPath.contains(newUrl)) { - curPath.add(newUrl); - LOG.info("Added jar[" + newUrl + "] to classpath."); - newPathAdded = true; - } - } - - if (newPathAdded) { - URLClassLoader newLoader = - new URLClassLoader(curPath.toArray(new URL[curPath.size()]), loader); - Thread.currentThread().setContextClassLoader(newLoader); - } - return localNewPaths; - } - - /** - * Create a URL from a string representing a path to a local file. - * The path string can be just a path, or can start with file:/, file:/// - * - * @param path path string - * @return - */ - private static URL urlFromPathString(String path, Long timeStamp, - Configuration conf, File localTmpDir) { - URL url = null; - try { - if (StringUtils.indexOf(path, "file:/") == 0) { - url = new URL(path); - } else if (StringUtils.indexOf(path, "hdfs:/") == 0 - || StringUtils.indexOf(path, "viewfs:/") == 0) { - Path remoteFile = new Path(path); - Path localFile = - new Path(localTmpDir.getAbsolutePath() + File.separator + remoteFile.getName()); - Long currentTS = downloadedFiles.get(path); - if (currentTS == null) { - currentTS = -1L; - } - if (!new File(localFile.toString()).exists() || currentTS < timeStamp) { - LOG.info("Copying " + remoteFile + " to " + localFile); - FileSystem remoteFS = remoteFile.getFileSystem(conf); - remoteFS.copyToLocalFile(remoteFile, localFile); - downloadedFiles.put(path, timeStamp); - } - return urlFromPathString(localFile.toString(), timeStamp, conf, localTmpDir); - } else { - url = new File(path).toURL(); - } - } catch (Exception err) { - LOG.error("Bad URL " + path + ", ignoring path", err); - } - return url; - } - - public static boolean isYarnClusterMode(String master, String deployMode) { - return "yarn-cluster".equals(master) || - ("yarn".equals(master) && "cluster".equals(deployMode)); - } - - public static boolean isYarnClientMode(String master, String deployMode) { - return "yarn-client".equals(master) || - ("yarn".equals(master) && "client".equals(deployMode)); - } - - public static boolean isYarnMaster(String master) { - return master != null && master.startsWith("yarn"); - } - - public static boolean isLocalMaster(String master) { - return master != null && master.startsWith("local"); - } - - public static String getDeployModeFromMaster(String master) { - if (master != null) { - if (master.equals("yarn-client")) { - return "client"; - } else if (master.equals("yarn-cluster")) { - return "cluster"; - } - } - return null; - } - - public static String findKryoRegistratorJar(HiveConf conf) throws FileNotFoundException { - // find the jar in local maven repo for testing - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_IN_TEST)) { - String repo = System.getProperty("maven.local.repository"); - String version = System.getProperty("hive.version"); - String jarName = HIVE_KRYO_REG_JAR_NAME + "-" + version + ".jar"; - String[] parts = new String[]{repo, "org", "apache", "hive", - HIVE_KRYO_REG_JAR_NAME, version, jarName}; - String jar = Joiner.on(File.separator).join(parts); - if (!new File(jar).exists()) { - throw new FileNotFoundException(jar + " doesn't exist."); - } - return jar; - } - Option option = SparkContext.jarOfClass(SparkClientUtilities.class); - if (!option.isDefined()) { - throw new FileNotFoundException("Cannot find the path to hive-exec.jar"); - } - File path = new File(option.get()); - File[] jars = path.getParentFile().listFiles((dir, name) -> - name.startsWith(HIVE_KRYO_REG_JAR_NAME)); - if (jars != null && jars.length > 0) { - return jars[0].getAbsolutePath(); - } - throw new FileNotFoundException("Cannot find the " + HIVE_KRYO_REG_JAR_NAME + - " jar under " + path.getParent()); - } - - public static void addJarToContextLoader(File jar) throws MalformedURLException { - ClassLoader loader = Thread.currentThread().getContextClassLoader(); - if (loader instanceof MutableURLClassLoader) { - ((MutableURLClassLoader) loader).addURL(jar.toURI().toURL()); - } else { - URLClassLoader newLoader = - new URLClassLoader(new URL[]{jar.toURI().toURL()}, loader); - Thread.currentThread().setContextClassLoader(newLoader); - } - } -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/DataReadMethod.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/DataReadMethod.java deleted file mode 100644 index d62b5adfcbf..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/DataReadMethod.java +++ /dev/null @@ -1,29 +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.hive.spark.client.metrics; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Method by which input data was read. Network means that the data was read over the network - * from a remote block manager (which may have stored the data on-disk or in-memory). - */ -@InterfaceAudience.Private -public enum DataReadMethod { - Memory, Disk, Hadoop, Network, Multiple -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java deleted file mode 100644 index f137007b406..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/InputMetrics.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hive.spark.client.metrics; - -import java.io.Serializable; - -import org.apache.spark.executor.TaskMetrics; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Metrics pertaining to reading input data. - */ -@InterfaceAudience.Private -public class InputMetrics implements Serializable { - public final long bytesRead; - - private InputMetrics() { - // For Serialization only. - this(0L); - } - - public InputMetrics( - long bytesRead) { - this.bytesRead = bytesRead; - } - - public InputMetrics(TaskMetrics metrics) { - this(metrics.inputMetrics().bytesRead()); - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java deleted file mode 100644 index b718b3bd951..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/Metrics.java +++ /dev/null @@ -1,130 +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.hive.spark.client.metrics; - -import java.io.Serializable; -import java.util.concurrent.TimeUnit; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.scheduler.TaskInfo; - -/** - * Metrics tracked during the execution of a job. - * - * Depending on how the metrics object is obtained (by calling methods in the `MetricsCollection` - * class), metrics will refer to one or more tasks. - */ -@InterfaceAudience.Private -public class Metrics implements Serializable { - - /** Time taken on the executor to deserialize tasks. */ - public final long executorDeserializeTime; - /** CPU time taken on the executor to deserialize tasks. */ - public final long executorDeserializeCpuTime; - /** Time the executor spends actually running the task (including fetching shuffle data). */ - public final long executorRunTime; - /** CPU time the executor spends running the task (including fetching shuffle data). */ - public final long executorCpuTime; - /** The number of bytes sent back to the driver by tasks. */ - public final long resultSize; - /** Amount of time the JVM spent in garbage collection while executing tasks. */ - public final long jvmGCTime; - /** Amount of time spent serializing task results. */ - public final long resultSerializationTime; - /** The number of in-memory bytes spilled by tasks. */ - public final long memoryBytesSpilled; - /** The number of on-disk bytes spilled by tasks. */ - public final long diskBytesSpilled; - /** Amount of time spent executing tasks. */ - public final long taskDurationTime; - /** If tasks read from a HadoopRDD or from persisted data, metrics on how much data was read. */ - public final InputMetrics inputMetrics; - /** - * If tasks read from shuffle output, metrics on getting shuffle data. This includes read metrics - * aggregated over all the tasks' shuffle dependencies. - */ - public final ShuffleReadMetrics shuffleReadMetrics; - /** If tasks wrote to shuffle output, metrics on the written shuffle data. */ - public final ShuffleWriteMetrics shuffleWriteMetrics; - - private Metrics() { - // For Serialization only. - this(0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, null, null, null); - } - - public Metrics( - long executorDeserializeTime, - long executorDeserializeCpuTime, - long executorRunTime, - long executorCpuTime, - long resultSize, - long jvmGCTime, - long resultSerializationTime, - long memoryBytesSpilled, - long diskBytesSpilled, - long taskDurationTime, - InputMetrics inputMetrics, - ShuffleReadMetrics shuffleReadMetrics, - ShuffleWriteMetrics shuffleWriteMetrics) { - this.executorDeserializeTime = executorDeserializeTime; - this.executorDeserializeCpuTime = executorDeserializeCpuTime; - this.executorRunTime = executorRunTime; - this.executorCpuTime = executorCpuTime; - this.resultSize = resultSize; - this.jvmGCTime = jvmGCTime; - this.resultSerializationTime = resultSerializationTime; - this.memoryBytesSpilled = memoryBytesSpilled; - this.diskBytesSpilled = diskBytesSpilled; - this.taskDurationTime = taskDurationTime; - this.inputMetrics = inputMetrics; - this.shuffleReadMetrics = shuffleReadMetrics; - this.shuffleWriteMetrics = shuffleWriteMetrics; - } - - public Metrics(TaskMetrics metrics, TaskInfo taskInfo) { - this( - metrics.executorDeserializeTime(), - TimeUnit.NANOSECONDS.toMillis(metrics.executorDeserializeCpuTime()), - metrics.executorRunTime(), - TimeUnit.NANOSECONDS.toMillis(metrics.executorCpuTime()), - metrics.resultSize(), - metrics.jvmGCTime(), - metrics.resultSerializationTime(), - metrics.memoryBytesSpilled(), - metrics.diskBytesSpilled(), - taskInfo.duration(), - optionalInputMetric(metrics), - optionalShuffleReadMetric(metrics), - optionalShuffleWriteMetrics(metrics)); - } - - private static InputMetrics optionalInputMetric(TaskMetrics metrics) { - return (metrics.inputMetrics() != null) ? new InputMetrics(metrics) : null; - } - - private static ShuffleReadMetrics optionalShuffleReadMetric(TaskMetrics metrics) { - return (metrics.shuffleReadMetrics() != null) ? new ShuffleReadMetrics(metrics) : null; - } - - private static ShuffleWriteMetrics optionalShuffleWriteMetrics(TaskMetrics metrics) { - return (metrics.shuffleWriteMetrics() != null) ? new ShuffleWriteMetrics(metrics) : null; - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java deleted file mode 100644 index 9ff4d0ff0fd..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleReadMetrics.java +++ /dev/null @@ -1,76 +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.hive.spark.client.metrics; - -import java.io.Serializable; - -import org.apache.spark.executor.TaskMetrics; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Metrics pertaining to reading shuffle data. - */ -@InterfaceAudience.Private -public class ShuffleReadMetrics implements Serializable { - - /** Number of remote blocks fetched in shuffles by tasks. */ - public final long remoteBlocksFetched; - /** Number of local blocks fetched in shuffles by tasks. */ - public final long localBlocksFetched; - /** - * Time tasks spent waiting for remote shuffle blocks. This only includes the - * time blocking on shuffle input data. For instance if block B is being - * fetched while the task is still not finished processing block A, it is not - * considered to be blocking on block B. - */ - public final long fetchWaitTime; - /** Total number of remote bytes read from the shuffle by tasks. */ - public final long remoteBytesRead; - - private ShuffleReadMetrics() { - // For Serialization only. - this(0, 0, 0L, 0L); - } - - public ShuffleReadMetrics( - long remoteBlocksFetched, - long localBlocksFetched, - long fetchWaitTime, - long remoteBytesRead) { - this.remoteBlocksFetched = remoteBlocksFetched; - this.localBlocksFetched = localBlocksFetched; - this.fetchWaitTime = fetchWaitTime; - this.remoteBytesRead = remoteBytesRead; - } - - public ShuffleReadMetrics(TaskMetrics metrics) { - this(metrics.shuffleReadMetrics().remoteBlocksFetched(), - metrics.shuffleReadMetrics().localBlocksFetched(), - metrics.shuffleReadMetrics().fetchWaitTime(), - metrics.shuffleReadMetrics().remoteBytesRead()); - } - - /** - * Number of blocks fetched in shuffle by tasks (remote or local). - */ - public long getTotalBlocksFetched() { - return remoteBlocksFetched + localBlocksFetched; - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java b/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java deleted file mode 100644 index 64a4b860420..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/metrics/ShuffleWriteMetrics.java +++ /dev/null @@ -1,54 +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.hive.spark.client.metrics; - -import java.io.Serializable; - -import org.apache.spark.executor.TaskMetrics; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Metrics pertaining to writing shuffle data. - */ -@InterfaceAudience.Private -public class ShuffleWriteMetrics implements Serializable { - - /** Number of bytes written for the shuffle by tasks. */ - public final long shuffleBytesWritten; - /** Time tasks spent blocking on writes to disk or buffer cache, in nanoseconds. */ - public final long shuffleWriteTime; - - private ShuffleWriteMetrics() { - // For Serialization only. - this(0L, 0L); - } - - public ShuffleWriteMetrics( - long shuffleBytesWritten, - long shuffleWriteTime) { - this.shuffleBytesWritten = shuffleBytesWritten; - this.shuffleWriteTime = shuffleWriteTime; - } - - public ShuffleWriteMetrics(TaskMetrics metrics) { - this(metrics.shuffleWriteMetrics().shuffleBytesWritten(), - metrics.shuffleWriteMetrics().shuffleWriteTime()); - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/KryoMessageCodec.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/KryoMessageCodec.java deleted file mode 100644 index 9e789cf5be8..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/KryoMessageCodec.java +++ /dev/null @@ -1,175 +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.hive.spark.client.rpc; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.List; - -import org.objenesis.strategy.StdInstantiatorStrategy; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.ByteBufferInputStream; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import com.google.common.base.Preconditions; - -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.ByteToMessageCodec; - -/** - * Codec that serializes / deserializes objects using Kryo. Objects are encoded with a 4-byte - * header with the length of the serialized data. - */ -class KryoMessageCodec extends ByteToMessageCodec { - - private static final Logger LOG = LoggerFactory.getLogger(KryoMessageCodec.class); - - // Kryo docs say 0-8 are taken. Strange things happen if you don't set an ID when registering - // classes. - private static final int REG_ID_BASE = 16; - - private final int maxMessageSize; - private final List> messages; - private final ThreadLocal kryos = new ThreadLocal() { - @Override - protected Kryo initialValue() { - Kryo kryo = new Kryo(); - int count = 0; - for (Class klass : messages) { - kryo.register(klass, REG_ID_BASE + count); - count++; - } - kryo.setInstantiatorStrategy(new Kryo.DefaultInstantiatorStrategy(new StdInstantiatorStrategy())); - return kryo; - } - }; - - private volatile EncryptionHandler encryptionHandler; - - public KryoMessageCodec(int maxMessageSize, Class... messages) { - this.maxMessageSize = maxMessageSize; - this.messages = Arrays.asList(messages); - this.encryptionHandler = null; - } - - @Override - protected void decode(ChannelHandlerContext ctx, ByteBuf in, List out) - throws Exception { - if (in.readableBytes() < 4) { - return; - } - - in.markReaderIndex(); - int msgSize = in.readInt(); - checkSize(msgSize); - - if (in.readableBytes() < msgSize) { - // Incomplete message in buffer. - in.resetReaderIndex(); - return; - } - - try { - ByteBuffer nioBuffer = maybeDecrypt(in.nioBuffer(in.readerIndex(), msgSize)); - Input kryoIn = new Input(new ByteBufferInputStream(nioBuffer)); - - Object msg = kryos.get().readClassAndObject(kryoIn); - LOG.debug("Decoded message of type {} ({} bytes)", - msg != null ? msg.getClass().getName() : msg, msgSize); - out.add(msg); - } finally { - in.skipBytes(msgSize); - } - } - - @Override - protected void encode(ChannelHandlerContext ctx, Object msg, ByteBuf buf) - throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - Output kryoOut = new Output(bytes); - kryos.get().writeClassAndObject(kryoOut, msg); - kryoOut.flush(); - - byte[] msgData = maybeEncrypt(bytes.toByteArray()); - LOG.debug("Encoded message of type {} ({} bytes)", msg.getClass().getName(), msgData.length); - checkSize(msgData.length); - - buf.ensureWritable(msgData.length + 4); - buf.writeInt(msgData.length); - buf.writeBytes(msgData); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - if (encryptionHandler != null) { - encryptionHandler.dispose(); - } - super.channelInactive(ctx); - } - - private void checkSize(int msgSize) { - Preconditions.checkArgument(msgSize > 0, "Message size (%s bytes) must be positive.", msgSize); - Preconditions.checkArgument(maxMessageSize <= 0 || msgSize <= maxMessageSize, - "Message (%s bytes) exceeds maximum allowed size (%s bytes).", msgSize, maxMessageSize); - } - - private byte[] maybeEncrypt(byte[] data) throws Exception { - return (encryptionHandler != null) ? encryptionHandler.wrap(data, 0, data.length) : data; - } - - private ByteBuffer maybeDecrypt(ByteBuffer data) throws Exception { - if (encryptionHandler != null) { - byte[] encrypted; - int len = data.limit() - data.position(); - int offset; - if (data.hasArray()) { - encrypted = data.array(); - offset = data.position() + data.arrayOffset(); - data.position(data.limit()); - } else { - encrypted = new byte[len]; - offset = 0; - data.get(encrypted); - } - return ByteBuffer.wrap(encryptionHandler.unwrap(encrypted, offset, len)); - } else { - return data; - } - } - - void setEncryptionHandler(EncryptionHandler handler) { - this.encryptionHandler = handler; - } - - interface EncryptionHandler { - - byte[] wrap(byte[] data, int offset, int len) throws IOException; - - byte[] unwrap(byte[] data, int offset, int len) throws IOException; - - void dispose() throws IOException; - - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/README.md b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/README.md deleted file mode 100644 index eafc3f4b36d..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/README.md +++ /dev/null @@ -1,33 +0,0 @@ -Spark Client RPC -================ - -Basic flow of events: - -- Client side creates an RPC server -- Client side spawns RemoteDriver, which manages the SparkContext, and provides a secret -- Client side sets up a timer to wait for RemoteDriver to connect back -- RemoteDriver connects back to client, SASL handshake ensues -- Connection is established and now there's a session between the client and the driver. - -Features of the RPC layer: - -- All messages serialized via Kryo -- All messages are replied to. It's either an empty "ack" or an actual response - that depends - on the message. -- RPC send API is asynchronous - callers get a future that can be used to wait for the message. -- Currently, no connection retry. If connection goes down, both sides tear down the session. - -Notes: - -- Because serialization is using Kryo, types need explicit empty constructors or things will - fail to deserialize. This can be seen in the way exceptions are propagated - the throwing - side sends just a string stack trace to the remote, because certain fields on exceptions - don't have empty constructors. -- The above is especially important because at the moment there's no way to register custom - serializers in the RPC library. - -Future work: - -- Random initial RPC id + id wrapping. -- SSL / security in general. -- Remove "Serializable" from the API. Not needed with kryo. diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java deleted file mode 100644 index cbbfb1ca645..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/Rpc.java +++ /dev/null @@ -1,504 +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.hive.spark.client.rpc; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.sasl.RealmCallback; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslClient; -import javax.security.sasl.SaslException; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.google.common.collect.Lists; -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.channel.ChannelOption; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; -import io.netty.channel.socket.nio.NioSocketChannel; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.logging.LogLevel; -import io.netty.util.concurrent.EventExecutorGroup; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.ImmediateEventExecutor; -import io.netty.util.concurrent.Promise; -import io.netty.util.concurrent.ScheduledFuture; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * Encapsulates the RPC functionality. Provides higher-level methods to talk to the remote - * endpoint. - */ -@InterfaceAudience.Private -public class Rpc implements Closeable { - - private static final Logger LOG = LoggerFactory.getLogger(Rpc.class); - - static final String SASL_REALM = "rsc"; - static final String SASL_USER = "rsc"; - static final String SASL_PROTOCOL = "rsc"; - static final String SASL_AUTH_CONF = "auth-conf"; - - /** - * Creates an RPC client for a server running on the given remote host and port. - * - * @param config RPC configuration data. - * @param eloop Event loop for managing the connection. - * @param host Host name or IP address to connect to. - * @param port Port where server is listening. - * @param clientId The client ID that identifies the connection. - * @param secret Secret for authenticating the client with the server. - * @param dispatcher Dispatcher used to handle RPC calls. - * @return A future that can be used to monitor the creation of the RPC object. - */ - public static Promise createClient( - Map config, - final NioEventLoopGroup eloop, - String host, - int port, - final String clientId, - final String secret, - final RpcDispatcher dispatcher) throws Exception { - final RpcConfiguration rpcConf = new RpcConfiguration(config); - int connectTimeoutMs = (int) rpcConf.getConnectTimeoutMs(); - - final ChannelFuture cf = new Bootstrap() - .group(eloop) - .handler(new ChannelInboundHandlerAdapter() { }) - .channel(NioSocketChannel.class) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeoutMs) - .connect(host, port); - - final Promise promise = eloop.next().newPromise(); - final AtomicReference rpc = new AtomicReference(); - - // Set up a timeout to undo everything. - final Runnable timeoutTask = new Runnable() { - @Override - public void run() { - promise.setFailure(new TimeoutException("Timed out waiting for RPC server connection.")); - } - }; - final ScheduledFuture timeoutFuture = eloop.schedule(timeoutTask, - connectTimeoutMs, TimeUnit.MILLISECONDS); - - // The channel listener instantiates the Rpc instance when the connection is established, - // and initiates the SASL handshake. - cf.addListener(new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture cf) throws Exception { - if (cf.isSuccess()) { - SaslClientHandler saslHandler = new SaslClientHandler(rpcConf, clientId, promise, - timeoutFuture, secret, dispatcher); - Rpc rpc = createRpc(rpcConf, saslHandler, (SocketChannel) cf.channel(), eloop); - saslHandler.rpc = rpc; - saslHandler.sendHello(cf.channel()); - } else { - promise.setFailure(cf.cause()); - } - } - }); - - // Handle cancellation of the promise. - promise.addListener(new GenericFutureListener>() { - @Override - public void operationComplete(Promise p) { - if (p.isCancelled()) { - cf.cancel(true); - } - } - }); - - return promise; - } - - static Rpc createServer(SaslHandler saslHandler, RpcConfiguration config, SocketChannel channel, - EventExecutorGroup egroup) throws IOException { - return createRpc(config, saslHandler, channel, egroup); - } - - private static Rpc createRpc(RpcConfiguration config, - SaslHandler saslHandler, - SocketChannel client, - EventExecutorGroup egroup) - throws IOException { - LogLevel logLevel = LogLevel.TRACE; - if (config.getRpcChannelLogLevel() != null) { - try { - logLevel = LogLevel.valueOf(config.getRpcChannelLogLevel()); - } catch (Exception e) { - LOG.warn("Invalid log level {}, reverting to default.", config.getRpcChannelLogLevel()); - } - } - - boolean logEnabled = false; - switch (logLevel) { - case DEBUG: - logEnabled = LOG.isDebugEnabled(); - break; - case ERROR: - logEnabled = LOG.isErrorEnabled(); - break; - case INFO: - logEnabled = LOG.isInfoEnabled(); - break; - case TRACE: - logEnabled = LOG.isTraceEnabled(); - break; - case WARN: - logEnabled = LOG.isWarnEnabled(); - break; - } - - if (logEnabled) { - client.pipeline().addLast("logger", new LoggingHandler(Rpc.class, logLevel)); - } - - KryoMessageCodec kryo = new KryoMessageCodec(config.getMaxMessageSize(), - MessageHeader.class, NullMessage.class, SaslMessage.class); - saslHandler.setKryoMessageCodec(kryo); - client.pipeline() - .addLast("codec", kryo) - .addLast("sasl", saslHandler); - return new Rpc(config, client, egroup); - } - - @VisibleForTesting - static Rpc createEmbedded(RpcDispatcher dispatcher) { - EmbeddedChannel c = new EmbeddedChannel( - new LoggingHandler(Rpc.class), - new KryoMessageCodec(0, MessageHeader.class, NullMessage.class), - dispatcher); - Rpc rpc = new Rpc(new RpcConfiguration(Collections.emptyMap()), - c, ImmediateEventExecutor.INSTANCE); - rpc.dispatcher = dispatcher; - return rpc; - } - - private final RpcConfiguration config; - private final AtomicBoolean rpcClosed; - private final AtomicLong rpcId; - private final Channel channel; - private final Collection listeners; - private final EventExecutorGroup egroup; - private volatile RpcDispatcher dispatcher; - - private Rpc(RpcConfiguration config, Channel channel, EventExecutorGroup egroup) { - Preconditions.checkArgument(channel != null); - Preconditions.checkArgument(egroup != null); - this.config = config; - this.channel = channel; - this.dispatcher = null; - this.egroup = egroup; - this.listeners = Lists.newLinkedList(); - this.rpcClosed = new AtomicBoolean(); - this.rpcId = new AtomicLong(); - - // Note: this does not work for embedded channels. - channel.pipeline().addLast("monitor", new ChannelInboundHandlerAdapter() { - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - close(); - super.channelInactive(ctx); - } - }); - } - - public void addListener(Listener l) { - synchronized (listeners) { - listeners.add(l); - } - } - - /** - * Send an RPC call to the remote endpoint and returns a future that can be used to monitor the - * operation. - */ - public Future call(Object msg) { - return call(msg, Void.class); - } - - public boolean isActive() { - return channel.isActive(); - } - - /** - * Send an RPC call to the remote endpoint and returns a future that can be used to monitor the - * operation. - * - * @param msg RPC call to send. - * @param retType Type of expected reply. - * @return A future used to monitor the operation. - */ - public Future call(final Object msg, Class retType) { - Preconditions.checkArgument(msg != null); - Preconditions.checkState(channel.isActive(), "RPC channel is closed."); - try { - final long id = rpcId.getAndIncrement(); - final Promise promise = createPromise(); - final ChannelFutureListener listener = new ChannelFutureListener() { - @Override - public void operationComplete(ChannelFuture cf) { - if (!cf.isSuccess() && !promise.isDone()) { - LOG.warn("Failed to send RPC, closing connection.", cf.cause()); - promise.setFailure(cf.cause()); - dispatcher.discardRpc(id); - close(); - } - } - }; - - dispatcher.registerRpc(id, promise, msg.getClass().getName()); - channel.eventLoop().submit(new Runnable() { - @Override - public void run() { - channel.write(new MessageHeader(id, Rpc.MessageType.CALL)).addListener(listener); - channel.writeAndFlush(msg).addListener(listener); - } - }); - return promise; - } catch (Exception e) { - throw Throwables.propagate(e); - } - } - - /** - * Creates a promise backed by this RPC's event loop. - */ - public Promise createPromise() { - return egroup.next().newPromise(); - } - - @VisibleForTesting - Channel getChannel() { - return channel; - } - - void setDispatcher(RpcDispatcher dispatcher) { - Preconditions.checkNotNull(dispatcher); - Preconditions.checkState(this.dispatcher == null); - this.dispatcher = dispatcher; - channel.pipeline().addLast("dispatcher", dispatcher); - } - - @Override - public void close() { - if (!rpcClosed.compareAndSet(false, true)) { - return; - } - try { - channel.close().sync(); - } catch (InterruptedException ie) { - Thread.interrupted(); - } finally { - synchronized (listeners) { - for (Listener l : listeners) { - try { - l.rpcClosed(this); - } catch (Exception e) { - LOG.warn("Error caught in Rpc.Listener invocation.", e); - } - } - } - } - } - - public interface Listener { - - void rpcClosed(Rpc rpc); - - } - - static enum MessageType { - CALL, - REPLY, - ERROR; - } - - static class MessageHeader { - final long id; - final MessageType type; - - MessageHeader() { - this(-1, null); - } - - MessageHeader(long id, MessageType type) { - this.id = id; - this.type = type; - } - - } - - static class NullMessage { - - } - - static class SaslMessage { - final String clientId; - final byte[] payload; - - SaslMessage() { - this(null, null); - } - - SaslMessage(byte[] payload) { - this(null, payload); - } - - SaslMessage(String clientId, byte[] payload) { - this.clientId = clientId; - this.payload = payload; - } - - } - - private static class SaslClientHandler extends SaslHandler implements CallbackHandler { - - private final SaslClient client; - private final String clientId; - private final String secret; - private final RpcDispatcher dispatcher; - private Promise promise; - private ScheduledFuture timeout; - - // Can't be set in constructor due to circular dependency. - private Rpc rpc; - - SaslClientHandler( - RpcConfiguration config, - String clientId, - Promise promise, - ScheduledFuture timeout, - String secret, - RpcDispatcher dispatcher) - throws IOException { - super(config); - this.clientId = clientId; - this.promise = promise; - this.timeout = timeout; - this.secret = secret; - this.dispatcher = dispatcher; - this.client = Sasl.createSaslClient(new String[] { config.getSaslMechanism() }, - null, SASL_PROTOCOL, SASL_REALM, config.getSaslOptions(), this); - } - - @Override - protected boolean isComplete() { - return client.isComplete(); - } - - @Override - protected String getNegotiatedProperty(String name) { - return (String) client.getNegotiatedProperty(name); - } - - @Override - protected SaslMessage update(SaslMessage challenge) throws IOException { - byte[] response = client.evaluateChallenge(challenge.payload); - return response != null ? new SaslMessage(response) : null; - } - - @Override - public byte[] wrap(byte[] data, int offset, int len) throws IOException { - return client.wrap(data, offset, len); - } - - @Override - public byte[] unwrap(byte[] data, int offset, int len) throws IOException { - return client.unwrap(data, offset, len); - } - - @Override - public void dispose() throws IOException { - if (!client.isComplete()) { - onError(new SaslException("Client closed before SASL negotiation finished.")); - } - client.dispose(); - } - - @Override - protected void onComplete() throws Exception { - timeout.cancel(true); - rpc.setDispatcher(dispatcher); - promise.setSuccess(rpc); - timeout = null; - promise = null; - } - - @Override - protected void onError(Throwable error) { - timeout.cancel(true); - if (!promise.isDone()) { - promise.setFailure(error); - } - } - - @Override - public void handle(Callback[] callbacks) { - for (Callback cb : callbacks) { - if (cb instanceof NameCallback) { - ((NameCallback)cb).setName(clientId); - } else if (cb instanceof PasswordCallback) { - ((PasswordCallback)cb).setPassword(secret.toCharArray()); - } else if (cb instanceof RealmCallback) { - RealmCallback rb = (RealmCallback) cb; - rb.setText(rb.getDefaultText()); - } - } - } - - void sendHello(Channel c) throws Exception { - byte[] hello = client.hasInitialResponse() ? - client.evaluateChallenge(new byte[0]) : new byte[0]; - c.writeAndFlush(new SaslMessage(clientId, hello)).addListener(future -> { - if (!future.isSuccess()) { - LOG.error("Failed to send hello to server", future.cause()); - onError(future.cause()); - } - }); - } - - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java deleted file mode 100644 index a535b8d3337..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcConfiguration.java +++ /dev/null @@ -1,211 +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.hive.spark.client.rpc; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import javax.security.sasl.Sasl; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.hive.common.ServerUtils; -import org.apache.hadoop.hive.common.classification.InterfaceAudience; -import org.apache.hadoop.hive.conf.HiveConf; - -/** - * Definitions of configuration keys and default values for the RPC layer. - */ -@InterfaceAudience.Private -public final class RpcConfiguration { - - private static final Logger LOG = LoggerFactory.getLogger(RpcConfiguration.class); - - public static final ImmutableSet HIVE_SPARK_RSC_CONFIGS = ImmutableSet.of( - HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname, - HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname, - HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, - HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.varname, - HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.varname, - HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.varname, - HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname - ); - public static final ImmutableSet HIVE_SPARK_TIME_CONFIGS = ImmutableSet.of( - HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname, - HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname - ); - - /** Prefix for other SASL options. */ - public static final String RPC_SASL_OPT_PREFIX = "hive.spark.client.rpc.sasl."; - - private final Map config; - - private static final HiveConf DEFAULT_CONF = new HiveConf(); - - public RpcConfiguration(Map config) { - // make sure we don't modify the config in RpcConfiguration - this.config = Collections.unmodifiableMap(config); - } - - long getConnectTimeoutMs() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT.varname); - return value != null ? Long.parseLong(value) : DEFAULT_CONF.getTimeVar( - HiveConf.ConfVars.SPARK_RPC_CLIENT_CONNECT_TIMEOUT, TimeUnit.MILLISECONDS); - } - - int getMaxMessageSize() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.varname); - return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_MAX_MESSAGE_SIZE.defaultIntVal; - } - - long getServerConnectTimeoutMs() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT.varname); - return value != null ? Long.parseLong(value) : DEFAULT_CONF.getTimeVar( - HiveConf.ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT, TimeUnit.MILLISECONDS); - } - - int getSecretBits() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.varname); - return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_SECRET_RANDOM_BITS.defaultIntVal; - } - - /** - * Here we assume that the remote driver will connect back to HS2 using the same network interface - * as if it were just a HS2 client. If this isn't true, we can have a separate configuration for that. - * For now, I think we are okay. - * @return server host name in the network - * @throws IOException - */ - String getServerAddress() throws IOException { - String hiveHost = config.get(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname); - if(StringUtils.isEmpty(hiveHost)) { - hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); - if (hiveHost == null) { - hiveHost = config.get(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname); - } - } - return ServerUtils.getHostAddress(hiveHost).getHostName(); - } - - /** - * Parses the port string like 49152-49222,49228 into the port list. A default 0 - * is added for the empty port string. - * @return a list of configured ports. - * @exception IOException is thrown if the property is not configured properly - */ - List getServerPorts() throws IOException { - String errMsg = "Incorrect RPC server port configuration for HiveServer2"; - String portString = config.get(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname); - ArrayList ports = new ArrayList(); - try { - if(!StringUtils.isEmpty(portString)) { - for (String portRange : portString.split(",")) { - String[] range = portRange.split("-"); - if (range.length == 0 || range.length > 2 - || (range.length == 2 && Integer.valueOf(range[0]) > Integer.valueOf(range[1]))) { - throw new IOException(errMsg); - } - if (range.length == 1) { - ports.add(Integer.valueOf(range[0])); - } else { - for (int i = Integer.valueOf(range[0]); i <= Integer.valueOf(range[1]); i++) { - ports.add(i); - } - } - } - } else { - ports.add(0); - } - - return ports; - } catch(NumberFormatException e) { - throw new IOException(errMsg); - } - } - - String getRpcChannelLogLevel() { - return config.get(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname); - } - - public int getRpcThreadCount() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.varname); - return value != null ? Integer.parseInt(value) : HiveConf.ConfVars.SPARK_RPC_MAX_THREADS.defaultIntVal; - } - - /** - * Utility method for a given RpcConfiguration key, to convert value to millisecond if it is a time value, - * and return as string in either case. - * @param conf hive configuration - * @param key Rpc configuration to lookup (hive.spark.*) - * @return string form of the value - */ - public static String getValue(HiveConf conf, String key) { - if (HIVE_SPARK_TIME_CONFIGS.contains(key)) { - HiveConf.ConfVars confVar = HiveConf.getConfVars(key); - return String.valueOf(conf.getTimeVar(confVar, TimeUnit.MILLISECONDS)); - } else { - return conf.get(key); - } - } - - String getSaslMechanism() { - String value = config.get(HiveConf.ConfVars.SPARK_RPC_SASL_MECHANISM.varname); - return value != null ? value : HiveConf.ConfVars. SPARK_RPC_SASL_MECHANISM.defaultStrVal; - } - - /** - * SASL options are namespaced under "hive.spark.client.rpc.sasl.*"; each option is the - * lower-case version of the constant in the "javax.security.sasl.Sasl" class (e.g. "strength" - * for cipher strength). - */ - Map getSaslOptions() { - Map opts = new HashMap(); - Map saslOpts = ImmutableMap.builder() - .put(Sasl.CREDENTIALS, "credentials") - .put(Sasl.MAX_BUFFER, "max_buffer") - .put(Sasl.POLICY_FORWARD_SECRECY, "policy_forward_secrecy") - .put(Sasl.POLICY_NOACTIVE, "policy_noactive") - .put(Sasl.POLICY_NOANONYMOUS, "policy_noanonymous") - .put(Sasl.POLICY_NODICTIONARY, "policy_nodictionary") - .put(Sasl.POLICY_NOPLAINTEXT, "policy_noplaintext") - .put(Sasl.POLICY_PASS_CREDENTIALS, "policy_pass_credentials") - .put(Sasl.QOP, "qop") - .put(Sasl.RAW_SEND_SIZE, "raw_send_size") - .put(Sasl.REUSE, "reuse") - .put(Sasl.SERVER_AUTH, "server_auth") - .put(Sasl.STRENGTH, "strength") - .build(); - for (Map.Entry e : saslOpts.entrySet()) { - String value = config.get(RPC_SASL_OPT_PREFIX + e.getValue()); - if (value != null) { - opts.put(e.getKey(), value); - } - } - return opts; - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java deleted file mode 100644 index 00f5a17412e..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcDispatcher.java +++ /dev/null @@ -1,200 +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.hive.spark.client.rpc; - -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; - -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.util.concurrent.Promise; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * An implementation of ChannelInboundHandler that dispatches incoming messages to an instance - * method based on the method signature. - *

- * A handler's signature must be of the form: - *

- *
protected void handle(ChannelHandlerContext, MessageType)
- *

- * Where "MessageType" must match exactly the type of the message to handle. Polymorphism is not - * supported. Handlers can return a value, which becomes the RPC reply; if a null is returned, then - * a reply is still sent, with an empty payload. - *

- */ -@InterfaceAudience.Private -public abstract class RpcDispatcher extends SimpleChannelInboundHandler { - - private static final Logger LOG = LoggerFactory.getLogger(RpcDispatcher.class); - - private final Map, Method> handlers = Maps.newConcurrentMap(); - private final Collection rpcs = new ConcurrentLinkedQueue(); - - private volatile Rpc.MessageHeader lastHeader; - - /** Override this to add a name to the dispatcher, for debugging purposes. */ - protected String name() { - return getClass().getSimpleName(); - } - - @Override - protected final void channelRead0(ChannelHandlerContext ctx, Object msg) throws Exception { - if (lastHeader == null) { - if (!(msg instanceof Rpc.MessageHeader)) { - LOG.warn("[{}] Expected RPC header, got {} instead.", name(), - msg != null ? msg.getClass().getName() : null); - throw new IllegalArgumentException(); - } - lastHeader = (Rpc.MessageHeader) msg; - } else { - LOG.debug("[{}] Received RPC message: type={} id={} payload={}", name(), - lastHeader.type, lastHeader.id, msg != null ? msg.getClass().getName() : null); - try { - switch (lastHeader.type) { - case CALL: - handleCall(ctx, msg); - break; - case REPLY: - handleReply(ctx, msg, findRpc(lastHeader.id)); - break; - case ERROR: - handleError(ctx, msg, findRpc(lastHeader.id)); - break; - default: - throw new IllegalArgumentException("Unknown RPC message type: " + lastHeader.type); - } - } finally { - lastHeader = null; - } - } - } - - private OutstandingRpc findRpc(long id) { - for (Iterator it = rpcs.iterator(); it.hasNext();) { - OutstandingRpc rpc = it.next(); - if (rpc.id == id) { - it.remove(); - return rpc; - } - } - throw new IllegalArgumentException(String.format( - "Received RPC reply for unknown RPC (%d).", id)); - } - - private void handleCall(ChannelHandlerContext ctx, Object msg) throws Exception { - Method handler = handlers.get(msg.getClass()); - if (handler == null) { - handler = getClass().getDeclaredMethod("handle", ChannelHandlerContext.class, - msg.getClass()); - handler.setAccessible(true); - handlers.put(msg.getClass(), handler); - } - - Rpc.MessageType replyType; - Object replyPayload; - try { - replyPayload = handler.invoke(this, ctx, msg); - if (replyPayload == null) { - replyPayload = new Rpc.NullMessage(); - } - replyType = Rpc.MessageType.REPLY; - } catch (InvocationTargetException ite) { - LOG.debug(String.format("[%s] Error in RPC handler.", name()), ite.getCause()); - replyPayload = Throwables.getStackTraceAsString(ite.getCause()); - replyType = Rpc.MessageType.ERROR; - } - ctx.channel().write(new Rpc.MessageHeader(lastHeader.id, replyType)); - ctx.channel().writeAndFlush(replyPayload); - } - - private void handleReply(ChannelHandlerContext ctx, Object msg, OutstandingRpc rpc) - throws Exception { - rpc.future.setSuccess(msg instanceof Rpc.NullMessage ? null : msg); - } - - private void handleError(ChannelHandlerContext ctx, Object msg, OutstandingRpc rpc) - throws Exception { - if (msg instanceof String) { - LOG.warn("Received error message:{}.", msg); - rpc.future.setFailure(new RpcException((String) msg)); - } else { - String error = String.format("Received error with unexpected payload (%s).", - msg != null ? msg.getClass().getName() : null); - LOG.warn(String.format("[%s] %s", name(), error)); - rpc.future.setFailure(new IllegalArgumentException(error)); - ctx.close(); - } - } - - @Override - public final void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - LOG.error(String.format("[%s] Closing channel due to exception in pipeline.", name()), cause); - - if (lastHeader != null) { - // There's an RPC waiting for a reply. Exception was most probably caught while processing - // the RPC, so send an error. - ctx.channel().write(new Rpc.MessageHeader(lastHeader.id, Rpc.MessageType.ERROR)); - ctx.channel().writeAndFlush(Throwables.getStackTraceAsString(cause)); - lastHeader = null; - } - - ctx.close(); - } - - @Override - public final void channelInactive(ChannelHandlerContext ctx) throws Exception { - if (rpcs.size() > 0) { - LOG.warn("[{}] Closing RPC channel with {} outstanding RPCs.", name(), rpcs.size()); - for (OutstandingRpc rpc : rpcs) { - rpc.future.cancel(true); - } - } - super.channelInactive(ctx); - } - - void registerRpc(long id, Promise promise, String type) { - LOG.debug("[{}] Registered outstanding rpc {} ({}).", name(), id, type); - rpcs.add(new OutstandingRpc(id, promise)); - } - - void discardRpc(long id) { - LOG.debug("[{}] Discarding failed RPC {}.", name(), id); - findRpc(id); - } - - private static class OutstandingRpc { - final long id; - final Promise future; - - OutstandingRpc(long id, Promise future) { - this.id = id; - this.future = future; - } - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcException.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcException.java deleted file mode 100644 index e926e392e96..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcException.java +++ /dev/null @@ -1,29 +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.hive.spark.client.rpc; - -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -@InterfaceAudience.Private -public class RpcException extends RuntimeException { - - RpcException(String remoteStackTrace) { - super(remoteStackTrace); - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java deleted file mode 100644 index 6c6ab74ce7a..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/RpcServer.java +++ /dev/null @@ -1,367 +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.hive.spark.client.rpc; - -import java.io.Closeable; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.security.SecureRandom; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.TimeUnit; - -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.RealmCallback; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslException; -import javax.security.sasl.SaslServer; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; -import com.google.common.collect.Maps; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelInitializer; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.SocketChannel; -import io.netty.channel.socket.nio.NioServerSocketChannel; -import io.netty.util.concurrent.Future; -import io.netty.util.concurrent.GenericFutureListener; -import io.netty.util.concurrent.Promise; -import io.netty.util.concurrent.ScheduledFuture; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.hadoop.hive.common.classification.InterfaceAudience; - -/** - * An RPC server. The server matches remote clients based on a secret that is generated on - * the server - the secret needs to be given to the client through some other mechanism for - * this to work. - */ -@InterfaceAudience.Private -public class RpcServer implements Closeable { - - private static final Logger LOG = LoggerFactory.getLogger(RpcServer.class); - private static final SecureRandom RND = new SecureRandom(); - - private final String address; - private final Channel channel; - private final EventLoopGroup group; - private final int port; - private final ConcurrentMap pendingClients; - private final RpcConfiguration config; - - public RpcServer(Map mapConf) throws IOException, InterruptedException { - this.config = new RpcConfiguration(mapConf); - this.group = new NioEventLoopGroup( - this.config.getRpcThreadCount(), - new ThreadFactoryBuilder() - .setNameFormat("RPC-Handler-%d") - .setDaemon(true) - .build()); - ServerBootstrap serverBootstrap = new ServerBootstrap() - .group(group) - .channel(NioServerSocketChannel.class) - .childHandler(new ChannelInitializer() { - @Override - public void initChannel(SocketChannel ch) throws Exception { - SaslServerHandler saslHandler = new SaslServerHandler(config); - final Rpc newRpc = Rpc.createServer(saslHandler, config, ch, group); - saslHandler.rpc = newRpc; - - Runnable cancelTask = new Runnable() { - @Override - public void run() { - LOG.warn("Timed out waiting for hello from client."); - newRpc.close(); - } - }; - saslHandler.cancelTask = group.schedule(cancelTask, - RpcServer.this.config.getConnectTimeoutMs(), - TimeUnit.MILLISECONDS); - - } - }) - .option(ChannelOption.SO_REUSEADDR, true) - .childOption(ChannelOption.SO_KEEPALIVE, true); - - this.channel = bindServerPort(serverBootstrap).channel(); - this.port = ((InetSocketAddress) channel.localAddress()).getPort(); - this.pendingClients = Maps.newConcurrentMap(); - this.address = this.config.getServerAddress(); - } - - /** - * Retry the list of configured ports until one is found - * @param serverBootstrap - * @return - * @throws InterruptedException - * @throws IOException - */ - private ChannelFuture bindServerPort(ServerBootstrap serverBootstrap) - throws InterruptedException, IOException { - List ports = config.getServerPorts(); - if (ports.contains(0)) { - return serverBootstrap.bind(0).sync(); - } else { - Random rand = new Random(); - while(!ports.isEmpty()) { - int index = rand.nextInt(ports.size()); - int port = ports.get(index); - ports.remove(index); - try { - return serverBootstrap.bind(port).sync(); - } catch(Exception e) { - // Retry the next port - } - } - throw new IOException("No available ports from configured RPC Server ports for HiveServer2"); - } - } - - /** - * Tells the RPC server to expect a connection from a new client. - * - * @param clientId An identifier for the client. Must be unique. - * @param secret The secret the client will send to the server to identify itself. - * @param serverDispatcher The dispatcher to use when setting up the RPC instance. - * @return A future that can be used to wait for the client connection, which also provides the - * secret needed for the client to connect. - */ - public Future registerClient(final String clientId, String secret, - RpcDispatcher serverDispatcher) { - return registerClient(clientId, secret, serverDispatcher, config.getServerConnectTimeoutMs()); - } - - @VisibleForTesting - Future registerClient(final String clientId, String secret, - RpcDispatcher serverDispatcher, long clientTimeoutMs) { - final Promise promise = group.next().newPromise(); - - Runnable timeout = new Runnable() { - @Override - public void run() { - promise.setFailure(new TimeoutException("Timed out waiting for client connection.")); - } - }; - ScheduledFuture timeoutFuture = group.schedule(timeout, - clientTimeoutMs, - TimeUnit.MILLISECONDS); - final ClientInfo client = new ClientInfo(clientId, promise, secret, serverDispatcher, - timeoutFuture); - if (pendingClients.putIfAbsent(clientId, client) != null) { - throw new IllegalStateException( - String.format("Client '%s' already registered.", clientId)); - } - - promise.addListener(new GenericFutureListener>() { - @Override - public void operationComplete(Promise p) { - if (!p.isSuccess()) { - pendingClients.remove(clientId); - } - } - }); - - return promise; - } - - /** - * Tells the RPC server to cancel the connection from an existing pending client - * @param clientId The identifier for the client - * @param msg The error message about why the connection should be canceled - */ - public void cancelClient(final String clientId, final String msg) { - final ClientInfo cinfo = pendingClients.remove(clientId); - if (cinfo == null) { - // Nothing to be done here. - return; - } - cinfo.timeoutFuture.cancel(true); - if (!cinfo.promise.isDone()) { - cinfo.promise.setFailure(new RuntimeException( - String.format("Cancel client '%s'. Error: " + msg, clientId))); - } - } - - /** - * Creates a secret for identifying a client connection. - */ - public String createSecret() { - byte[] secret = new byte[config.getSecretBits() / 8]; - RND.nextBytes(secret); - - StringBuilder sb = new StringBuilder(); - for (byte b : secret) { - if (b < 10) { - sb.append("0"); - } - sb.append(Integer.toHexString(b)); - } - return sb.toString(); - } - - public String getAddress() { - return address; - } - - public int getPort() { - return port; - } - - @Override - public void close() { - try { - channel.close(); - for (ClientInfo client : pendingClients.values()) { - client.promise.cancel(true); - } - pendingClients.clear(); - } finally { - group.shutdownGracefully(); - } - } - - private class SaslServerHandler extends SaslHandler implements CallbackHandler { - - private final SaslServer server; - private Rpc rpc; - private ScheduledFuture cancelTask; - private String clientId; - private ClientInfo client; - - SaslServerHandler(RpcConfiguration config) throws IOException { - super(config); - this.server = Sasl.createSaslServer(config.getSaslMechanism(), Rpc.SASL_PROTOCOL, - Rpc.SASL_REALM, config.getSaslOptions(), this); - } - - @Override - protected boolean isComplete() { - return server.isComplete(); - } - - @Override - protected String getNegotiatedProperty(String name) { - return (String) server.getNegotiatedProperty(name); - } - - @Override - protected Rpc.SaslMessage update(Rpc.SaslMessage challenge) throws IOException { - if (clientId == null) { - Preconditions.checkArgument(challenge.clientId != null, - "Missing client ID in SASL handshake."); - clientId = challenge.clientId; - client = pendingClients.get(clientId); - Preconditions.checkArgument(client != null, - "Unexpected client ID '%s' in SASL handshake.", clientId); - } - - return new Rpc.SaslMessage(server.evaluateResponse(challenge.payload)); - } - - @Override - public byte[] wrap(byte[] data, int offset, int len) throws IOException { - return server.wrap(data, offset, len); - } - - @Override - public byte[] unwrap(byte[] data, int offset, int len) throws IOException { - return server.unwrap(data, offset, len); - } - - @Override - public void dispose() throws IOException { - if (!server.isComplete()) { - onError(new SaslException("Server closed before SASL negotiation finished.")); - } - server.dispose(); - } - - @Override - protected void onComplete() throws Exception { - cancelTask.cancel(true); - client.timeoutFuture.cancel(true); - rpc.setDispatcher(client.dispatcher); - client.promise.setSuccess(rpc); - pendingClients.remove(client.id); - } - - @Override - protected void onError(Throwable error) { - cancelTask.cancel(true); - if (client != null) { - client.timeoutFuture.cancel(true); - if (!client.promise.isDone()) { - client.promise.setFailure(error); - } - } - } - - @Override - public void handle(Callback[] callbacks) { - Preconditions.checkState(client != null, "Handshake not initialized yet."); - for (Callback cb : callbacks) { - if (cb instanceof NameCallback) { - ((NameCallback)cb).setName(clientId); - } else if (cb instanceof PasswordCallback) { - ((PasswordCallback)cb).setPassword(client.secret.toCharArray()); - } else if (cb instanceof AuthorizeCallback) { - ((AuthorizeCallback) cb).setAuthorized(true); - } else if (cb instanceof RealmCallback) { - RealmCallback rb = (RealmCallback) cb; - rb.setText(rb.getDefaultText()); - } - } - } - - } - - private static class ClientInfo { - - final String id; - final Promise promise; - final String secret; - final RpcDispatcher dispatcher; - final ScheduledFuture timeoutFuture; - - private ClientInfo(String id, Promise promise, String secret, RpcDispatcher dispatcher, - ScheduledFuture timeoutFuture) { - this.id = id; - this.promise = promise; - this.secret = secret; - this.dispatcher = dispatcher; - this.timeoutFuture = timeoutFuture; - } - - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java b/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java deleted file mode 100644 index 8a42773668c..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/client/rpc/SaslHandler.java +++ /dev/null @@ -1,114 +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.hive.spark.client.rpc; - -import java.io.IOException; -import javax.security.sasl.Sasl; -import javax.security.sasl.SaslException; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Abstract SASL handler. Abstracts the auth protocol handling and encryption, if it's enabled. - * Needs subclasses to provide access to the actual underlying SASL implementation (client or - * server). - */ -abstract class SaslHandler extends SimpleChannelInboundHandler - implements KryoMessageCodec.EncryptionHandler { - - // LOG is not static to make debugging easier (being able to identify which sub-class - // generated the log message). - protected final Logger LOG; - private final boolean requiresEncryption; - private KryoMessageCodec kryo; - private boolean hasAuthResponse = false; - - protected SaslHandler(RpcConfiguration config) { - this.requiresEncryption = Rpc.SASL_AUTH_CONF.equals(config.getSaslOptions().get(Sasl.QOP)); - this.LOG = LoggerFactory.getLogger(getClass()); - } - - // Use a separate method to make it easier to create a SaslHandler without having to - // plumb the KryoMessageCodec instance through the constructors. - void setKryoMessageCodec(KryoMessageCodec kryo) { - this.kryo = kryo; - } - - @Override - protected final void channelRead0(ChannelHandlerContext ctx, Rpc.SaslMessage msg) - throws Exception { - LOG.debug("Handling SASL challenge message..."); - Rpc.SaslMessage response = update(msg); - if (response != null) { - LOG.debug("Sending SASL challenge response..."); - hasAuthResponse = true; - ctx.channel().writeAndFlush(response).sync(); - } - - if (!isComplete()) { - return; - } - - // If negotiation is complete, remove this handler from the pipeline, and register it with - // the Kryo instance to handle encryption if needed. - ctx.channel().pipeline().remove(this); - String qop = getNegotiatedProperty(Sasl.QOP); - LOG.debug("SASL negotiation finished with QOP {}.", qop); - if (Rpc.SASL_AUTH_CONF.equals(qop)) { - LOG.info("SASL confidentiality enabled."); - kryo.setEncryptionHandler(this); - } else { - if (requiresEncryption) { - throw new SaslException("Encryption required, but SASL negotiation did not set it up."); - } - dispose(); - } - - onComplete(); - } - - @Override - public void channelInactive(ChannelHandlerContext ctx) throws Exception { - dispose(); - super.channelInactive(ctx); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - if (!isComplete()) { - LOG.info("Exception in SASL negotiation.", cause); - onError(cause); - ctx.close(); - } - ctx.fireExceptionCaught(cause); - } - - protected abstract boolean isComplete(); - - protected abstract String getNegotiatedProperty(String name); - - protected abstract Rpc.SaslMessage update(Rpc.SaslMessage challenge) throws IOException; - - protected abstract void onComplete() throws Exception; - - protected abstract void onError(Throwable t); - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounter.java b/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounter.java deleted file mode 100644 index d0eb1fa446a..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounter.java +++ /dev/null @@ -1,109 +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.hive.spark.counter; - -import java.io.Serializable; - -import org.apache.spark.Accumulator; -import org.apache.spark.AccumulatorParam; -import org.apache.spark.api.java.JavaSparkContext; - -public class SparkCounter implements Serializable { - - private String name; - private String displayName; - private Accumulator accumulator; - - // Values of accumulators can only be read on the SparkContext side. This field is used when - // creating a snapshot to be sent to the RSC client. - private long accumValue; - - public SparkCounter() { - // For serialization. - } - - private SparkCounter( - String name, - String displayName, - long value) { - this.name = name; - this.displayName = displayName; - this.accumValue = value; - } - - public SparkCounter( - String name, - String displayName, - String groupName, - long initValue, - JavaSparkContext sparkContext) { - - this.name = name; - this.displayName = displayName; - LongAccumulatorParam longParam = new LongAccumulatorParam(); - String accumulatorName = groupName + "_" + name; - this.accumulator = sparkContext.accumulator(initValue, accumulatorName, longParam); - } - - public long getValue() { - if (accumulator != null) { - return accumulator.value(); - } else { - return accumValue; - } - } - - public void increment(long incr) { - accumulator.add(incr); - } - - public String getName() { - return name; - } - - public String getDisplayName() { - return displayName; - } - - public void setDisplayName(String displayName) { - this.displayName = displayName; - } - - SparkCounter snapshot() { - return new SparkCounter(name, displayName, accumulator.value()); - } - - class LongAccumulatorParam implements AccumulatorParam { - - @Override - public Long addAccumulator(Long t1, Long t2) { - return t1 + t2; - } - - @Override - public Long addInPlace(Long r1, Long r2) { - return r1 + r2; - } - - @Override - public Long zero(Long initialValue) { - return 0L; - } - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounterGroup.java b/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounterGroup.java deleted file mode 100644 index ff24f9ba3d1..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounterGroup.java +++ /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.hive.spark.counter; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; - -import org.apache.spark.api.java.JavaSparkContext; - -/** - * We use group to fold all the same kind of counters. - */ -public class SparkCounterGroup implements Serializable { - private static final long serialVersionUID = 1L; - private String groupName; - private String groupDisplayName; - private Map sparkCounters; - - private transient JavaSparkContext javaSparkContext; - - private SparkCounterGroup() { - // For serialization. - } - - public SparkCounterGroup( - String groupName, - String groupDisplayName, - JavaSparkContext javaSparkContext) { - this.groupName = groupName; - this.groupDisplayName = groupDisplayName; - this.javaSparkContext = javaSparkContext; - this.sparkCounters = new HashMap(); - } - - public void createCounter(String name, long initValue) { - SparkCounter counter = new SparkCounter(name, name, groupName, initValue, javaSparkContext); - sparkCounters.put(name, counter); - } - - public SparkCounter getCounter(String name) { - return sparkCounters.get(name); - } - - public String getGroupName() { - return groupName; - } - - public String getGroupDisplayName() { - return groupDisplayName; - } - - public void setGroupDisplayName(String groupDisplayName) { - this.groupDisplayName = groupDisplayName; - } - - public Map getSparkCounters() { - return sparkCounters; - } - - SparkCounterGroup snapshot() { - SparkCounterGroup snapshot = new SparkCounterGroup(getGroupName(), getGroupDisplayName(), null); - for (SparkCounter counter : sparkCounters.values()) { - SparkCounter counterSnapshot = counter.snapshot(); - snapshot.sparkCounters.put(counterSnapshot.getName(), counterSnapshot); - } - return snapshot; - } - -} diff --git a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java b/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java deleted file mode 100644 index fc6c10e99c7..00000000000 --- a/spark-client/src/main/java/org/apache/hive/spark/counter/SparkCounters.java +++ /dev/null @@ -1,159 +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.hive.spark.counter; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.apache.spark.api.java.JavaSparkContext; - -/** - * SparkCounters is used to collect Hive operator metric through Spark accumulator. There are few - * limitation of Spark accumulator, like: - * 1. accumulator should be created at Spark context side. - * 2. Spark tasks can only increment metric count. - * 3. User can only get accumulator value at Spark context side. - * These Spark Counter API is designed to fit into Hive requirement, while with several access - * restriction due to Spark accumulator previous mentioned: - * 1. Counter should be created on driver side if it would be accessed in task. - * 2. increment could only be invoked task side. - * 3. Hive could only get Counter value at driver side. - */ -public class SparkCounters implements Serializable { - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(SparkCounters.class); - - private Map sparkCounterGroups; - - private final transient JavaSparkContext javaSparkContext; - - private SparkCounters() { - this(null); - } - - public SparkCounters(JavaSparkContext javaSparkContext) { - this.javaSparkContext = javaSparkContext; - this.sparkCounterGroups = new HashMap(); - } - - public void createCounter(Enum key) { - createCounter(key.getDeclaringClass().getName(), key.name()); - } - - public void createCounter(String groupName, Enum key) { - createCounter(groupName, key.name(), 0L); - } - - public void createCounter(String groupName, String counterName) { - createCounter(groupName, counterName, 0L); - } - - public void createCounter(String groupName, String counterName, long initValue) { - getGroup(groupName).createCounter(counterName, initValue); - } - - public void increment(Enum key, long incrValue) { - increment(key.getDeclaringClass().getName(), key.name(), incrValue); - } - - public void increment(String groupName, String counterName, long value) { - SparkCounter counter = getGroup(groupName).getCounter(counterName); - if (counter == null) { - LOG.error( - String.format("counter[%s, %s] has not initialized before.", groupName, counterName)); - } else { - counter.increment(value); - } - } - - public long getValue(String groupName, String counterName) { - SparkCounter counter = getGroup(groupName).getCounter(counterName); - if (counter == null) { - LOG.error( - String.format("counter[%s, %s] has not initialized before.", groupName, counterName)); - return 0; - } else { - return counter.getValue(); - } - } - - public SparkCounter getCounter(String groupName, String counterName) { - return getGroup(groupName).getCounter(counterName); - } - - public SparkCounter getCounter(Enum key) { - return getCounter(key.getDeclaringClass().getName(), key.name()); - } - - private SparkCounterGroup getGroup(String groupName) { - SparkCounterGroup group = sparkCounterGroups.get(groupName); - if (group == null) { - group = new SparkCounterGroup(groupName, groupName, javaSparkContext); - sparkCounterGroups.put(groupName, group); - } - return group; - } - - public Map getSparkCounterGroups() { - return sparkCounterGroups; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - Map groups = getSparkCounterGroups(); - if (groups != null) { - for (Map.Entry groupEntry : groups.entrySet()) { - String groupName = groupEntry.getKey(); - SparkCounterGroup group = groupEntry.getValue(); - sb.append(groupName).append("\n"); - Map counters = group.getSparkCounters(); - for (Map.Entry counterEntry : counters.entrySet()) { - String counterName = counterEntry.getKey(); - SparkCounter counter = counterEntry.getValue(); - sb.append("\t") - .append(counterName) - .append(": ") - .append(counter.getValue()) - .append("\n"); - } - } - } - - return sb.toString(); - } - - /** - * Create a snapshot of the current counters to send back to the client. This copies the values - * of all current counters into a new SparkCounters instance that cannot be used to update the - * counters, but will serialize cleanly when sent back to the RSC client. - */ - public SparkCounters snapshot() { - SparkCounters snapshot = new SparkCounters(); - for (SparkCounterGroup group : sparkCounterGroups.values()) { - SparkCounterGroup groupSnapshot = group.snapshot(); - snapshot.sparkCounterGroups.put(groupSnapshot.getGroupName(), groupSnapshot); - } - return snapshot; - } - -} diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java deleted file mode 100644 index d6b627b6306..00000000000 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestJobHandle.java +++ /dev/null @@ -1,104 +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.hive.spark.client; - -import java.io.Serializable; - -import com.google.common.collect.Lists; -import io.netty.util.concurrent.Promise; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.InOrder; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -@RunWith(MockitoJUnitRunner.class) -public class TestJobHandle { - - @Mock private SparkClientImpl client; - @Mock private Promise promise; - @Mock private JobHandle.Listener listener; - @Mock private JobHandle.Listener listener2; - - @Test - public void testStateChanges() throws Exception { - JobHandleImpl handle = - new JobHandleImpl(client, promise, "job", Lists.newArrayList(listener)); - - assertTrue(handle.changeState(JobHandle.State.QUEUED)); - verify(listener).onJobQueued(handle); - - assertTrue(handle.changeState(JobHandle.State.STARTED)); - verify(listener).onJobStarted(handle); - - handle.addSparkJobId(1); - verify(listener).onSparkJobStarted(same(handle), eq(1)); - - assertTrue(handle.changeState(JobHandle.State.CANCELLED)); - verify(listener).onJobCancelled(handle); - - assertFalse(handle.changeState(JobHandle.State.STARTED)); - assertFalse(handle.changeState(JobHandle.State.FAILED)); - assertFalse(handle.changeState(JobHandle.State.SUCCEEDED)); - } - - @Test - public void testFailedJob() throws Exception { - JobHandleImpl handle = - new JobHandleImpl(client, promise, "job", Lists.newArrayList(listener)); - - Throwable cause = new Exception(); - when(promise.cause()).thenReturn(cause); - - assertTrue(handle.changeState(JobHandle.State.FAILED)); - verify(promise).cause(); - verify(listener).onJobFailed(handle, cause); - } - - @Test - public void testSucceededJob() throws Exception { - JobHandleImpl handle = - new JobHandleImpl(client, promise, "job", Lists.newArrayList(listener)); - - Serializable result = new Exception(); - when(promise.get()).thenReturn(result); - - assertTrue(handle.changeState(JobHandle.State.SUCCEEDED)); - verify(promise).get(); - verify(listener).onJobSucceeded(handle, result); - } - - @Test - public void testImmediateCallback() throws Exception { - JobHandleImpl handle = - new JobHandleImpl(client, promise, "job", Lists.newArrayList(listener, listener2)); - assertTrue(handle.changeState(JobHandle.State.QUEUED)); - verify(listener).onJobQueued(handle); - - handle.changeState(JobHandle.State.STARTED); - handle.addSparkJobId(1); - handle.changeState(JobHandle.State.CANCELLED); - - InOrder inOrder = inOrder(listener2); - inOrder.verify(listener2).onSparkJobStarted(same(handle), eq(1)); - inOrder.verify(listener2).onJobCancelled(same(handle)); - } - -} diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java deleted file mode 100644 index c5884cf06d5..00000000000 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestMetricsCollection.java +++ /dev/null @@ -1,173 +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.hive.spark.client; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - -import java.util.Arrays; - -import org.apache.hive.spark.client.metrics.DataReadMethod; -import org.apache.hive.spark.client.metrics.InputMetrics; -import org.apache.hive.spark.client.metrics.Metrics; -import org.apache.hive.spark.client.metrics.ShuffleReadMetrics; -import org.apache.hive.spark.client.metrics.ShuffleWriteMetrics; -import org.junit.Test; - -import com.google.common.collect.ImmutableSet; - -public class TestMetricsCollection { - - @Test - public void testMetricsAggregation() { - MetricsCollection collection = new MetricsCollection(); - // 2 jobs, 2 stages per job, 2 tasks per stage. - for (int i : Arrays.asList(1, 2)) { - for (int j : Arrays.asList(1, 2)) { - for (long k : Arrays.asList(1L, 2L)) { - collection.addMetrics(i, j, k, makeMetrics(i, j, k)); - } - } - } - - assertEquals(ImmutableSet.of(1, 2), collection.getJobIds()); - assertEquals(ImmutableSet.of(1, 2), collection.getStageIds(1)); - assertEquals(ImmutableSet.of(1L, 2L), collection.getTaskIds(1, 1)); - - Metrics task112 = collection.getTaskMetrics(1, 1, 2); - checkMetrics(task112, taskValue(1, 1, 2)); - - Metrics stage21 = collection.getStageMetrics(2, 1); - checkMetrics(stage21, stageValue(2, 1, 2)); - - Metrics job1 = collection.getJobMetrics(1); - checkMetrics(job1, jobValue(1, 2, 2)); - - Metrics global = collection.getAllMetrics(); - checkMetrics(global, globalValue(2, 2, 2)); - } - - @Test - public void testOptionalMetrics() { - long value = taskValue(1, 1, 1L); - Metrics metrics = new Metrics(value, value, value, value, value, value, value, value, value, - value, null, null, null); - - MetricsCollection collection = new MetricsCollection(); - for (int i : Arrays.asList(1, 2)) { - collection.addMetrics(i, 1, 1, metrics); - } - - Metrics global = collection.getAllMetrics(); - assertNull(global.inputMetrics); - assertNull(global.shuffleReadMetrics); - assertNull(global.shuffleWriteMetrics); - - collection.addMetrics(3, 1, 1, makeMetrics(3, 1, 1)); - - Metrics global2 = collection.getAllMetrics(); - assertNotNull(global2.inputMetrics); - assertEquals(taskValue(3, 1, 1), global2.inputMetrics.bytesRead); - - assertNotNull(global2.shuffleReadMetrics); - assertNotNull(global2.shuffleWriteMetrics); - } - - @Test - public void testInputReadMethodAggregation() { - MetricsCollection collection = new MetricsCollection(); - - long value = taskValue(1, 1, 1); - - Metrics metrics1 = new Metrics(value, value, value, value, value, value, value, value, value, - value, new InputMetrics(value), null, null); - Metrics metrics2 = new Metrics(value, value, value, value, value, value, value, value, value, - value, new InputMetrics(value), null, null); - - collection.addMetrics(1, 1, 1, metrics1); - collection.addMetrics(1, 1, 2, metrics2); - - Metrics global = collection.getAllMetrics(); - assertNotNull(global.inputMetrics); - } - - private Metrics makeMetrics(int jobId, int stageId, long taskId) { - long value = 1000000 * jobId + 1000 * stageId + taskId; - return new Metrics(value, value, value, value, value, value, value, value, value, value, - new InputMetrics(value), - new ShuffleReadMetrics((int) value, (int) value, value, value), - new ShuffleWriteMetrics(value, value)); - } - - /** - * The metric values will all be the same. This makes it easy to calculate the aggregated values - * of jobs and stages without fancy math. - */ - private long taskValue(int jobId, int stageId, long taskId) { - return 1000000 * jobId + 1000 * stageId + taskId; - } - - private long stageValue(int jobId, int stageId, int taskCount) { - long value = 0; - for (int i = 1; i <= taskCount; i++) { - value += taskValue(jobId, stageId, i); - } - return value; - } - - private long jobValue(int jobId, int stageCount, int tasksPerStage) { - long value = 0; - for (int i = 1; i <= stageCount; i++) { - value += stageValue(jobId, i, tasksPerStage); - } - return value; - } - - private long globalValue(int jobCount, int stagesPerJob, int tasksPerStage) { - long value = 0; - for (int i = 1; i <= jobCount; i++) { - value += jobValue(i, stagesPerJob, tasksPerStage); - } - return value; - } - - private void checkMetrics(Metrics metrics, long expected) { - assertEquals(expected, metrics.executorDeserializeTime); - assertEquals(expected, metrics.executorDeserializeCpuTime); - assertEquals(expected, metrics.executorRunTime); - assertEquals(expected, metrics.executorCpuTime); - assertEquals(expected, metrics.resultSize); - assertEquals(expected, metrics.jvmGCTime); - assertEquals(expected, metrics.resultSerializationTime); - assertEquals(expected, metrics.memoryBytesSpilled); - assertEquals(expected, metrics.diskBytesSpilled); - assertEquals(expected, metrics.taskDurationTime); - - assertEquals(expected, metrics.inputMetrics.bytesRead); - - assertEquals(expected, metrics.shuffleReadMetrics.remoteBlocksFetched); - assertEquals(expected, metrics.shuffleReadMetrics.localBlocksFetched); - assertEquals(expected, metrics.shuffleReadMetrics.fetchWaitTime); - assertEquals(expected, metrics.shuffleReadMetrics.remoteBytesRead); - - assertEquals(expected, metrics.shuffleWriteMetrics.shuffleBytesWritten); - assertEquals(expected, metrics.shuffleWriteMetrics.shuffleWriteTime); - } - -} diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java b/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java deleted file mode 100644 index fb31c933ca4..00000000000 --- a/spark-client/src/test/java/org/apache/hive/spark/client/TestSparkClient.java +++ /dev/null @@ -1,442 +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.hive.spark.client; - -import com.google.common.collect.Lists; -import org.apache.hive.spark.client.JobHandle.Listener; - -import org.slf4j.Logger; - -import org.slf4j.LoggerFactory; - -import org.mockito.invocation.InvocationOnMock; - -import org.mockito.stubbing.Answer; - -import org.mockito.Mockito; - -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.InputStream; -import java.io.PrintStream; -import java.io.Serializable; -import java.net.URI; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.jar.JarOutputStream; -import java.util.zip.ZipEntry; - -import com.google.common.base.Objects; -import com.google.common.base.Strings; -import com.google.common.io.ByteStreams; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hive.spark.counter.SparkCounters; -import org.apache.spark.SparkException; -import org.apache.spark.SparkFiles; -import org.apache.spark.api.java.JavaFutureAction; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; -import org.junit.Test; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; - -public class TestSparkClient { - - // Timeouts are bad... mmmkay. - private static final long TIMEOUT = 20; - private static final HiveConf HIVECONF = new HiveConf(); - - static { - HIVECONF.set("hive.spark.client.connect.timeout", "30000ms"); - } - - private Map createConf() { - Map conf = new HashMap(); - - String classpath = System.getProperty("java.class.path"); - conf.put("spark.master", "local"); - conf.put("spark.app.name", "SparkClientSuite Remote App"); - conf.put("spark.driver.extraClassPath", classpath); - conf.put("spark.executor.extraClassPath", classpath); - - if (!Strings.isNullOrEmpty(System.getProperty("spark.home"))) { - conf.put("spark.home", System.getProperty("spark.home")); - } - - return conf; - } - - @Test - public void testJobSubmission() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - JobHandle.Listener listener = newListener(); - List> listeners = Lists.newArrayList(listener);; - JobHandle handle = client.submit(new SimpleJob(), listeners); - assertEquals("hello", handle.get(TIMEOUT, TimeUnit.SECONDS)); - - // Try an invalid state transition on the handle. This ensures that the actual state - // change we're interested in actually happened, since internally the handle serializes - // state changes. - assertFalse(((JobHandleImpl)handle).changeState(JobHandle.State.SENT)); - - verify(listener).onJobStarted(handle); - verify(listener).onJobSucceeded(same(handle), eq(handle.get())); - } - }); - } - - @Test - public void testSimpleSparkJob() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - JobHandle handle = client.submit(new SparkJob()); - assertEquals(Long.valueOf(5L), handle.get(TIMEOUT, TimeUnit.SECONDS)); - } - }); - } - - @Test - public void testErrorJob() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - JobHandle.Listener listener = newListener(); - List> listeners = Lists.newArrayList(listener); - JobHandle handle = client.submit(new ErrorJob(), listeners); - try { - handle.get(TIMEOUT, TimeUnit.SECONDS); - fail("Should have thrown an exception."); - } catch (ExecutionException ee) { - assertTrue(ee.getCause() instanceof SparkException); - assertTrue(ee.getCause().getMessage().contains("IllegalStateException: Hello")); - } - - // Try an invalid state transition on the handle. This ensures that the actual state - // change we're interested in actually happened, since internally the handle serializes - // state changes. - assertFalse(((JobHandleImpl)handle).changeState(JobHandle.State.SENT)); - - verify(listener).onJobQueued(handle); - verify(listener).onJobStarted(handle); - verify(listener).onJobFailed(same(handle), any(Throwable.class)); - } - }); - } - - @Test - public void testSyncRpc() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - Future result = client.run(new SyncRpc()); - assertEquals("Hello", result.get(TIMEOUT, TimeUnit.SECONDS)); - } - }); - } - - @Test - public void testMetricsCollection() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - JobHandle.Listener listener = newListener(); - List> listeners = Lists.newArrayList(listener); - JobHandle future = client.submit(new AsyncSparkJob(), listeners); - future.get(TIMEOUT, TimeUnit.SECONDS); - MetricsCollection metrics = future.getMetrics(); - assertEquals(1, metrics.getJobIds().size()); - assertTrue(metrics.getAllMetrics().executorRunTime >= 0L); - verify(listener).onSparkJobStarted(same(future), - eq(metrics.getJobIds().iterator().next())); - - JobHandle.Listener listener2 = newListener(); - List> listeners2 = Lists.newArrayList(listener2); - JobHandle future2 = client.submit(new AsyncSparkJob(), listeners2); - future2.get(TIMEOUT, TimeUnit.SECONDS); - MetricsCollection metrics2 = future2.getMetrics(); - assertEquals(1, metrics2.getJobIds().size()); - assertFalse(Objects.equal(metrics.getJobIds(), metrics2.getJobIds())); - assertTrue(metrics2.getAllMetrics().executorRunTime >= 0L); - verify(listener2).onSparkJobStarted(same(future2), - eq(metrics2.getJobIds().iterator().next())); - } - }); - } - - @Test - public void testAddJarsAndFiles() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - File jar = null; - File file = null; - - try { - // Test that adding a jar to the remote context makes it show up in the classpath. - jar = File.createTempFile("test", ".jar"); - - JarOutputStream jarFile = new JarOutputStream(new FileOutputStream(jar)); - jarFile.putNextEntry(new ZipEntry("test.resource")); - jarFile.write("test resource".getBytes("UTF-8")); - jarFile.closeEntry(); - jarFile.close(); - - client.addJar(new URI("file:" + jar.getAbsolutePath())) - .get(TIMEOUT, TimeUnit.SECONDS); - - // Need to run a Spark job to make sure the jar is added to the class loader. Monitoring - // SparkContext#addJar() doesn't mean much, we can only be sure jars have been distributed - // when we run a task after the jar has been added. - String result = client.submit(new JarJob()).get(TIMEOUT, TimeUnit.SECONDS); - assertEquals("test resource", result); - - // Test that adding a file to the remote context makes it available to executors. - file = File.createTempFile("test", ".file"); - - FileOutputStream fileStream = new FileOutputStream(file); - fileStream.write("test file".getBytes("UTF-8")); - fileStream.close(); - - client.addJar(new URI("file:" + file.getAbsolutePath())) - .get(TIMEOUT, TimeUnit.SECONDS); - - // The same applies to files added with "addFile". They're only guaranteed to be available - // to tasks started after the addFile() call completes. - result = client.submit(new FileJob(file.getName())) - .get(TIMEOUT, TimeUnit.SECONDS); - assertEquals("test file", result); - } finally { - if (jar != null) { - jar.delete(); - } - if (file != null) { - file.delete(); - } - } - } - }); - } - - @Test - public void testCounters() throws Exception { - runTest(new TestFunction() { - @Override - public void call(SparkClient client) throws Exception { - JobHandle job = client.submit(new CounterIncrementJob()); - job.get(TIMEOUT, TimeUnit.SECONDS); - - SparkCounters counters = job.getSparkCounters(); - assertNotNull(counters); - - long expected = 1 + 2 + 3 + 4 + 5; - assertEquals(expected, counters.getCounter("group1", "counter1").getValue()); - assertEquals(expected, counters.getCounter("group2", "counter2").getValue()); - } - }); - } - - private static final Logger LOG = LoggerFactory.getLogger(TestSparkClient.class); - - private JobHandle.Listener newListener() { - @SuppressWarnings("unchecked") - JobHandle.Listener listener = mock(JobHandle.Listener.class); - answerWhen(listener, "cancelled").onJobCancelled(Mockito.>any()); - answerWhen(listener, "queued").onJobQueued(Mockito.>any()); - answerWhen(listener, "started").onJobStarted(Mockito.>any()); - answerWhen(listener, "succeeded").onJobSucceeded( - Mockito.>any(), Mockito.any()); - answerWhen(listener, "job started").onSparkJobStarted( - Mockito.>any(), Mockito.anyInt()); - Mockito.doAnswer(new Answer() { - public Void answer(InvocationOnMock invocation) throws Throwable { - @SuppressWarnings("rawtypes") - JobHandleImpl arg = ((JobHandleImpl)invocation.getArguments()[0]); - LOG.info("Job failed " + arg.getClientJobId(), - (Throwable)invocation.getArguments()[1]); - return null; - }; - }).when(listener).onJobFailed(Mockito.>any(), Mockito.any()); - return listener; - } - - protected Listener answerWhen( - Listener listener, final String logStr) { - return Mockito.doAnswer(new Answer() { - public Void answer(InvocationOnMock invocation) throws Throwable { - @SuppressWarnings("rawtypes") - JobHandleImpl arg = ((JobHandleImpl)invocation.getArguments()[0]); - LOG.info("Job " + logStr + " " + arg.getClientJobId()); - return null; - }; - }).when(listener); - } - - private void runTest(TestFunction test) throws Exception { - Map conf = createConf(); - SparkClientFactory.initialize(conf); - SparkClient client = null; - try { - test.config(conf); - client = SparkClientFactory.createClient(conf, HIVECONF, UUID.randomUUID().toString()); - test.call(client); - } finally { - if (client != null) { - client.stop(); - } - SparkClientFactory.stop(); - } - } - - private static class SimpleJob implements Job { - - @Override - public String call(JobContext jc) { - return "hello"; - } - - } - - private static class ErrorJob implements Job { - - @Override - public String call(JobContext jc) { - throw new IllegalStateException("Hello"); - } - - } - - private static class SparkJob implements Job { - - @Override - public Long call(JobContext jc) { - JavaRDD rdd = jc.sc().parallelize(Arrays.asList(1, 2, 3, 4, 5)); - return rdd.count(); - } - - } - - private static class AsyncSparkJob implements Job { - - @Override - public Integer call(JobContext jc) throws Exception { - JavaRDD rdd = jc.sc().parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaFutureAction future = jc.monitor(rdd.foreachAsync(new VoidFunction() { - @Override - public void call(Integer l) throws Exception { - - } - }), null, null); - - future.get(TIMEOUT, TimeUnit.SECONDS); - - return 1; - } - - } - - private static class JarJob implements Job, Function { - - @Override - public String call(JobContext jc) { - return jc.sc().parallelize(Arrays.asList(1)).map(this).collect().get(0); - } - - @Override - public String call(Integer i) throws Exception { - ClassLoader ccl = Thread.currentThread().getContextClassLoader(); - InputStream in = ccl.getResourceAsStream("test.resource"); - byte[] bytes = ByteStreams.toByteArray(in); - in.close(); - return new String(bytes, 0, bytes.length, "UTF-8"); - } - - } - - private static class FileJob implements Job, Function { - - private final String fileName; - - FileJob(String fileName) { - this.fileName = fileName; - } - - @Override - public String call(JobContext jc) { - return jc.sc().parallelize(Arrays.asList(1)).map(this).collect().get(0); - } - - @Override - public String call(Integer i) throws Exception { - InputStream in = new FileInputStream(SparkFiles.get(fileName)); - byte[] bytes = ByteStreams.toByteArray(in); - in.close(); - return new String(bytes, 0, bytes.length, "UTF-8"); - } - - } - - private static class CounterIncrementJob implements Job, VoidFunction { - - private SparkCounters counters; - - @Override - public String call(JobContext jc) { - counters = new SparkCounters(jc.sc()); - counters.createCounter("group1", "counter1"); - counters.createCounter("group2", "counter2"); - - jc.monitor(jc.sc().parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).foreachAsync(this), - counters, null); - - return null; - } - - @Override - public void call(Integer l) throws Exception { - counters.getCounter("group1", "counter1").increment(l.longValue()); - counters.getCounter("group2", "counter2").increment(l.longValue()); - } - - } - - private static class SyncRpc implements Job { - - @Override - public String call(JobContext jc) { - return "Hello"; - } - - } - - private abstract static class TestFunction { - abstract void call(SparkClient client) throws Exception; - void config(Map conf) { } - } - -} diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java deleted file mode 100644 index fb736471b2b..00000000000 --- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestKryoMessageCodec.java +++ /dev/null @@ -1,233 +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.hive.spark.client.rpc; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; - -import com.google.common.collect.Lists; -import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.logging.LoggingHandler; -import org.junit.Test; -import static org.junit.Assert.*; - -public class TestKryoMessageCodec { - - private static final String MESSAGE = "Hello World!"; - - @Test - public void testKryoCodec() throws Exception { - List objects = encodeAndDecode(MESSAGE, null); - assertEquals(1, objects.size()); - assertEquals(MESSAGE, objects.get(0)); - } - - @Test - public void testFragmentation() throws Exception { - ByteBuf buf = newBuffer(); - Object[] messages = { "msg1", "msg2" }; - int[] indices = new int[messages.length]; - - KryoMessageCodec codec = new KryoMessageCodec(0); - - for (int i = 0; i < messages.length; i++) { - codec.encode(null, messages[i], buf); - indices[i] = buf.writerIndex(); - } - - List objects = Lists.newArrayList(); - - // Don't read enough data for the first message to be decoded. - codec.decode(null, buf.slice(0, indices[0] - 1), objects); - assertEquals(0, objects.size()); - - // Read enough data for just the first message to be decoded. - codec.decode(null, buf.slice(0, indices[0] + 1), objects); - assertEquals(1, objects.size()); - } - - @Test - public void testEmbeddedChannel() throws Exception { - EmbeddedChannel c = new EmbeddedChannel( - new LoggingHandler(getClass()), - new KryoMessageCodec(0)); - c.writeAndFlush(MESSAGE); - assertEquals(1, c.outboundMessages().size()); - assertFalse(MESSAGE.getClass().equals(c.outboundMessages().peek().getClass())); - Object readOutboundResult = c.readOutbound(); - c.writeInbound(readOutboundResult); - assertEquals(1, c.inboundMessages().size()); - assertEquals(MESSAGE, c.readInbound()); - c.close(); - } - - @Test - public void testAutoRegistration() throws Exception { - KryoMessageCodec codec = new KryoMessageCodec(0, TestMessage.class); - ByteBuf buf = newBuffer(); - codec.encode(null, new TestMessage(), buf); - - List out = Lists.newArrayList(); - codec.decode(null, buf, out); - - assertEquals(1, out.size()); - assertTrue(out.get(0) instanceof TestMessage); - } - - @Test - public void testMaxMessageSize() throws Exception { - KryoMessageCodec codec = new KryoMessageCodec(1024); - ByteBuf buf = newBuffer(); - codec.encode(null, new TestMessage(new byte[512]), buf); - - try { - codec.encode(null, new TestMessage(new byte[1025]), buf); - fail("Should have failed to encode large message."); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().indexOf("maximum allowed size") > 0); - } - - KryoMessageCodec unlimited = new KryoMessageCodec(0); - buf = newBuffer(); - unlimited.encode(null, new TestMessage(new byte[1025]), buf); - - try { - List out = Lists.newArrayList(); - codec.decode(null, buf, out); - fail("Should have failed to decode large message."); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().indexOf("maximum allowed size") > 0); - } - } - - @Test - public void testNegativeMessageSize() throws Exception { - KryoMessageCodec codec = new KryoMessageCodec(1024); - ByteBuf buf = newBuffer(); - buf.writeInt(-1); - - try { - List out = Lists.newArrayList(); - codec.decode(null, buf, out); - fail("Should have failed to decode message with negative size."); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().indexOf("must be positive") > 0); - } - } - - @Test - public void testEncryptionOnly() throws Exception { - List objects = Collections.emptyList(); - try { - objects = encodeAndDecode(MESSAGE, new TestEncryptionHandler(true, false)); - } catch (Exception e) { - // Pass. - } - // Do this check in case the ciphertext actually makes sense in some way. - for (Object msg : objects) { - assertFalse(MESSAGE.equals(objects.get(0))); - } - } - - @Test - public void testDecryptionOnly() throws Exception { - List objects = Collections.emptyList(); - try { - objects = encodeAndDecode(MESSAGE, new TestEncryptionHandler(false, true)); - } catch (Exception e) { - // Pass. - } - // Do this check in case the decrypted plaintext actually makes sense in some way. - for (Object msg : objects) { - assertFalse(MESSAGE.equals(objects.get(0))); - } - } - - @Test - public void testEncryptDecrypt() throws Exception { - List objects = encodeAndDecode(MESSAGE, new TestEncryptionHandler(true, true)); - assertEquals(1, objects.size()); - assertEquals(MESSAGE, objects.get(0)); - } - - private List encodeAndDecode(Object message, KryoMessageCodec.EncryptionHandler eh) - throws Exception { - ByteBuf buf = newBuffer(); - KryoMessageCodec codec = new KryoMessageCodec(0); - codec.setEncryptionHandler(eh); - codec.encode(null, message, buf); - - List objects = Lists.newArrayList(); - codec.decode(null, buf, objects); - return objects; - } - - private ByteBuf newBuffer() { - return UnpooledByteBufAllocator.DEFAULT.buffer(1024); - } - - private static class TestMessage { - byte[] data; - - TestMessage() { - this(null); - } - - TestMessage(byte[] data) { - this.data = data; - } - } - - private static class TestEncryptionHandler implements KryoMessageCodec.EncryptionHandler { - - private static final byte KEY = 0x42; - - private final boolean encrypt; - private final boolean decrypt; - - TestEncryptionHandler(boolean encrypt, boolean decrypt) { - this.encrypt = encrypt; - this.decrypt = decrypt; - } - - public byte[] wrap(byte[] data, int offset, int len) throws IOException { - return encrypt ? transform(data, offset, len) : data; - } - - public byte[] unwrap(byte[] data, int offset, int len) throws IOException { - return decrypt ? transform(data, offset, len) : data; - } - - public void dispose() throws IOException { - - } - - private byte[] transform(byte[] data, int offset, int len) { - byte[] dest = new byte[len]; - for (int i = 0; i < len; i++) { - dest[i] = (byte) (data[offset + i] ^ KEY); - } - return dest; - } - - } - -} diff --git a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java b/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java deleted file mode 100644 index 013bcff30c1..00000000000 --- a/spark-client/src/test/java/org/apache/hive/spark/client/rpc/TestRpc.java +++ /dev/null @@ -1,443 +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.hive.spark.client.rpc; - -import java.io.Closeable; -import java.net.InetAddress; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicInteger; - -import javax.security.sasl.SaslException; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.util.concurrent.Future; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.hive.common.ServerUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.*; - -public class TestRpc { - - private static final Logger LOG = LoggerFactory.getLogger(TestRpc.class); - - private Collection closeables; - private static final Map emptyConfig = - ImmutableMap.of(HiveConf.ConfVars.SPARK_RPC_CHANNEL_LOG_LEVEL.varname, "DEBUG"); - private static final int RETRY_ACQUIRE_PORT_COUNT = 10; - - @Before - public void setUp() { - closeables = Lists.newArrayList(); - } - - @After - public void cleanUp() throws Exception { - for (Closeable c : closeables) { - IOUtils.closeQuietly(c); - } - } - - private T autoClose(T closeable) { - closeables.add(closeable); - return closeable; - } - - @Test - public void testRpcDispatcher() throws Exception { - Rpc serverRpc = autoClose(Rpc.createEmbedded(new TestDispatcher())); - Rpc clientRpc = autoClose(Rpc.createEmbedded(new TestDispatcher())); - - TestMessage outbound = new TestMessage("Hello World!"); - Future call = clientRpc.call(outbound, TestMessage.class); - - LOG.debug("Transferring messages..."); - transfer(serverRpc, clientRpc); - - TestMessage reply = call.get(10, TimeUnit.SECONDS); - assertEquals(outbound.message, reply.message); - } - - @Test - public void testClientServer() throws Exception { - RpcServer server = autoClose(new RpcServer(emptyConfig)); - Rpc[] rpcs = createRpcConnection(server); - Rpc serverRpc = rpcs[0]; - Rpc client = rpcs[1]; - - TestMessage outbound = new TestMessage("Hello World!"); - Future call = client.call(outbound, TestMessage.class); - TestMessage reply = call.get(10, TimeUnit.SECONDS); - assertEquals(outbound.message, reply.message); - - TestMessage another = new TestMessage("Hello again!"); - Future anotherCall = client.call(another, TestMessage.class); - TestMessage anotherReply = anotherCall.get(10, TimeUnit.SECONDS); - assertEquals(another.message, anotherReply.message); - - String errorMsg = "This is an error."; - try { - client.call(new ErrorCall(errorMsg)).get(10, TimeUnit.SECONDS); - } catch (ExecutionException ee) { - assertTrue(ee.getCause() instanceof RpcException); - assertTrue(ee.getCause().getMessage().indexOf(errorMsg) >= 0); - } - - // Test from server to client too. - TestMessage serverMsg = new TestMessage("Hello from the server!"); - Future serverCall = serverRpc.call(serverMsg, TestMessage.class); - TestMessage serverReply = serverCall.get(10, TimeUnit.SECONDS); - assertEquals(serverMsg.message, serverReply.message); - } - - @Test - public void testServerAddress() throws Exception { - String hostAddress = InetAddress.getLocalHost().getHostName(); - Map config = new HashMap(); - - // Test if rpc_server_address is configured - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, hostAddress); - RpcServer server1 = autoClose(new RpcServer(config)); - assertTrue("Host address should match the expected one", server1.getAddress() == hostAddress); - - // Test if rpc_server_address is not configured but HS2 server host is configured - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, ""); - config.put(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, hostAddress); - RpcServer server2 = autoClose(new RpcServer(config)); - assertTrue("Host address should match the expected one", server2.getAddress() == hostAddress); - - // Test if both are not configured - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_ADDRESS.varname, ""); - config.put(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, ""); - RpcServer server3 = autoClose(new RpcServer(config)); - assertTrue("Host address should match the expected one", server3.getAddress() == InetAddress.getLocalHost().getHostName()); - } - - @Test - public void testBadHello() throws Exception { - RpcServer server = autoClose(new RpcServer(emptyConfig)); - - Future serverRpcFuture = server.registerClient("client", "newClient", - new TestDispatcher()); - NioEventLoopGroup eloop = new NioEventLoopGroup(); - - Future clientRpcFuture = Rpc.createClient(emptyConfig, eloop, - "localhost", server.getPort(), "client", "wrongClient", new TestDispatcher()); - - try { - autoClose(clientRpcFuture.get(10, TimeUnit.SECONDS)); - fail("Should have failed to create client with wrong secret."); - } catch (ExecutionException ee) { - // On failure, the SASL handler will throw an exception indicating that the SASL - // negotiation failed. - assertTrue("Unexpected exception: " + ee.getCause(), - ee.getCause() instanceof SaslException); - } - - serverRpcFuture.cancel(true); - } - - @Test - public void testServerPort() throws Exception { - Map config = new HashMap(); - - RpcServer server0 = new RpcServer(config); - assertTrue("Empty port range should return a random valid port: " + server0.getPort(), server0.getPort() >= 0); - IOUtils.closeQuietly(server0); - - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49152-49222,49223,49224-49333"); - RpcServer server1 = new RpcServer(config); - assertTrue("Port should be within configured port range:" + server1.getPort(), server1.getPort() >= 49152 && server1.getPort() <= 49333); - IOUtils.closeQuietly(server1); - - int expectedPort = ServerUtils.findFreePort(); - RpcServer server2 = null; - for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) { - try { - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort)); - server2 = new RpcServer(config); - break; - } catch (Exception e) { - LOG.debug("Error while connecting to port " + expectedPort + " retrying: " + e.getMessage()); - expectedPort = ServerUtils.findFreePort(); - } - } - - assertNotNull("Unable to create RpcServer with any attempted port", server2); - assertEquals("Port should match configured one: " + server2.getPort(), expectedPort, server2.getPort()); - IOUtils.closeQuietly(server2); - - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, "49552-49222,49223,49224-49333"); - try { - autoClose(new RpcServer(config)); - assertTrue("Invalid port range should throw an exception", false); // Should not reach here - } catch(IllegalArgumentException e) { - assertEquals( - "Malformed configuration value for " + HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, - e.getMessage()); - } - - // Retry logic - expectedPort = ServerUtils.findFreePort(); - RpcServer server3 = null; - for (int i = 0; i < RETRY_ACQUIRE_PORT_COUNT; i++) { - try { - config.put(HiveConf.ConfVars.SPARK_RPC_SERVER_PORT.varname, String.valueOf(expectedPort) + ",21-23"); - server3 = new RpcServer(config); - break; - } catch (Exception e) { - LOG.debug("Error while connecting to port " + expectedPort + " retrying"); - expectedPort = ServerUtils.findFreePort(); - } - } - assertNotNull("Unable to create RpcServer with any attempted port", server3); - assertEquals("Port should match configured one:" + server3.getPort(), expectedPort, server3.getPort()); - IOUtils.closeQuietly(server3); - } - - @Test - public void testCloseListener() throws Exception { - RpcServer server = autoClose(new RpcServer(emptyConfig)); - Rpc[] rpcs = createRpcConnection(server); - Rpc client = rpcs[1]; - - final AtomicInteger closeCount = new AtomicInteger(); - client.addListener(new Rpc.Listener() { - @Override - public void rpcClosed(Rpc rpc) { - closeCount.incrementAndGet(); - } - }); - - client.close(); - client.close(); - assertEquals(1, closeCount.get()); - } - - @Test - public void testNotDeserializableRpc() throws Exception { - RpcServer server = autoClose(new RpcServer(emptyConfig)); - Rpc[] rpcs = createRpcConnection(server); - Rpc client = rpcs[1]; - - try { - client.call(new NotDeserializable(42)).get(10, TimeUnit.SECONDS); - } catch (ExecutionException ee) { - assertTrue(ee.getCause() instanceof RpcException); - assertTrue(ee.getCause().getMessage().indexOf("KryoException") >= 0); - } - } - - @Test - public void testEncryption() throws Exception { - Map eConf = ImmutableMap.builder() - .putAll(emptyConfig) - .put(RpcConfiguration.RPC_SASL_OPT_PREFIX + "qop", Rpc.SASL_AUTH_CONF) - .build(); - RpcServer server = autoClose(new RpcServer(eConf)); - Rpc[] rpcs = createRpcConnection(server, eConf, null); - Rpc client = rpcs[1]; - - TestMessage outbound = new TestMessage("Hello World!"); - Future call = client.call(outbound, TestMessage.class); - TestMessage reply = call.get(10, TimeUnit.SECONDS); - assertEquals(outbound.message, reply.message); - } - - @Test - public void testClientTimeout() throws Exception { - Map conf = ImmutableMap.builder() - .putAll(emptyConfig) - .build(); - RpcServer server = autoClose(new RpcServer(conf)); - String secret = server.createSecret(); - - try { - autoClose(server.registerClient("client", secret, new TestDispatcher(), 1L).get()); - fail("Server should have timed out client."); - } catch (ExecutionException ee) { - assertTrue(ee.getCause() instanceof TimeoutException); - } - - NioEventLoopGroup eloop = new NioEventLoopGroup(); - Future clientRpcFuture = Rpc.createClient(conf, eloop, - "localhost", server.getPort(), "client", secret, new TestDispatcher()); - try { - autoClose(clientRpcFuture.get()); - fail("Client should have failed to connect to server."); - } catch (ExecutionException ee) { - // Error should not be a timeout. - assertFalse(ee.getCause() instanceof TimeoutException); - } - } - - @Test - public void testRpcServerMultiThread() throws Exception { - final RpcServer server = autoClose(new RpcServer(emptyConfig)); - final String msg = "Hello World!"; - Callable callable = () -> { - Rpc[] rpcs = createRpcConnection(server, emptyConfig, UUID.randomUUID().toString()); - Rpc rpc; - if (ThreadLocalRandom.current().nextBoolean()) { - rpc = rpcs[0]; - } else { - rpc = rpcs[1]; - } - TestMessage outbound = new TestMessage("Hello World!"); - Future call = rpc.call(outbound, TestMessage.class); - TestMessage reply = call.get(10, TimeUnit.SECONDS); - return reply.message; - }; - final int numThreads = ThreadLocalRandom.current().nextInt(5) + 5; - ExecutorService executor = Executors.newFixedThreadPool(numThreads); - List> futures = new ArrayList<>(numThreads); - for (int i = 0; i < numThreads; i++) { - futures.add(executor.submit(callable)); - } - executor.shutdown(); - for (java.util.concurrent.Future future : futures) { - assertEquals(msg, future.get()); - } - } - - private void transfer(Rpc serverRpc, Rpc clientRpc) { - EmbeddedChannel client = (EmbeddedChannel) clientRpc.getChannel(); - EmbeddedChannel server = (EmbeddedChannel) serverRpc.getChannel(); - - server.runPendingTasks(); - client.runPendingTasks(); - - int count = 0; - while (!client.outboundMessages().isEmpty()) { - Object readOutboundResult = client.readOutbound(); - server.writeInbound(readOutboundResult); - count++; - } - server.flush(); - LOG.debug("Transferred {} outbound client messages.", count); - - count = 0; - while (!server.outboundMessages().isEmpty()) { - Object readOutboundResult = server.readOutbound(); - client.writeInbound(readOutboundResult); - count++; - } - client.flush(); - LOG.debug("Transferred {} outbound server messages.", count); - } - - /** - * Creates a client connection between the server and a client. - * - * @return two-tuple (server rpc, client rpc) - */ - private Rpc[] createRpcConnection(RpcServer server) throws Exception { - return createRpcConnection(server, emptyConfig, null); - } - - private Rpc[] createRpcConnection(RpcServer server, Map clientConf, - String clientId) throws Exception { - if (clientId == null) { - clientId = "client"; - } - String secret = server.createSecret(); - Future serverRpcFuture = server.registerClient(clientId, secret, new TestDispatcher()); - NioEventLoopGroup eloop = new NioEventLoopGroup(); - Future clientRpcFuture = Rpc.createClient(clientConf, eloop, - "localhost", server.getPort(), clientId, secret, new TestDispatcher()); - - Rpc serverRpc = autoClose(serverRpcFuture.get(10, TimeUnit.SECONDS)); - Rpc clientRpc = autoClose(clientRpcFuture.get(10, TimeUnit.SECONDS)); - return new Rpc[]{serverRpc, clientRpc}; - } - - private static class TestMessage { - - final String message; - - public TestMessage() { - this(null); - } - - public TestMessage(String message) { - this.message = message; - } - - } - - private static class ErrorCall { - - final String error; - - public ErrorCall() { - this(null); - } - - public ErrorCall(String error) { - this.error = error; - } - - } - - private static class NotDeserializable { - - NotDeserializable(int unused) { - - } - - } - - private static class TestDispatcher extends RpcDispatcher { - protected TestMessage handle(ChannelHandlerContext ctx, TestMessage msg) { - return msg; - } - - protected void handle(ChannelHandlerContext ctx, ErrorCall msg) { - throw new IllegalArgumentException(msg.error); - } - - protected void handle(ChannelHandlerContext ctx, NotDeserializable msg) { - // No op. Shouldn't actually be called, if it is, the test will fail. - } - - } -} diff --git a/spark-client/src/test/resources/log4j2.properties b/spark-client/src/test/resources/log4j2.properties deleted file mode 100644 index c65ef874558..00000000000 --- a/spark-client/src/test/resources/log4j2.properties +++ /dev/null @@ -1,38 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -status = INFO -name = SparkClientLog4j2 -packages = org.apache.hadoop.hive.ql.log - -# list of properties -property.spark.log.level = DEBUG -property.spark.root.logger = console - -# list of all appenders -appenders = console - -# console appender -appender.console.type = Console -appender.console.name = console -appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# root logger -rootLogger.level = ${sys:spark.log.level} -rootLogger.appenderRefs = root -rootLogger.appenderRef.root.ref = ${sys:spark.root.logger} diff --git a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java index ad6dad4ae87..49887aea9fb 100644 --- a/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java +++ b/testutils/ptest2/src/main/java/org/apache/hive/ptest/execution/conf/TestParser.java @@ -50,7 +50,6 @@ public class TestParser { .trimResults().omitEmptyStrings(); private static final String QTEST_MODULE_NAME = "itests/qtest"; - private static final String QTEST_SPARK_MODULE_NAME = "itests/qtest-spark"; private final AtomicInteger batchIdCounter; @@ -247,11 +246,7 @@ private void expandTestProperties(Set tests, Map pro } private String getModuleName(String driverName) { - if (driverName.toLowerCase().contains("spark")) { - return QTEST_SPARK_MODULE_NAME; - } else { - return QTEST_MODULE_NAME; - } + return QTEST_MODULE_NAME; } public Supplier> parse() {