Index: contrib/src/test/results/clientnegative/case_with_row_sequence.q.out =================================================================== --- contrib/src/test/results/clientnegative/case_with_row_sequence.q.out (revision 1500544) +++ contrib/src/test/results/clientnegative/case_with_row_sequence.q.out (working copy) @@ -25,4 +25,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java =================================================================== --- cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (revision 1500544) +++ cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (working copy) @@ -52,9 +52,9 @@ import org.apache.hadoop.hive.ql.CommandNeedRetryException; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper; import org.apache.hadoop.hive.ql.parse.HiveParser; import org.apache.hadoop.hive.ql.parse.VariableSubstitution; import org.apache.hadoop.hive.ql.processors.CommandProcessor; Index: ql/src/test/results/clientnegative/script_broken_pipe2.q.out =================================================================== --- ql/src/test/results/clientnegative/script_broken_pipe2.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/script_broken_pipe2.q.out (working copy) @@ -13,4 +13,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out =================================================================== --- ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff.q.out (working copy) @@ -2,4 +2,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.MapRedTask. Unable to initialize custom script. +FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script. Index: ql/src/test/results/clientnegative/stats_publisher_error_1.q.out =================================================================== --- ql/src/test/results/clientnegative/stats_publisher_error_1.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/stats_publisher_error_1.q.out (working copy) @@ -39,4 +39,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/local_mapred_error_cache.q.out =================================================================== --- ql/src/test/results/clientnegative/local_mapred_error_cache.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/local_mapred_error_cache.q.out (working copy) @@ -20,4 +20,4 @@ org.apache.hadoop.hive.ql.metadata.HiveException: [Error 20003]: An error occurred when trying to close the Operator running your custom script. #### A masked pattern was here #### Error during job, obtaining debugging information... -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out =================================================================== --- ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/mapreduce_stack_trace_hadoop20.q.out (working copy) @@ -10,4 +10,4 @@ Hive Runtime Error while processing row {"key":"238","value":"val_238"} FATAL ExecMapper: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"key":"238","value":"val_238"} Hive Runtime Error while processing row {"key":"238","value":"val_238"} -FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.MapRedTask. Unable to initialize custom script. +FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script. Index: ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out =================================================================== --- ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q.out (working copy) @@ -2,4 +2,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.MapRedTask. Unable to initialize custom script. +FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script. Index: ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out =================================================================== --- ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out (working copy) @@ -28,4 +28,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/dyn_part3.q.out =================================================================== --- ql/src/test/results/clientnegative/dyn_part3.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/dyn_part3.q.out (working copy) @@ -17,4 +17,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/index_compact_size_limit.q.out =================================================================== --- ql/src/test/results/clientnegative/index_compact_size_limit.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/index_compact_size_limit.q.out (working copy) @@ -42,4 +42,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udfnull.q.out =================================================================== --- ql/src/test/results/clientnegative/udfnull.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udfnull.q.out (working copy) @@ -16,4 +16,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/cachingprintstream.q.out =================================================================== --- ql/src/test/results/clientnegative/cachingprintstream.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/cachingprintstream.q.out (working copy) @@ -34,4 +34,4 @@ #### A masked pattern was here #### End cached logs. -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/script_broken_pipe3.q.out =================================================================== --- ql/src/test/results/clientnegative/script_broken_pipe3.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/script_broken_pipe3.q.out (working copy) @@ -13,4 +13,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udf_reflect_neg.q.out =================================================================== --- ql/src/test/results/clientnegative/udf_reflect_neg.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udf_reflect_neg.q.out (working copy) @@ -19,4 +19,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out =================================================================== --- ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/cluster_tasklog_retrieval.q.out (working copy) @@ -11,4 +11,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/stats_publisher_error_2.q.out =================================================================== --- ql/src/test/results/clientnegative/stats_publisher_error_2.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/stats_publisher_error_2.q.out (working copy) @@ -37,4 +37,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udf_assert_true2.q.out =================================================================== --- ql/src/test/results/clientnegative/udf_assert_true2.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udf_assert_true2.q.out (working copy) @@ -73,4 +73,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/minimr_broken_pipe.q.out =================================================================== --- ql/src/test/results/clientnegative/minimr_broken_pipe.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/minimr_broken_pipe.q.out (working copy) @@ -3,4 +3,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 20003 from org.apache.hadoop.hive.ql.exec.MapRedTask. An error occurred when trying to close the Operator running your custom script. +FAILED: Execution Error, return code 20003 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. An error occurred when trying to close the Operator running your custom script. Index: ql/src/test/results/clientnegative/script_broken_pipe1.q.out =================================================================== --- ql/src/test/results/clientnegative/script_broken_pipe1.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/script_broken_pipe1.q.out (working copy) @@ -13,4 +13,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udf_test_error.q.out =================================================================== --- ql/src/test/results/clientnegative/udf_test_error.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udf_test_error.q.out (working copy) @@ -16,4 +16,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/autolocal1.q.out =================================================================== --- ql/src/test/results/clientnegative/autolocal1.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/autolocal1.q.out (working copy) @@ -12,4 +12,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/fatal.q.out =================================================================== --- ql/src/test/results/clientnegative/fatal.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/fatal.q.out (working copy) @@ -2,4 +2,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: file:/tmp/nzhang/hive_2010-08-02_13-41-52_752_1156521578782717030/-mr-10000 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udf_assert_true.q.out =================================================================== --- ql/src/test/results/clientnegative/udf_assert_true.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udf_assert_true.q.out (working copy) @@ -149,4 +149,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out =================================================================== --- ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/mapreduce_stack_trace.q.out (working copy) @@ -2,4 +2,4 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src #### A masked pattern was here #### -FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.MapRedTask. Unable to initialize custom script. +FAILED: Execution Error, return code 20000 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. Unable to initialize custom script. Index: ql/src/test/results/clientnegative/index_compact_entry_limit.q.out =================================================================== --- ql/src/test/results/clientnegative/index_compact_entry_limit.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/index_compact_entry_limit.q.out (working copy) @@ -42,4 +42,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/script_error.q.out =================================================================== --- ql/src/test/results/clientnegative/script_error.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/script_error.q.out (working copy) @@ -59,4 +59,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/udf_test_error_reduce.q.out =================================================================== --- ql/src/test/results/clientnegative/udf_test_error_reduce.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/udf_test_error_reduce.q.out (working copy) @@ -21,4 +21,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/serde_regex2.q.out =================================================================== --- ql/src/test/results/clientnegative/serde_regex2.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/serde_regex2.q.out (working copy) @@ -64,4 +64,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out =================================================================== --- ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out (revision 1500544) +++ ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out (working copy) @@ -37,4 +37,4 @@ Logs: #### A masked pattern was here #### -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask Index: ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out =================================================================== --- ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out (revision 1500544) +++ ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out (working copy) @@ -84,7 +84,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: -- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin. -- Tests a join which is not converted to a map join, the output should be bucketed and sorted. Index: ql/src/test/results/clientpositive/mapjoin_hook.q.out =================================================================== --- ql/src/test/results/clientpositive/mapjoin_hook.q.out (revision 1500544) +++ ql/src/test/results/clientpositive/mapjoin_hook.q.out (working copy) @@ -37,7 +37,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask [MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 1 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 1 PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src3.value @@ -55,7 +55,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask Execution failed with exit status: 3 Obtaining error information @@ -67,5 +67,5 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask [MapJoinCounter PostHook] CONVERTED_LOCAL_MAPJOIN: 2 CONVERTED_MAPJOIN: 0 LOCAL_MAPJOIN: 0 COMMON_JOIN: 0 BACKUP_COMMON_JOIN: 2 Index: ql/src/test/results/clientpositive/auto_join25.q.out =================================================================== --- ql/src/test/results/clientpositive/auto_join25.q.out (revision 1500544) +++ ql/src/test/results/clientpositive/auto_join25.q.out (working copy) @@ -27,7 +27,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest1 SELECT src1.key, src2.value where (src1.ds = '2008-04-08' or src1.ds = '2008-04-09' )and (src1.hr = '12' or src1.hr = '11') @@ -75,7 +75,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask Execution failed with exit status: 3 Obtaining error information @@ -87,7 +87,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) JOIN src src3 ON (src1.key + src2.key = src3.key) INSERT OVERWRITE TABLE dest_j2 SELECT src1.key, src3.value POSTHOOK: type: QUERY @@ -135,7 +135,7 @@ #### A masked pattern was here #### FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.MapredLocalTask -ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.MapRedTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key) INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value POSTHOOK: type: QUERY Index: ql/src/test/results/clientpositive/loadpart_err.q.out =================================================================== --- ql/src/test/results/clientpositive/loadpart_err.q.out (revision 1500544) +++ ql/src/test/results/clientpositive/loadpart_err.q.out (working copy) @@ -9,7 +9,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@loadpart1@ds=2009-01-01 -FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask PREHOOK: query: DESCRIBE loadpart1 PREHOOK: type: DESCTABLE POSTHOOK: query: DESCRIBE loadpart1 Index: ql/src/test/results/compiler/plan/join2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join2.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join2.q.xml (working copy) @@ -1,10 +1,10 @@ #### A masked pattern was here #### - + - + Index: ql/src/test/results/compiler/plan/input2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input2.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input2.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + @@ -578,7 +578,7 @@ - + @@ -901,7 +901,7 @@ - + @@ -1095,7 +1095,7 @@ - + @@ -1422,7 +1422,7 @@ - + Index: ql/src/test/results/compiler/plan/join3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join3.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join3.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Index: ql/src/test/results/compiler/plan/input3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input3.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input3.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + @@ -578,7 +578,7 @@ - + @@ -901,7 +901,7 @@ - + @@ -1095,7 +1095,7 @@ - + @@ -1422,7 +1422,7 @@ - + @@ -1593,7 +1593,7 @@ - + @@ -1830,7 +1830,7 @@ - + Index: ql/src/test/results/compiler/plan/join4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join4.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join4.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input4.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input4.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Index: ql/src/test/results/compiler/plan/join5.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join5.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join5.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input5.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input5.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input5.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Index: ql/src/test/results/compiler/plan/join6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join6.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join6.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input_testxpath2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input_testxpath2.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input_testxpath2.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input6.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input6.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/join7.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join7.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join7.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input7.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input7.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input7.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/input_testsequencefile.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input_testsequencefile.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input_testsequencefile.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/input8.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input8.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input8.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/join8.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join8.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join8.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/union.q.xml =================================================================== --- ql/src/test/results/compiler/plan/union.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/union.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -25,7 +25,7 @@ - + @@ -279,7 +279,7 @@ - + Index: ql/src/test/results/compiler/plan/input9.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input9.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input9.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/udf1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/udf1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/udf1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/udf4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/udf4.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/udf4.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input_testxpath.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input_testxpath.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input_testxpath.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/udf6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/udf6.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/udf6.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input_part1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input_part1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input_part1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/groupby1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Index: ql/src/test/results/compiler/plan/udf_case.q.xml =================================================================== --- ql/src/test/results/compiler/plan/udf_case.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/udf_case.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/groupby2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby2.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby2.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/subq.q.xml =================================================================== --- ql/src/test/results/compiler/plan/subq.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/subq.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -25,7 +25,7 @@ - + @@ -279,7 +279,7 @@ - + Index: ql/src/test/results/compiler/plan/groupby3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby3.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby3.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/groupby4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby4.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby4.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/groupby5.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby5.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby5.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/groupby6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/groupby6.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/groupby6.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/case_sensitivity.q.xml =================================================================== --- ql/src/test/results/compiler/plan/case_sensitivity.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/case_sensitivity.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/udf_when.q.xml =================================================================== --- ql/src/test/results/compiler/plan/udf_when.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/udf_when.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/input20.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input20.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input20.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/sample1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/sample2.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample2.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample2.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/sample3.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample3.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample3.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/sample4.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample4.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample4.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/sample5.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample5.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample5.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/sample6.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample6.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample6.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/sample7.q.xml =================================================================== --- ql/src/test/results/compiler/plan/sample7.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/sample7.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/results/compiler/plan/cast1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/cast1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/cast1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Stage-1 Index: ql/src/test/results/compiler/plan/join1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/join1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/join1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + Index: ql/src/test/results/compiler/plan/input1.q.xml =================================================================== --- ql/src/test/results/compiler/plan/input1.q.xml (revision 1500544) +++ ql/src/test/results/compiler/plan/input1.q.xml (working copy) @@ -1,6 +1,6 @@ #### A masked pattern was here #### - + @@ -57,7 +57,7 @@ - + @@ -384,7 +384,7 @@ - + Index: ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java =================================================================== --- ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (revision 1500544) +++ ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java (working copy) @@ -31,6 +31,8 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Table; Index: ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java =================================================================== --- ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java (revision 1500544) +++ ql/src/test/org/apache/hadoop/hive/ql/io/TestSymlinkTextInputFormat.java (working copy) @@ -35,9 +35,9 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.exec.ExecDriver; -import org.apache.hadoop.hive.ql.exec.MapRedTask; 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.session.SessionState; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java (working copy) @@ -33,7 +33,6 @@ 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.ExecDriver; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; @@ -43,6 +42,7 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.Partition; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java (working copy) @@ -34,12 +34,12 @@ import org.apache.hadoop.hive.ql.exec.ConditionalTask; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.Operator; 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.mr.MapRedTask; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils; import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingInferenceOptimizer.java (working copy) @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; -import org.apache.hadoop.hive.ql.exec.ExecDriver; import org.apache.hadoop.hive.ql.exec.ExtractOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.FilterOperator; @@ -36,6 +35,7 @@ import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.SelectOperator; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.lib.GraphWalker; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/AbstractJoinTaskDispatcher.java (working copy) @@ -27,9 +27,9 @@ import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.lib.TaskGraphWalker.TaskGraphWalkerContext; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SamplingOptimizer.java (working copy) @@ -20,12 +20,12 @@ package org.apache.hadoop.hive.ql.optimizer.physical; -import org.apache.hadoop.hive.ql.exec.MapRedTask; 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.Task; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.MapredWork; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/MapJoinResolver.java (working copy) @@ -28,12 +28,12 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.MapredLocalTask; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; import org.apache.hadoop.hive.ql.exec.Operator; 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.mr.MapredLocalTask; import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; import org.apache.hadoop.hive.ql.lib.Dispatcher; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SortMergeJoinTaskDispatcher.java (working copy) @@ -35,13 +35,13 @@ import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; -import org.apache.hadoop.hive.ql.exec.MapRedTask; 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.Task; import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.lib.Dispatcher; import org.apache.hadoop.hive.ql.optimizer.MapJoinProcessor; import org.apache.hadoop.hive.ql.parse.OpParseContext; Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/index/IndexWhereProcessor.java (working copy) @@ -31,10 +31,10 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.hive.metastore.api.Index; -import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.index.HiveIndexHandler; import org.apache.hadoop.hive.ql.index.HiveIndexQueryContext; Index: ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/QueryPlan.java (working copy) @@ -38,10 +38,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.ExecDriver; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.hooks.LineageInfo; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; Index: ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/IndexUpdater.java (working copy) @@ -40,7 +40,6 @@ import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.hive.metastore.api.Index; import org.apache.hadoop.hive.ql.exec.FilterOperator; -import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.Utilities; Index: ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/MapReduceCompiler.java (working copy) @@ -50,8 +50,8 @@ import org.apache.hadoop.hive.ql.exec.TaskFactory; import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.ql.exec.ExecDriver; -import org.apache.hadoop.hive.ql.exec.MapRedTask; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; Index: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (working copy) @@ -53,13 +53,11 @@ import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator; import org.apache.hadoop.hive.ql.exec.ArchiveUtils; import org.apache.hadoop.hive.ql.exec.ColumnInfo; -import org.apache.hadoop.hive.ql.exec.ExecDriver; import org.apache.hadoop.hive.ql.exec.FileSinkOperator; import org.apache.hadoop.hive.ql.exec.FunctionInfo; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.GroupByOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; -import org.apache.hadoop.hive.ql.exec.MapRedTask; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.OperatorFactory; import org.apache.hadoop.hive.ql.exec.RecordReader; @@ -74,6 +72,8 @@ import org.apache.hadoop.hive.ql.exec.UnionOperator; import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.WindowFunctionInfo; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.hooks.ReadEntity; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java (working copy) @@ -74,7 +74,7 @@ super.initialize(conf, queryPlan, ctx); work.initializeForFetch(); try { - JobConf job = new JobConf(conf, ExecDriver.class); + JobConf job = new JobConf(conf); ftOp = new FetchOperator(work.getfWork(), job); } catch (Exception e) { LOG.error(StringUtils.stringifyException(e)); Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java (working copy) @@ -65,7 +65,7 @@ try { // Create a file system handle - JobConf job = new JobConf(conf, ExecDriver.class); + JobConf job = new JobConf(conf); Operator source = work.getSource(); if (source instanceof TableScanOperator) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredContext.java (working copy) @@ -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; - -import java.io.Closeable; -import java.io.IOException; -import java.lang.reflect.Method; -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; - -/** - * Runtime context of MapredTask providing additional information to GenericUDF - */ -public class MapredContext { - - private static final Log logger = LogFactory.getLog("MapredContext"); - private static final ThreadLocal contexts = new ThreadLocal(); - - static MapredContext get() { - return contexts.get(); - } - - static MapredContext init(boolean isMap, JobConf jobConf) { - MapredContext context = new MapredContext(isMap, jobConf); - contexts.set(context); - return context; - } - - static void close() { - MapredContext context = contexts.get(); - if (context != null) { - context.closeAll(); - } - contexts.remove(); - } - - private final boolean isMap; - private final JobConf jobConf; - private final List udfs; - - private Reporter reporter; - - private MapredContext(boolean isMap, JobConf jobConf) { - this.isMap = isMap; - this.jobConf = jobConf; - this.udfs = new ArrayList(); - } - - /** - * Returns whether the UDF is called from Map or Reduce task. - */ - public boolean isMap() { - return isMap; - } - - /** - * Returns Reporter, which is set right before reading the first row. - */ - public Reporter getReporter() { - return reporter; - } - - /** - * Returns JobConf. - */ - public JobConf getJobConf() { - return jobConf; - } - - void setReporter(Reporter reporter) { - this.reporter = reporter; - } - - private void registerCloseable(Closeable closeable) { - udfs.add(closeable); - } - - private void closeAll() { - for (Closeable eval : udfs) { - try { - eval.close(); - } catch (IOException e) { - logger.info("Hit error while closing udf " + eval); - } - } - udfs.clear(); - } - - void setup(GenericUDF genericUDF) { - if (needConfigure(genericUDF)) { - genericUDF.configure(this); - } - if (needClose(genericUDF)) { - registerCloseable(genericUDF); - } - } - - void setup(GenericUDAFEvaluator genericUDAF) { - if (needConfigure(genericUDAF)) { - genericUDAF.configure(this); - } - if (needClose(genericUDAF)) { - registerCloseable(genericUDAF); - } - } - - void setup(GenericUDTF genericUDTF) { - if (needConfigure(genericUDTF)) { - genericUDTF.configure(this); - } - // close is called by UDTFOperator - } - - private boolean needConfigure(Object func) { - try { - Method initMethod = func.getClass().getMethod("configure", MapredContext.class); - return initMethod.getDeclaringClass() != GenericUDF.class && - initMethod.getDeclaringClass() != GenericUDAFEvaluator.class && - initMethod.getDeclaringClass() != GenericUDTF.class; - } catch (Exception e) { - return false; - } - } - - private boolean needClose(Closeable func) { - try { - Method closeMethod = func.getClass().getMethod("close"); - return closeMethod.getDeclaringClass() != GenericUDF.class && - closeMethod.getDeclaringClass() != GenericUDAFEvaluator.class; - } catch (Exception e) { - return false; - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/UDFContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/UDFContext.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/UDFContext.java (working copy) @@ -29,77 +29,44 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Reporter; /** * Runtime context of MapredTask providing additional information to GenericUDF */ -public class MapredContext { +public class UDFContext { - private static final Log logger = LogFactory.getLog("MapredContext"); - private static final ThreadLocal contexts = new ThreadLocal(); + private static final Log logger = LogFactory.getLog(UDFContext.class); + private static final ThreadLocal contexts = new ThreadLocal(); - static MapredContext get() { + public static UDFContext get() { return contexts.get(); } - static MapredContext init(boolean isMap, JobConf jobConf) { - MapredContext context = new MapredContext(isMap, jobConf); + public static UDFContext init() { + UDFContext context = new UDFContext(); contexts.set(context); return context; } - static void close() { - MapredContext context = contexts.get(); + public static void close() { + UDFContext context = contexts.get(); if (context != null) { context.closeAll(); } contexts.remove(); } - private final boolean isMap; - private final JobConf jobConf; private final List udfs; - private Reporter reporter; - - private MapredContext(boolean isMap, JobConf jobConf) { - this.isMap = isMap; - this.jobConf = jobConf; + private UDFContext() { this.udfs = new ArrayList(); } - /** - * Returns whether the UDF is called from Map or Reduce task. - */ - public boolean isMap() { - return isMap; - } - - /** - * Returns Reporter, which is set right before reading the first row. - */ - public Reporter getReporter() { - return reporter; - } - - /** - * Returns JobConf. - */ - public JobConf getJobConf() { - return jobConf; - } - - void setReporter(Reporter reporter) { - this.reporter = reporter; - } - private void registerCloseable(Closeable closeable) { udfs.add(closeable); } - private void closeAll() { + public void closeAll() { for (Closeable eval : udfs) { try { eval.close(); @@ -110,7 +77,7 @@ udfs.clear(); } - void setup(GenericUDF genericUDF) { + public void setup(GenericUDF genericUDF) { if (needConfigure(genericUDF)) { genericUDF.configure(this); } @@ -119,7 +86,7 @@ } } - void setup(GenericUDAFEvaluator genericUDAF) { + public void setup(GenericUDAFEvaluator genericUDAF) { if (needConfigure(genericUDAF)) { genericUDAF.configure(this); } @@ -128,7 +95,7 @@ } } - void setup(GenericUDTF genericUDTF) { + public void setup(GenericUDTF genericUDTF) { if (needConfigure(genericUDTF)) { genericUDTF.configure(this); } @@ -137,7 +104,7 @@ private boolean needConfigure(Object func) { try { - Method initMethod = func.getClass().getMethod("configure", MapredContext.class); + Method initMethod = func.getClass().getMethod("configure", UDFContext.class); return initMethod.getDeclaringClass() != GenericUDF.class && initMethod.getDeclaringClass() != GenericUDAFEvaluator.class && initMethod.getDeclaringClass() != GenericUDTF.class; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java (working copy) @@ -33,6 +33,7 @@ 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.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; import org.apache.hadoop.hive.ql.plan.MapredWork; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JobTrackerURLResolver.java (working copy) @@ -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; - -import java.io.IOException; -import java.net.InetSocketAddress; - -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.net.NetUtils; - -/** - * JobTrackerURLResolver. - * - */ -public final class JobTrackerURLResolver { - public static String getURL(JobConf conf) throws IOException { - String infoAddr = ShimLoader.getHadoopShims().getJobLauncherHttpAddress(conf); - if (infoAddr == null) { - throw new IOException("Unable to find job tracker info port."); - } - InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr); - int infoPort = infoSocAddr.getPort(); - - String jobTrackerStr = - ShimLoader.getHadoopShims().getJobLauncherRpcAddress(conf); - InetSocketAddress jobTrackerSocAddr = - NetUtils.createSocketAddr(jobTrackerStr); - - String tracker = "http://" + jobTrackerSocAddr.getHostName() - + ":" + infoPort; - - return tracker; - } - - private JobTrackerURLResolver() { - // prevent instantiation - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java (working copy) @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.ql.CommandNeedRetryException; -import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.lib.Node; @@ -50,6 +49,8 @@ public abstract class Task implements Serializable, Node { private static final long serialVersionUID = 1L; + public transient HashMap taskCounters; + public transient TaskHandle taskHandle; protected transient boolean started; protected transient boolean initialized; protected transient boolean isdone; @@ -58,8 +59,6 @@ protected transient Hive db; protected transient LogHelper console; protected transient QueryPlan queryPlan; - protected transient TaskHandle taskHandle; - protected transient HashMap taskCounters; protected transient DriverContext driverContext; protected transient boolean clonedConf = false; protected transient String jobID; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JobDebugger.java (working copy) @@ -1,336 +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.IOException; -import java.lang.Exception; -import java.net.MalformedURLException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.exec.errors.ErrorAndSolution; -import org.apache.hadoop.hive.ql.exec.errors.TaskLogProcessor; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RunningJob; -import org.apache.hadoop.mapred.TaskCompletionEvent; - -/** - * JobDebugger takes a RunningJob that has failed and grabs the top 4 failing - * tasks and outputs this information to the Hive CLI. - */ -public class JobDebugger implements Runnable { - private final JobConf conf; - private final RunningJob rj; - private final LogHelper console; - private final Map>> stackTraces; - // Mapping from task ID to the number of failures - private final Map failures = new HashMap(); - private final Set successes = new HashSet(); // Successful task ID's - private final Map taskIdToInfo = new HashMap(); - private int maxFailures = 0; - - // Used for showJobFailDebugInfo - private static class TaskInfo { - String jobId; - Set logUrls; - int errorCode; // Obtained from the HiveException thrown - String[] diagnosticMesgs; - - public TaskInfo(String jobId) { - this.jobId = jobId; - logUrls = new HashSet(); - errorCode = 0; - diagnosticMesgs = null; - } - - public void addLogUrl(String logUrl) { - logUrls.add(logUrl); - } - - public void setErrorCode(int errorCode) { - this.errorCode = errorCode; - } - - public void setDiagnosticMesgs(String[] diagnosticMesgs) { - this.diagnosticMesgs = diagnosticMesgs; - } - - public Set getLogUrls() { - return logUrls; - } - - public String getJobId() { - return jobId; - } - - public int getErrorCode() { - return errorCode; - } - - public String[] getDiagnosticMesgs() { - return diagnosticMesgs; - } - } - - public JobDebugger(JobConf conf, RunningJob rj, LogHelper console) { - this.conf = conf; - this.rj = rj; - this.console = console; - this.stackTraces = null; - } - - public JobDebugger(JobConf conf, RunningJob rj, LogHelper console, - Map>> stackTraces) { - this.conf = conf; - this.rj = rj; - this.console = console; - this.stackTraces = stackTraces; - } - - public void run() { - try { - showJobFailDebugInfo(); - } catch (IOException e) { - console.printError(e.getMessage()); - } - } - - public static int extractErrorCode(String[] diagnostics) { - int result = 0; - Pattern errorCodeRegex = ErrorMsg.getErrorCodePattern(); - for (String mesg : diagnostics) { - Matcher matcher = errorCodeRegex.matcher(mesg); - if (matcher.find()) { - result = Integer.parseInt(matcher.group(1)); - // We don't exit the loop early because we want to extract the error code - // corresponding to the bottommost error coded exception. - } - } - return result; - } - - class TaskInfoGrabber implements Runnable { - - public void run() { - try { - getTaskInfos(); - } catch (Exception e) { - console.printError(e.getMessage()); - } - } - - private void getTaskInfos() throws IOException, MalformedURLException { - int startIndex = 0; - while (true) { - TaskCompletionEvent[] taskCompletions = rj.getTaskCompletionEvents(startIndex); - - if (taskCompletions == null || taskCompletions.length == 0) { - break; - } - - boolean more = true; - boolean firstError = true; - for (TaskCompletionEvent t : taskCompletions) { - // getTaskJobIDs returns Strings for compatibility with Hadoop versions - // without TaskID or TaskAttemptID - String[] taskJobIds = ShimLoader.getHadoopShims().getTaskJobIDs(t); - - if (taskJobIds == null) { - console.printError("Task attempt info is unavailable in this Hadoop version"); - more = false; - break; - } - - // For each task completion event, get the associated task id, job id - // and the logs - String taskId = taskJobIds[0]; - String jobId = taskJobIds[1]; - if (firstError) { - console.printError("Examining task ID: " + taskId + " (and more) from job " + jobId); - firstError = false; - } - - TaskInfo ti = taskIdToInfo.get(taskId); - if (ti == null) { - ti = new TaskInfo(jobId); - taskIdToInfo.put(taskId, ti); - } - // These tasks should have come from the same job. - assert (ti.getJobId() != null && ti.getJobId().equals(jobId)); - String taskAttemptLogUrl = ShimLoader.getHadoopShims().getTaskAttemptLogUrl( - conf, t.getTaskTrackerHttp(), t.getTaskId()); - if (taskAttemptLogUrl != null) { - ti.getLogUrls().add(taskAttemptLogUrl); - } - - // If a task failed, fetch its error code (if available). - // Also keep track of the total number of failures for that - // task (typically, a task gets re-run up to 4 times if it fails. - if (t.getTaskStatus() != TaskCompletionEvent.Status.SUCCEEDED) { - String[] diags = rj.getTaskDiagnostics(t.getTaskAttemptId()); - ti.setDiagnosticMesgs(diags); - if (ti.getErrorCode() == 0) { - ti.setErrorCode(extractErrorCode(diags)); - } - - Integer failAttempts = failures.get(taskId); - if (failAttempts == null) { - failAttempts = Integer.valueOf(0); - } - failAttempts = Integer.valueOf(failAttempts.intValue() + 1); - failures.put(taskId, failAttempts); - } else { - successes.add(taskId); - } - } - if (!more) { - break; - } - startIndex += taskCompletions.length; - } - } - } - - private void computeMaxFailures() { - maxFailures = 0; - for (Integer failCount : failures.values()) { - if (maxFailures < failCount.intValue()) { - maxFailures = failCount.intValue(); - } - } - } - - @SuppressWarnings("deprecation") - private void showJobFailDebugInfo() throws IOException { - console.printError("Error during job, obtaining debugging information..."); - if (!conf.get("mapred.job.tracker", "local").equals("local")) { - // Show Tracking URL for remotely running jobs. - console.printError("Job Tracking URL: " + rj.getTrackingURL()); - } - // Loop to get all task completion events because getTaskCompletionEvents - // only returns a subset per call - TaskInfoGrabber tlg = new TaskInfoGrabber(); - Thread t = new Thread(tlg); - try { - t.start(); - t.join(HiveConf.getIntVar(conf, HiveConf.ConfVars.TASKLOG_DEBUG_TIMEOUT)); - } catch (InterruptedException e) { - console.printError("Timed out trying to finish grabbing task log URLs, " - + "some task info may be missing"); - } - - // Remove failures for tasks that succeeded - for (String task : successes) { - failures.remove(task); - } - - if (failures.keySet().size() == 0) { - return; - } - // Find the highest failure count - computeMaxFailures() ; - - // Display Error Message for tasks with the highest failure count - String jtUrl = null; - try { - jtUrl = JobTrackerURLResolver.getURL(conf); - } catch (Exception e) { - console.printError("Unable to retrieve URL for Hadoop Task logs. " - + e.getMessage()); - } - - for (String task : failures.keySet()) { - if (failures.get(task).intValue() == maxFailures) { - TaskInfo ti = taskIdToInfo.get(task); - String jobId = ti.getJobId(); - String taskUrl = (jtUrl == null) ? null : - jtUrl + "/taskdetails.jsp?jobid=" + jobId + "&tipid=" + task.toString(); - - TaskLogProcessor tlp = new TaskLogProcessor(conf); - for (String logUrl : ti.getLogUrls()) { - tlp.addTaskAttemptLogUrl(logUrl); - } - - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES) && - stackTraces != null) { - if (!stackTraces.containsKey(jobId)) { - stackTraces.put(jobId, new ArrayList>()); - } - stackTraces.get(jobId).addAll(tlp.getStackTraces()); - } - - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.SHOW_JOB_FAIL_DEBUG_INFO)) { - List errors = tlp.getErrors(); - - StringBuilder sb = new StringBuilder(); - // We use a StringBuilder and then call printError only once as - // printError will write to both stderr and the error log file. In - // situations where both the stderr and the log file output is - // simultaneously output to a single stream, this will look cleaner. - sb.append("\n"); - sb.append("Task with the most failures(" + maxFailures + "): \n"); - sb.append("-----\n"); - sb.append("Task ID:\n " + task + "\n\n"); - if (taskUrl != null) { - sb.append("URL:\n " + taskUrl + "\n"); - } - - for (ErrorAndSolution e : errors) { - sb.append("\n"); - sb.append("Possible error:\n " + e.getError() + "\n\n"); - sb.append("Solution:\n " + e.getSolution() + "\n"); - } - sb.append("-----\n"); - - sb.append("Diagnostic Messages for this Task:\n"); - String[] diagMesgs = ti.getDiagnosticMesgs(); - for (String mesg : diagMesgs) { - sb.append(mesg + "\n"); - } - console.printError(sb.toString()); - } - - // Only print out one task because that's good enough for debugging. - break; - } - } - return; - } - - public int getErrorCode() { - for (String task : failures.keySet()) { - if (failures.get(task).intValue() == maxFailures) { - TaskInfo ti = taskIdToInfo.get(task); - return ti.getErrorCode(); - } - } - // Should never reach here unless there were no failed tasks. - return 0; - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (working copy) @@ -338,7 +338,7 @@ jc = (JobConf) hconf; } else { // test code path - jc = new JobConf(hconf, ExecDriver.class); + jc = new JobConf(hconf); } if (multiFileSpray) { @@ -808,7 +808,7 @@ private String lsDir() { String specPath = conf.getDirName(); // need to get a JobConf here because it's not passed through at client side - JobConf jobConf = new JobConf(ExecDriver.class); + JobConf jobConf = new JobConf(); Path tmpPath = Utilities.toTempPath(specPath); StringBuilder sb = new StringBuilder("\n"); try { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExprNodeGenericFuncEvaluator.java (working copy) @@ -120,7 +120,7 @@ for (int i = 0; i < children.length; i++) { childrenOIs[i] = children[i].initialize(rowInspector); } - MapredContext context = MapredContext.get(); + UDFContext context = UDFContext.get(); if (context != null) { context.setup(genericUDF); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapper.java (working copy) @@ -1,270 +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.IOException; -import java.lang.management.ManagementFactory; -import java.lang.management.MemoryMXBean; -import java.net.URLClassLoader; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.MapredWork; -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.MapReduceBase; -import org.apache.hadoop.mapred.Mapper; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.util.StringUtils; -/** - * ExecMapper. - * - */ -public class ExecMapper extends MapReduceBase implements Mapper { - - private MapOperator mo; - private Map fetchOperators; - private OutputCollector oc; - private JobConf jc; - private boolean abort = false; - private Reporter rp; - public static final Log l4j = LogFactory.getLog("ExecMapper"); - private static boolean done; - - // used to log memory usage periodically - public static MemoryMXBean memoryMXBean; - private long numRows = 0; - private long nextCntr = 1; - private MapredLocalWork localWork = null; - private boolean isLogInfoEnabled = false; - - private final ExecMapperContext execContext = new ExecMapperContext(); - - @Override - public void configure(JobConf job) { - // Allocate the bean at the beginning - - memoryMXBean = ManagementFactory.getMemoryMXBean(); - l4j.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax()); - - isLogInfoEnabled = l4j.isInfoEnabled(); - - try { - l4j.info("conf classpath = " - + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs())); - l4j.info("thread classpath = " - + Arrays.asList(((URLClassLoader) Thread.currentThread() - .getContextClassLoader()).getURLs())); - } catch (Exception e) { - l4j.info("cannot get classpath: " + e.getMessage()); - } - try { - jc = job; - execContext.setJc(jc); - // create map and fetch operators - MapredWork mrwork = Utilities.getMapRedWork(job); - mo = new MapOperator(); - mo.setConf(mrwork); - // initialize map operator - mo.setChildren(job); - l4j.info(mo.dump(0)); - // initialize map local work - localWork = mrwork.getMapLocalWork(); - execContext.setLocalWork(localWork); - - MapredContext.init(true, new JobConf(jc)); - - mo.setExecContext(execContext); - mo.initializeLocalWork(jc); - mo.initialize(jc, null); - - if (localWork == null) { - return; - } - - //The following code is for mapjoin - //initialize all the dummy ops - l4j.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); - } - } - } - - public void map(Object key, Object value, OutputCollector output, - Reporter reporter) throws IOException { - if (oc == null) { - oc = output; - rp = reporter; - mo.setOutputCollector(oc); - mo.setReporter(rp); - MapredContext.get().setReporter(reporter); - } - // reset the execContext for each new row - execContext.resetRow(); - - try { - if (mo.getDone()) { - done = true; - } else { - // Since there is no concept of a group, we don't invoke - // startGroup/endGroup for a mapper - mo.process((Writable)value); - if (isLogInfoEnabled) { - numRows++; - if (numRows == nextCntr) { - long used_memory = memoryMXBean.getHeapMemoryUsage().getUsed(); - l4j.info("ExecMapper: processing " + numRows - + " rows: used memory = " + used_memory); - nextCntr = getNextCntr(numRows); - } - } - } - } 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 { - l4j.fatal(StringUtils.stringifyException(e)); - throw new RuntimeException(e); - } - } - } - - - private long getNextCntr(long cntr) { - // 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 (cntr >= 1000000) { - return cntr + 1000000; - } - - return 10 * cntr; - } - - @Override - public void close() { - // No row was processed - if (oc == null) { - l4j.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 (fetchOperators != null) { - MapredLocalWork localWork = mo.getConf().getMapLocalWork(); - for (Map.Entry entry : fetchOperators.entrySet()) { - Operator forwardOp = localWork - .getAliasToWork().get(entry.getKey()); - forwardOp.close(abort); - } - } - - if (isLogInfoEnabled) { - long used_memory = memoryMXBean.getHeapMemoryUsage().getUsed(); - l4j.info("ExecMapper: processed " + numRows + " rows: used memory = " - + used_memory); - } - - reportStats rps = new reportStats(rp); - mo.preorderMap(rps); - return; - } catch (Exception e) { - if (!abort) { - // signal new failure to map-reduce - l4j.error("Hit error while closing operators - failing tree"); - throw new RuntimeException("Hive Runtime Error while closing operators", e); - } - } finally { - MapredContext.close(); - } - } - - public static boolean getDone() { - return done; - } - - public boolean isAbort() { - return abort; - } - - public void setAbort(boolean abort) { - this.abort = abort; - } - - public static void setDone(boolean done) { - ExecMapper.done = done; - } - - /** - * reportStats. - * - */ - public static class reportStats implements Operator.OperatorFunc { - Reporter rp; - - public reportStats(Reporter rp) { - this.rp = rp; - } - - public void func(Operator op) { - Map, Long> opStats = op.getStats(); - for (Map.Entry, Long> e : opStats.entrySet()) { - if (rp != null) { - rp.incrCounter(e.getKey(), e.getValue()); - } - } - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java (working copy) @@ -23,6 +23,8 @@ import java.util.List; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; +import org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask; import org.apache.hadoop.hive.ql.io.rcfile.merge.BlockMergeTask; import org.apache.hadoop.hive.ql.io.rcfile.merge.MergeWork; import org.apache.hadoop.hive.ql.io.rcfile.stats.PartialScanTask; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapRedTask.java (working copy) @@ -1,592 +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.File; -import java.io.IOException; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -import org.apache.commons.lang.StringUtils; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.io.CachingPrintStream; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.MapredWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.mapred.JobConf; -/** - * Extension of ExecDriver: - * - can optionally spawn a map-reduce task from a separate jvm - * - will make last minute adjustments to map-reduce job parameters, viz: - * * estimating number of reducers - * * estimating whether job should run locally - **/ -public class MapRedTask extends ExecDriver implements Serializable { - - private static final long serialVersionUID = 1L; - - static final String HADOOP_MEM_KEY = "HADOOP_HEAPSIZE"; - static final String HADOOP_OPTS_KEY = "HADOOP_OPTS"; - static final String HADOOP_CLIENT_OPTS = "HADOOP_CLIENT_OPTS"; - static final String HIVE_DEBUG_RECURSIVE = "HIVE_DEBUG_RECURSIVE"; - static final String HIVE_MAIN_CLIENT_DEBUG_OPTS = "HIVE_MAIN_CLIENT_DEBUG_OPTS"; - static final String HIVE_CHILD_CLIENT_DEBUG_OPTS = "HIVE_CHILD_CLIENT_DEBUG_OPTS"; - static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive"}; - - private transient ContentSummary inputSummary = null; - private transient boolean runningViaChild = false; - - private transient boolean inputSizeEstimated = false; - private transient long totalInputFileSize; - private transient long totalInputNumFiles; - - private Process executor; - - public MapRedTask() { - super(); - } - - public MapRedTask(MapredWork plan, JobConf job, boolean isSilent) throws HiveException { - throw new RuntimeException("Illegal Constructor call"); - } - - @Override - public int execute(DriverContext driverContext) { - - Context ctx = driverContext.getCtx(); - boolean ctxCreated = false; - - try { - if (ctx == null) { - ctx = new Context(conf); - ctxCreated = true; - } - - // estimate number of reducers - setNumberOfReducers(); - - // auto-determine local mode if allowed - if (!ctx.isLocalOnlyExecutionMode() && - conf.getBoolVar(HiveConf.ConfVars.LOCALMODEAUTO)) { - - if (inputSummary == null) { - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work, null); - } - - // set the values of totalInputFileSize and totalInputNumFiles, estimating them - // if percentage block sampling is being used - estimateInputSize(); - - // at this point the number of reducers is precisely defined in the plan - int numReducers = work.getNumReduceTasks(); - - if (LOG.isDebugEnabled()) { - LOG.debug("Task: " + getId() + ", Summary: " + - totalInputFileSize + "," + totalInputNumFiles + "," - + numReducers); - } - - String reason = MapRedTask.isEligibleForLocalMode(conf, numReducers, - totalInputFileSize, totalInputNumFiles); - if (reason == null) { - // clone configuration before modifying it on per-task basis - cloneConf(); - ShimLoader.getHadoopShims().setJobLauncherRpcAddress(conf, "local"); - console.printInfo("Selecting local mode for task: " + getId()); - this.setLocalMode(true); - } else { - console.printInfo("Cannot run job locally: " + reason); - this.setLocalMode(false); - } - } - - runningViaChild = ShimLoader.getHadoopShims().isLocalMode(conf) || - conf.getBoolVar(HiveConf.ConfVars.SUBMITVIACHILD); - - if(!runningViaChild) { - // we are not running this mapred task via child jvm - // so directly invoke ExecDriver - return super.execute(driverContext); - } - - // we need to edit the configuration to setup cmdline. clone it first - cloneConf(); - - // propagate input format if necessary - super.setInputAttributes(conf); - - // enable assertion - String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN); - String hiveJar = conf.getJar(); - - String libJarsOption; - String addedJars = Utilities.getResourceFiles(conf, SessionState.ResourceType.JAR); - conf.setVar(ConfVars.HIVEADDEDJARS, addedJars); - String auxJars = conf.getAuxJars(); - // Put auxjars and addedjars together into libjars - if (StringUtils.isEmpty(addedJars)) { - if (StringUtils.isEmpty(auxJars)) { - libJarsOption = " "; - } else { - libJarsOption = " -libjars " + auxJars + " "; - } - } else { - if (StringUtils.isEmpty(auxJars)) { - libJarsOption = " -libjars " + addedJars + " "; - } else { - libJarsOption = " -libjars " + addedJars + "," + auxJars + " "; - } - } - - // Generate the hiveConfArgs after potentially adding the jars - String hiveConfArgs = generateCmdLine(conf, ctx); - - // write out the plan to a local file - Path planPath = new Path(ctx.getLocalTmpFileURI(), "plan.xml"); - OutputStream out = FileSystem.getLocal(conf).create(planPath); - MapredWork plan = getWork(); - LOG.info("Generating plan file " + planPath.toString()); - Utilities.serializeMapRedWork(plan, out); - - String isSilent = "true".equalsIgnoreCase(System - .getProperty("test.silent")) ? "-nolog" : ""; - - String jarCmd; - if (ShimLoader.getHadoopShims().usesJobShell()) { - jarCmd = libJarsOption + hiveJar + " " + ExecDriver.class.getName(); - } else { - jarCmd = hiveJar + " " + ExecDriver.class.getName() + libJarsOption; - } - - String cmdLine = hadoopExec + " jar " + jarCmd + " -plan " - + planPath.toString() + " " + isSilent + " " + hiveConfArgs; - - String workDir = (new File(".")).getCanonicalPath(); - String files = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); - if (!files.isEmpty()) { - cmdLine = cmdLine + " -files " + files; - - workDir = (new Path(ctx.getLocalTmpFileURI())).toUri().getPath(); - - if (! (new File(workDir)).mkdir()) { - throw new IOException ("Cannot create tmp working dir: " + workDir); - } - - for (String f: StringUtils.split(files, ',')) { - Path p = new Path(f); - String target = p.toUri().getPath(); - String link = workDir + Path.SEPARATOR + p.getName(); - if (FileUtil.symLink(target, link) != 0) { - throw new IOException ("Cannot link to added file: " + target + " from: " + link); - } - } - } - - LOG.info("Executing: " + cmdLine); - - // Inherit Java system variables - String hadoopOpts; - StringBuilder sb = new StringBuilder(); - Properties p = System.getProperties(); - for (String element : HIVE_SYS_PROP) { - if (p.containsKey(element)) { - sb.append(" -D" + element + "=" + p.getProperty(element)); - } - } - hadoopOpts = sb.toString(); - // Inherit the environment variables - String[] env; - Map variables = new HashMap(System.getenv()); - // The user can specify the hadoop memory - - if (ShimLoader.getHadoopShims().isLocalMode(conf)) { - // if we are running in local mode - then the amount of memory used - // by the child jvm can no longer default to the memory used by the - // parent jvm - int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); - if (hadoopMem == 0) { - // remove env var that would default child jvm to use parent's memory - // as default. child jvm would use default memory for a hadoop client - variables.remove(HADOOP_MEM_KEY); - } else { - // user specified the memory for local mode hadoop run - variables.put(HADOOP_MEM_KEY, String.valueOf(hadoopMem)); - } - } else { - // nothing to do - we are not running in local mode - only submitting - // the job via a child process. in this case it's appropriate that the - // child jvm use the same memory as the parent jvm - } - - if (variables.containsKey(HADOOP_OPTS_KEY)) { - variables.put(HADOOP_OPTS_KEY, variables.get(HADOOP_OPTS_KEY) - + hadoopOpts); - } else { - variables.put(HADOOP_OPTS_KEY, hadoopOpts); - } - - if(variables.containsKey(HIVE_DEBUG_RECURSIVE)) { - configureDebugVariablesForChildJVM(variables); - } - - env = new String[variables.size()]; - int pos = 0; - for (Map.Entry entry : variables.entrySet()) { - String name = entry.getKey(); - String value = entry.getValue(); - env[pos++] = name + "=" + value; - } - // Run ExecDriver in another JVM - executor = Runtime.getRuntime().exec(cmdLine, env, new File(workDir)); - - CachingPrintStream errPrintStream = - new CachingPrintStream(SessionState.getConsole().getChildErrStream()); - - StreamPrinter outPrinter = new StreamPrinter( - executor.getInputStream(), null, - SessionState.getConsole().getChildOutStream()); - StreamPrinter errPrinter = new StreamPrinter( - executor.getErrorStream(), null, - errPrintStream); - - outPrinter.start(); - errPrinter.start(); - - int exitVal = jobExecHelper.progressLocal(executor, getId()); - - if (exitVal != 0) { - LOG.error("Execution failed with exit status: " + exitVal); - if (SessionState.get() != null) { - SessionState.get().addLocalMapRedErrors(getId(), errPrintStream.getOutput()); - } - } else { - LOG.info("Execution completed successfully"); - } - - return exitVal; - } catch (Exception e) { - e.printStackTrace(); - LOG.error("Exception: " + e.getMessage()); - return (1); - } finally { - try { - // creating the context can create a bunch of files. So make - // sure to clear it out - if(ctxCreated) { - ctx.clear(); - } - - } catch (Exception e) { - LOG.error("Exception: " + e.getMessage()); - } - } - } - - static void configureDebugVariablesForChildJVM(Map environmentVariables) { - // this method contains various asserts to warn if environment variables are in a buggy state - assert environmentVariables.containsKey(HADOOP_CLIENT_OPTS) - && environmentVariables.get(HADOOP_CLIENT_OPTS) != null : HADOOP_CLIENT_OPTS - + " environment variable must be set when JVM in debug mode"; - - String hadoopClientOpts = environmentVariables.get(HADOOP_CLIENT_OPTS); - - assert environmentVariables.containsKey(HIVE_MAIN_CLIENT_DEBUG_OPTS) - && environmentVariables.get(HIVE_MAIN_CLIENT_DEBUG_OPTS) != null : HIVE_MAIN_CLIENT_DEBUG_OPTS - + " environment variable must be set when JVM in debug mode"; - - assert hadoopClientOpts.contains(environmentVariables.get(HIVE_MAIN_CLIENT_DEBUG_OPTS)) : HADOOP_CLIENT_OPTS - + " environment variable must contain debugging parameters, when JVM in debugging mode"; - - assert "y".equals(environmentVariables.get(HIVE_DEBUG_RECURSIVE)) - || "n".equals(environmentVariables.get(HIVE_DEBUG_RECURSIVE)) : HIVE_DEBUG_RECURSIVE - + " environment variable must be set to \"y\" or \"n\" when debugging"; - - if (environmentVariables.get(HIVE_DEBUG_RECURSIVE).equals("y")) { - // swap debug options in HADOOP_CLIENT_OPTS to those that the child JVM should have - assert environmentVariables.containsKey(HIVE_CHILD_CLIENT_DEBUG_OPTS) - && environmentVariables.get(HIVE_MAIN_CLIENT_DEBUG_OPTS) != null : HIVE_CHILD_CLIENT_DEBUG_OPTS - + " environment variable must be set when JVM in debug mode"; - String newHadoopClientOpts = hadoopClientOpts.replace( - environmentVariables.get(HIVE_MAIN_CLIENT_DEBUG_OPTS), - environmentVariables.get(HIVE_CHILD_CLIENT_DEBUG_OPTS)); - environmentVariables.put(HADOOP_CLIENT_OPTS, newHadoopClientOpts); - } else { - // remove from HADOOP_CLIENT_OPTS any debug related options - String newHadoopClientOpts = hadoopClientOpts.replace( - environmentVariables.get(HIVE_MAIN_CLIENT_DEBUG_OPTS), "").trim(); - if (newHadoopClientOpts.isEmpty()) { - environmentVariables.remove(HADOOP_CLIENT_OPTS); - } else { - environmentVariables.put(HADOOP_CLIENT_OPTS, newHadoopClientOpts); - } - } - // child JVM won't need to change debug parameters when creating it's own children - environmentVariables.remove(HIVE_DEBUG_RECURSIVE); - } - - @Override - public boolean mapStarted() { - boolean b = super.mapStarted(); - return runningViaChild ? isdone : b; - } - - @Override - public boolean reduceStarted() { - boolean b = super.reduceStarted(); - return runningViaChild ? isdone : b; - } - - @Override - public boolean mapDone() { - boolean b = super.mapDone(); - return runningViaChild ? isdone : b; - } - - @Override - public boolean reduceDone() { - boolean b = super.reduceDone(); - return runningViaChild ? isdone : b; - } - - /** - * Set the number of reducers for the mapred work. - */ - private void setNumberOfReducers() throws IOException { - // this is a temporary hack to fix things that are not fixed in the compiler - Integer numReducersFromWork = work.getNumReduceTasks(); - - if (work.getReducer() == null) { - console - .printInfo("Number of reduce tasks is set to 0 since there's no reduce operator"); - work.setNumReduceTasks(Integer.valueOf(0)); - } else { - if (numReducersFromWork >= 0) { - console.printInfo("Number of reduce tasks determined at compile time: " - + work.getNumReduceTasks()); - } else if (job.getNumReduceTasks() > 0) { - int reducers = job.getNumReduceTasks(); - work.setNumReduceTasks(reducers); - console - .printInfo("Number of reduce tasks not specified. Defaulting to jobconf value of: " - + reducers); - } else { - int reducers = estimateNumberOfReducers(); - work.setNumReduceTasks(reducers); - console - .printInfo("Number of reduce tasks not specified. Estimated from input data size: " - + reducers); - - } - 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 - + "="); - } - } - - /** - * Estimate the number of reducers needed for this job, based on job input, - * and configuration parameters. - * - * The output of this method should only be used if the output of this - * MapRedTask is not being used to populate a bucketed table and the user - * has not specified the number of reducers to use. - * - * @return the number of reducers. - */ - private int estimateNumberOfReducers() throws IOException { - long bytesPerReducer = conf.getLongVar(HiveConf.ConfVars.BYTESPERREDUCER); - int maxReducers = conf.getIntVar(HiveConf.ConfVars.MAXREDUCERS); - - if(inputSummary == null) { - // compute the summary and stash it away - inputSummary = Utilities.getInputSummary(driverContext.getCtx(), work, null); - } - - // if all inputs are sampled, we should shrink the size of reducers accordingly. - estimateInputSize(); - - if (totalInputFileSize != inputSummary.getLength()) { - LOG.info("BytesPerReducer=" + bytesPerReducer + " maxReducers=" - + maxReducers + " estimated totalInputFileSize=" + totalInputFileSize); - } else { - LOG.info("BytesPerReducer=" + bytesPerReducer + " maxReducers=" - + maxReducers + " totalInputFileSize=" + totalInputFileSize); - } - - int reducers = (int) ((totalInputFileSize + bytesPerReducer - 1) / bytesPerReducer); - reducers = Math.max(1, reducers); - reducers = Math.min(maxReducers, reducers); - - // If this map reduce job writes final data to a table and bucketing is being inferred, - // and the user has configured Hive to do this, make sure the number of reducers is a - // power of two - if (conf.getBoolVar(HiveConf.ConfVars.HIVE_INFER_BUCKET_SORT_NUM_BUCKETS_POWER_TWO) && - work.isFinalMapRed() && !work.getBucketedColsByDirectory().isEmpty()) { - - int reducersLog = (int)(Math.log(reducers) / Math.log(2)) + 1; - int reducersPowerTwo = (int)Math.pow(2, reducersLog); - - // If the original number of reducers was a power of two, use that - if (reducersPowerTwo / 2 == reducers) { - return reducers; - } else if (reducersPowerTwo > maxReducers) { - // If the next power of two greater than the original number of reducers is greater - // than the max number of reducers, use the preceding power of two, which is strictly - // less than the original number of reducers and hence the max - reducers = reducersPowerTwo / 2; - } else { - // Otherwise use the smallest power of two greater than the original number of reducers - reducers = reducersPowerTwo; - } - } - - return reducers; - } - - /** - * Sets the values of totalInputFileSize and totalInputNumFiles. If percentage - * block sampling is used, these values are estimates based on the highest - * percentage being used for sampling multiplied by the value obtained from the - * input summary. Otherwise, these values are set to the exact value obtained - * from the input summary. - * - * Once the function completes, inputSizeEstimated is set so that the logic is - * never run more than once. - */ - private void estimateInputSize() { - if (inputSizeEstimated) { - // If we've already run this function, return - return; - } - - // Initialize the values to be those taken from the input summary - totalInputFileSize = inputSummary.getLength(); - totalInputNumFiles = inputSummary.getFileCount(); - - if (work.getNameToSplitSample() == null || work.getNameToSplitSample().isEmpty()) { - // If percentage block sampling wasn't used, we don't need to do any estimation - inputSizeEstimated = true; - return; - } - - // if all inputs are sampled, we should shrink the size of the input accordingly - double highestSamplePercentage = 0; - boolean allSample = false; - for (String alias : work.getAliasToWork().keySet()) { - if (work.getNameToSplitSample().containsKey(alias)) { - allSample = true; - Double rate = work.getNameToSplitSample().get(alias).getPercent(); - if (rate != null && rate > highestSamplePercentage) { - highestSamplePercentage = rate; - } - } else { - allSample = false; - break; - } - } - if (allSample) { - // This is a little bit dangerous if inputs turns out not to be able to be sampled. - // In that case, we significantly underestimate the input. - // It's the same as estimateNumberOfReducers(). It's just our best - // guess and there is no guarantee. - totalInputFileSize = Math.min((long) (totalInputFileSize * highestSamplePercentage / 100D) - , totalInputFileSize); - totalInputNumFiles = Math.min((long) (totalInputNumFiles * highestSamplePercentage / 100D) - , totalInputNumFiles); - } - - inputSizeEstimated = true; - } - - /** - * Find out if a job can be run in local mode based on it's characteristics - * - * @param conf Hive Configuration - * @param numReducers total number of reducers for this job - * @param inputLength the size of the input - * @param inputFileCount the number of files of input - * @return String null if job is eligible for local mode, reason otherwise - */ - public static String isEligibleForLocalMode(HiveConf conf, - int numReducers, - long inputLength, - long inputFileCount) { - - long maxBytes = conf.getLongVar(HiveConf.ConfVars.LOCALMODEMAXBYTES); - long maxInputFiles = conf.getIntVar(HiveConf.ConfVars.LOCALMODEMAXINPUTFILES); - - // check for max input size - if (inputLength > maxBytes) { - return "Input Size (= " + inputLength + ") is larger than " + - HiveConf.ConfVars.LOCALMODEMAXBYTES.varname + " (= " + maxBytes + ")"; - } - - // ideally we would like to do this check based on the number of splits - // in the absence of an easy way to get the number of splits - do this - // based on the total number of files (pessimistically assumming that - // splits are equal to number of files in worst case) - if (inputFileCount > maxInputFiles) { - return "Number of Input Files (= " + inputFileCount + - ") is larger than " + - HiveConf.ConfVars.LOCALMODEMAXINPUTFILES.varname + "(= " + maxInputFiles + ")"; - } - - // since local mode only runs with 1 reducers - make sure that the - // the number of reducers (set by user or inferred) is <=1 - if (numReducers > 1) { - return "Number of reducers (= " + numReducers + ") is more than 1"; - } - - return null; - } - - @Override - public Operator getReducer() { - return getWork().getReducer(); - } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.destroy(); - executor = null; - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java (working copy) @@ -339,7 +339,7 @@ for (ExprNodeEvaluator keyField : keyFields) { objectInspectors.add(null); } - MapredContext context = MapredContext.get(); + UDFContext context = UDFContext.get(); if (context != null) { for (GenericUDAFEvaluator genericUDAFEvaluator : aggregationEvaluators) { context.setup(genericUDAFEvaluator); Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FetchOperator.java (working copy) @@ -35,6 +35,7 @@ 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.ql.exec.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.io.HiveContextAwareRecordReader; import org.apache.hadoop.hive.ql.io.HiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveRecordReader; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapredLocalTask.java (working copy) @@ -1,479 +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.File; -import java.io.IOException; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.management.ManagementFactory; -import java.lang.management.MemoryMXBean; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.io.CachingPrintStream; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.Context; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; -import org.apache.hadoop.hive.ql.exec.persistence.AbstractMapJoinKey; -import org.apache.hadoop.hive.ql.exec.persistence.HashMapWrapper; -import org.apache.hadoop.hive.ql.exec.persistence.MapJoinObjectValue; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext; -import org.apache.hadoop.hive.ql.plan.FetchWork; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -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.hive.serde2.ColumnProjectionUtils; -import org.apache.hadoop.hive.serde2.objectinspector.InspectableObject; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.shims.HadoopShims; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.ReflectionUtils; - -public class MapredLocalTask extends Task implements Serializable { - - private Map fetchOperators; - protected HadoopJobExecHelper jobExecHelper; - private JobConf job; - public static transient final Log l4j = LogFactory.getLog(MapredLocalTask.class); - static final String HADOOP_MEM_KEY = "HADOOP_HEAPSIZE"; - static final String HADOOP_OPTS_KEY = "HADOOP_OPTS"; - static final String[] HIVE_SYS_PROP = {"build.dir", "build.dir.hive"}; - public static MemoryMXBean memoryMXBean; - private static final Log LOG = LogFactory.getLog(MapredLocalTask.class); - - // not sure we need this exec context; but all the operators in the work - // will pass this context throught - private final ExecMapperContext execContext = new ExecMapperContext(); - - private Process executor; - - public MapredLocalTask() { - super(); - } - - public MapredLocalTask(MapredLocalWork plan, JobConf job, boolean isSilent) throws HiveException { - setWork(plan); - this.job = job; - console = new LogHelper(LOG, isSilent); - } - - @Override - public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { - super.initialize(conf, queryPlan, driverContext); - job = new JobConf(conf, ExecDriver.class); - //we don't use the HadoopJobExecHooks for local tasks - this.jobExecHelper = new HadoopJobExecHelper(job, console, this, null); - } - - public static String now() { - Calendar cal = Calendar.getInstance(); - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-mm-dd hh:mm:ss"); - return sdf.format(cal.getTime()); - } - - @Override - public boolean requireLock() { - return true; - } - - @Override - public int execute(DriverContext driverContext) { - try { - // generate the cmd line to run in the child jvm - Context ctx = driverContext.getCtx(); - String hiveJar = conf.getJar(); - - String hadoopExec = conf.getVar(HiveConf.ConfVars.HADOOPBIN); - String libJarsOption; - - // write out the plan to a local file - Path planPath = new Path(ctx.getLocalTmpFileURI(), "plan.xml"); - OutputStream out = FileSystem.getLocal(conf).create(planPath); - MapredLocalWork plan = getWork(); - LOG.info("Generating plan file " + planPath.toString()); - Utilities.serializeMapRedLocalWork(plan, out); - - String isSilent = "true".equalsIgnoreCase(System.getProperty("test.silent")) ? "-nolog" : ""; - - String jarCmd; - - jarCmd = hiveJar + " " + ExecDriver.class.getName(); - String hiveConfArgs = ExecDriver.generateCmdLine(conf, ctx); - String cmdLine = hadoopExec + " jar " + jarCmd + " -localtask -plan " + planPath.toString() - + " " + isSilent + " " + hiveConfArgs; - - String workDir = (new File(".")).getCanonicalPath(); - String files = Utilities.getResourceFiles(conf, SessionState.ResourceType.FILE); - - if (!files.isEmpty()) { - cmdLine = cmdLine + " -files " + files; - - workDir = (new Path(ctx.getLocalTmpFileURI())).toUri().getPath(); - - if (!(new File(workDir)).mkdir()) { - throw new IOException("Cannot create tmp working dir: " + workDir); - } - - for (String f : StringUtils.split(files, ',')) { - Path p = new Path(f); - String target = p.toUri().getPath(); - String link = workDir + Path.SEPARATOR + p.getName(); - if (FileUtil.symLink(target, link) != 0) { - throw new IOException("Cannot link to added file: " + target + " from: " + link); - } - } - } - - LOG.info("Executing: " + cmdLine); - - // Inherit Java system variables - String hadoopOpts; - StringBuilder sb = new StringBuilder(); - Properties p = System.getProperties(); - for (String element : HIVE_SYS_PROP) { - if (p.containsKey(element)) { - sb.append(" -D" + element + "=" + p.getProperty(element)); - } - } - hadoopOpts = sb.toString(); - // Inherit the environment variables - String[] env; - Map variables = new HashMap(System.getenv()); - // The user can specify the hadoop memory - - // if ("local".equals(conf.getVar(HiveConf.ConfVars.HADOOPJT))) { - // if we are running in local mode - then the amount of memory used - // by the child jvm can no longer default to the memory used by the - // parent jvm - // int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); - int hadoopMem = conf.getIntVar(HiveConf.ConfVars.HIVEHADOOPMAXMEM); - if (hadoopMem == 0) { - // remove env var that would default child jvm to use parent's memory - // as default. child jvm would use default memory for a hadoop client - variables.remove(HADOOP_MEM_KEY); - } else { - // user specified the memory for local mode hadoop run - console.printInfo(" set heap size\t" + hadoopMem + "MB"); - variables.put(HADOOP_MEM_KEY, String.valueOf(hadoopMem)); - } - // } else { - // nothing to do - we are not running in local mode - only submitting - // the job via a child process. in this case it's appropriate that the - // child jvm use the same memory as the parent jvm - - // } - - //Set HADOOP_USER_NAME env variable for child process, so that - // it also runs with hadoop permissions for the user the job is running as - // This will be used by hadoop only in unsecure(/non kerberos) mode - HadoopShims shim = ShimLoader.getHadoopShims(); - String endUserName = shim.getShortUserName(shim.getUGIForConf(job)); - console.printInfo("setting HADOOP_USER_NAME\t" + endUserName); - variables.put("HADOOP_USER_NAME", endUserName); - - if (variables.containsKey(HADOOP_OPTS_KEY)) { - variables.put(HADOOP_OPTS_KEY, variables.get(HADOOP_OPTS_KEY) + hadoopOpts); - } else { - variables.put(HADOOP_OPTS_KEY, hadoopOpts); - } - - if(variables.containsKey(MapRedTask.HIVE_DEBUG_RECURSIVE)) { - MapRedTask.configureDebugVariablesForChildJVM(variables); - } - - env = new String[variables.size()]; - int pos = 0; - for (Map.Entry entry : variables.entrySet()) { - String name = entry.getKey(); - String value = entry.getValue(); - env[pos++] = name + "=" + value; - } - - // Run ExecDriver in another JVM - executor = Runtime.getRuntime().exec(cmdLine, env, new File(workDir)); - - CachingPrintStream errPrintStream = new CachingPrintStream(System.err); - - StreamPrinter outPrinter = new StreamPrinter(executor.getInputStream(), null, System.out); - StreamPrinter errPrinter = new StreamPrinter(executor.getErrorStream(), null, errPrintStream); - - outPrinter.start(); - errPrinter.start(); - - int exitVal = jobExecHelper.progressLocal(executor, getId()); - - if (exitVal != 0) { - LOG.error("Execution failed with exit status: " + exitVal); - if (SessionState.get() != null) { - SessionState.get().addLocalMapRedErrors(getId(), errPrintStream.getOutput()); - } - } else { - LOG.info("Execution completed successfully"); - console.printInfo("Mapred Local Task Succeeded . Convert the Join into MapJoin"); - } - - return exitVal; - } catch (Exception e) { - e.printStackTrace(); - LOG.error("Exception: " + e.getMessage()); - return (1); - } - } - - - - public int executeFromChildJVM(DriverContext driverContext) { - // check the local work - if (work == null) { - return -1; - } - memoryMXBean = ManagementFactory.getMemoryMXBean(); - long startTime = System.currentTimeMillis(); - console.printInfo(Utilities.now() - + "\tStarting to launch local task to process map join;\tmaximum memory = " - + memoryMXBean.getHeapMemoryUsage().getMax()); - fetchOperators = new HashMap(); - Map fetchOpJobConfMap = new HashMap(); - execContext.setJc(job); - // set the local work, so all the operator can get this context - execContext.setLocalWork(work); - boolean inputFileChangeSenstive = work.getInputFileChangeSensitive(); - try { - - initializeOperators(fetchOpJobConfMap); - // for each big table's bucket, call the start forward - if (inputFileChangeSenstive) { - for (Map> bigTableBucketFiles : work - .getBucketMapjoinContext().getAliasBucketFileNameMapping().values()) { - for (String bigTableBucket : bigTableBucketFiles.keySet()) { - startForward(inputFileChangeSenstive, bigTableBucket); - } - } - } else { - startForward(inputFileChangeSenstive, null); - } - long currentTime = System.currentTimeMillis(); - long elapsed = currentTime - startTime; - console.printInfo(Utilities.now() + "\tEnd of local task; Time Taken: " - + Utilities.showTime(elapsed) + " sec."); - } catch (Throwable e) { - if (e instanceof OutOfMemoryError - || (e instanceof HiveException && e.getMessage().equals("RunOutOfMeomoryUsage"))) { - // Don't create a new object if we are already out of memory - return 3; - } else { - l4j.error("Hive Runtime Error: Map local work failed"); - e.printStackTrace(); - return 2; - } - } - return 0; - } - - private void startForward(boolean inputFileChangeSenstive, String bigTableBucket) - throws Exception { - for (Map.Entry entry : fetchOperators.entrySet()) { - int fetchOpRows = 0; - String alias = entry.getKey(); - FetchOperator fetchOp = entry.getValue(); - - if (inputFileChangeSenstive) { - fetchOp.clearFetchContext(); - setUpFetchOpContext(fetchOp, alias, bigTableBucket); - } - - if (fetchOp.isEmptyTable()) { - //generate empty hashtable for empty table - this.generateDummyHashTable(alias, bigTableBucket); - continue; - } - - // get the root operator - Operator forwardOp = work.getAliasToWork().get(alias); - // walk through the operator tree - while (true) { - InspectableObject row = fetchOp.getNextRow(); - if (row == null) { - if (inputFileChangeSenstive) { - execContext.setCurrentBigBucketFile(bigTableBucket); - forwardOp.reset(); - } - forwardOp.close(false); - break; - } - fetchOpRows++; - forwardOp.process(row.o, 0); - // check if any operator had a fatal error or early exit during - // execution - if (forwardOp.getDone()) { - // ExecMapper.setDone(true); - break; - } - } - } - } - - private void initializeOperators(Map fetchOpJobConfMap) - throws HiveException { - // this mapper operator is used to initialize all the operators - for (Map.Entry entry : work.getAliasToFetchWork().entrySet()) { - JobConf jobClone = new JobConf(job); - - Operator tableScan = - work.getAliasToWork().get(entry.getKey()); - boolean setColumnsNeeded = false; - if (tableScan instanceof TableScanOperator) { - ArrayList list = ((TableScanOperator) tableScan).getNeededColumnIDs(); - if (list != null) { - ColumnProjectionUtils.appendReadColumnIDs(jobClone, list); - setColumnsNeeded = true; - } - } - - if (!setColumnsNeeded) { - ColumnProjectionUtils.setFullyReadColumns(jobClone); - } - - // create a fetch operator - FetchOperator fetchOp = new FetchOperator(entry.getValue(), jobClone); - fetchOpJobConfMap.put(fetchOp, jobClone); - fetchOperators.put(entry.getKey(), fetchOp); - l4j.info("fetchoperator for " + entry.getKey() + " created"); - } - // initilize all forward operator - for (Map.Entry entry : fetchOperators.entrySet()) { - // get the forward op - String alias = entry.getKey(); - Operator forwardOp = work.getAliasToWork().get(alias); - - // put the exe context into all the operators - forwardOp.setExecContext(execContext); - // All the operators need to be initialized before process - FetchOperator fetchOp = entry.getValue(); - JobConf jobConf = fetchOpJobConfMap.get(fetchOp); - - if (jobConf == null) { - jobConf = job; - } - // initialize the forward operator - ObjectInspector objectInspector = fetchOp.getOutputObjectInspector(); - forwardOp.initialize(jobConf, new ObjectInspector[] {objectInspector}); - l4j.info("fetchoperator for " + entry.getKey() + " initialized"); - } - } - - private void generateDummyHashTable(String alias, String bigBucketFileName) throws HiveException,IOException { - // find the (byte)tag for the map join(HashTableSinkOperator) - Operator parentOp = work.getAliasToWork().get(alias); - Operator childOp = parentOp.getChildOperators().get(0); - while ((childOp != null) && (!(childOp instanceof HashTableSinkOperator))) { - parentOp = childOp; - assert parentOp.getChildOperators().size() == 1; - childOp = parentOp.getChildOperators().get(0); - } - if (childOp == null) { - throw new HiveException( - "Cannot find HashTableSink op by tracing down the table scan operator tree"); - } - byte tag = (byte) childOp.getParentOperators().indexOf(parentOp); - - // generate empty hashtable for this (byte)tag - String tmpURI = this.getWork().getTmpFileURI(); - HashMapWrapper hashTable = - new HashMapWrapper(); - - String fileName = work.getBucketFileName(bigBucketFileName); - - HashTableSinkOperator htso = (HashTableSinkOperator)childOp; - String tmpURIPath = Utilities.generatePath(tmpURI, htso.getConf().getDumpFilePrefix(), - tag, fileName); - console.printInfo(Utilities.now() + "\tDump the hashtable into file: " + tmpURIPath); - Path path = new Path(tmpURIPath); - FileSystem fs = path.getFileSystem(job); - File file = new File(path.toUri().getPath()); - fs.create(path); - long fileLength = hashTable.flushMemoryCacheToPersistent(file); - console.printInfo(Utilities.now() + "\tUpload 1 File to: " + tmpURIPath + " File size: " - + fileLength); - hashTable.close(); - } - - private void setUpFetchOpContext(FetchOperator fetchOp, String alias, String currentInputFile) - throws Exception { - - BucketMapJoinContext bucketMatcherCxt = this.work.getBucketMapjoinContext(); - - Class bucketMatcherCls = bucketMatcherCxt.getBucketMatcherClass(); - BucketMatcher bucketMatcher = (BucketMatcher) ReflectionUtils.newInstance(bucketMatcherCls, - null); - bucketMatcher.setAliasBucketFileNameMapping(bucketMatcherCxt.getAliasBucketFileNameMapping()); - - List aliasFiles = bucketMatcher.getAliasBucketFiles(currentInputFile, bucketMatcherCxt - .getMapJoinBigTableAlias(), alias); - fetchOp.setupContext(aliasFiles); - } - - @Override - public boolean isMapRedLocalTask() { - return true; - } - - @Override - public Collection> getTopOperators() { - return getWork().getAliasToWork().values(); - } - - @Override - public String getName() { - return "MAPREDLOCAL"; - } - - @Override - public StageType getType() { - //assert false; - return StageType.MAPREDLOCAL; - } - - @Override - public void shutdown() { - super.shutdown(); - if (executor != null) { - executor.destroy(); - executor = null; - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java (working copy) @@ -200,7 +200,7 @@ jc = (JobConf) hconf; } else { // test code path - jc = new JobConf(hconf, ExecDriver.class); + jc = new JobConf(hconf); } currentStat = null; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHelper.java (working copy) @@ -1,640 +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.IOException; -import java.io.Serializable; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collections; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.JavaUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.ql.MapRedStats; -import org.apache.hadoop.hive.ql.exec.Operator.ProgressCounter; -import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; -import org.apache.hadoop.hive.ql.plan.ReducerTimeStatsPerJob; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; -import org.apache.hadoop.hive.ql.stats.ClientStatsPublisher; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.mapred.Counters; -import org.apache.hadoop.mapred.Counters.Counter; -import org.apache.hadoop.mapred.JobClient; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.RunningJob; -import org.apache.hadoop.mapred.TaskCompletionEvent; -import org.apache.hadoop.mapred.TaskReport; -import org.apache.log4j.Appender; -import org.apache.log4j.FileAppender; -import org.apache.log4j.LogManager; - -public class HadoopJobExecHelper { - - static final private Log LOG = LogFactory.getLog(HadoopJobExecHelper.class.getName()); - - protected transient JobConf job; - protected Task task; - - protected transient int mapProgress = 0; - protected transient int reduceProgress = 0; - public transient String jobId; - private LogHelper console; - private HadoopJobExecHook callBackObj; - - /** - * Update counters relevant to this task. - */ - private void updateCounters(Counters ctrs, RunningJob rj) throws IOException { - mapProgress = Math.round(rj.mapProgress() * 100); - mapProgress = mapProgress == 100 ? (int)Math.floor(rj.mapProgress() * 100) : mapProgress; - reduceProgress = Math.round(rj.reduceProgress() * 100); - reduceProgress = reduceProgress == 100 ? (int)Math.floor(rj.reduceProgress() * 100) : reduceProgress; - task.taskCounters.put("CNTR_NAME_" + task.getId() + "_MAP_PROGRESS", Long.valueOf(mapProgress)); - task.taskCounters.put("CNTR_NAME_" + task.getId() + "_REDUCE_PROGRESS", Long.valueOf(reduceProgress)); - if (ctrs == null) { - // hadoop might return null if it cannot locate the job. - // we may still be able to retrieve the job status - so ignore - return; - } - if(callBackObj != null) { - callBackObj.updateCounters(ctrs, rj); - } - } - - /** - * This msg pattern is used to track when a job is started. - * - * @param jobId - * @return - */ - private static String getJobStartMsg(String jobId) { - return "Starting Job = " + jobId; - } - - /** - * this msg pattern is used to track when a job is successfully done. - * - * @param jobId - * @return the job end message - */ - public static String getJobEndMsg(String jobId) { - return "Ended Job = " + jobId; - } - - public boolean mapStarted() { - return mapProgress > 0; - } - - public boolean reduceStarted() { - return reduceProgress > 0; - } - - public boolean mapDone() { - return mapProgress == 100; - } - - public boolean reduceDone() { - return reduceProgress == 100; - } - - - public String getJobId() { - return jobId; - } - - public void setJobId(String jobId) { - this.jobId = jobId; - } - - - public HadoopJobExecHelper() { - } - - public HadoopJobExecHelper(JobConf job, LogHelper console, - Task task, HadoopJobExecHook hookCallBack) { - this.job = job; - this.console = console; - this.task = task; - this.callBackObj = hookCallBack; - } - - - /** - * A list of the currently running jobs spawned in this Hive instance that is used to kill all - * running jobs in the event of an unexpected shutdown - i.e., the JVM shuts down while there are - * still jobs running. - */ - public static Map runningJobKillURIs = Collections - .synchronizedMap(new HashMap()); - - - /** - * In Hive, when the user control-c's the command line, any running jobs spawned from that command - * line are best-effort killed. - * - * This static constructor registers a shutdown thread to iterate over all the running job kill - * URLs and do a get on them. - * - */ - static { - if (new org.apache.hadoop.conf.Configuration() - .getBoolean("webinterface.private.actions", false)) { - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - killRunningJobs(); - } - }); - } - } - - public static void killRunningJobs() { - synchronized (runningJobKillURIs) { - for (String uri : runningJobKillURIs.values()) { - try { - System.err.println("killing job with: " + uri); - java.net.HttpURLConnection conn = (java.net.HttpURLConnection) new java.net.URL(uri) - .openConnection(); - conn.setRequestMethod("POST"); - int retCode = conn.getResponseCode(); - if (retCode != 200) { - System.err.println("Got an error trying to kill job with URI: " + uri + " = " - + retCode); - } - } catch (Exception e) { - System.err.println("trying to kill job, caught: " + e); - // do nothing - } - } - } - } - - public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { - if (ctrs == null) { - // hadoop might return null if it cannot locate the job. - // we may still be able to retrieve the job status - so ignore - return false; - } - // check for number of created files - long numFiles = ctrs.getCounter(ProgressCounter.CREATED_FILES); - long upperLimit = HiveConf.getLongVar(job, HiveConf.ConfVars.MAXCREATEDFILES); - if (numFiles > upperLimit) { - errMsg.append("total number of created files now is " + numFiles + ", which exceeds ").append(upperLimit); - return true; - } - return this.callBackObj.checkFatalErrors(ctrs, errMsg); - } - - private MapRedStats progress(ExecDriverTaskHandle th) throws IOException { - JobClient jc = th.getJobClient(); - RunningJob rj = th.getRunningJob(); - String lastReport = ""; - SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS"); - //DecimalFormat longFormatter = new DecimalFormat("###,###"); - long reportTime = System.currentTimeMillis(); - long maxReportInterval = - HiveConf.getLongVar(job, HiveConf.ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL); - boolean fatal = false; - StringBuilder errMsg = new StringBuilder(); - long pullInterval = HiveConf.getLongVar(job, HiveConf.ConfVars.HIVECOUNTERSPULLINTERVAL); - boolean initializing = true; - boolean initOutputPrinted = false; - long cpuMsec = -1; - int numMap = -1; - int numReduce = -1; - List clientStatPublishers = getClientStatPublishers(); - - while (!rj.isComplete()) { - try { - Thread.sleep(pullInterval); - } catch (InterruptedException e) { - } - - if (initializing && ShimLoader.getHadoopShims().isJobPreparing(rj)) { - // No reason to poll untill the job is initialized - continue; - } else { - // By now the job is initialized so no reason to do - // rj.getJobState() again and we do not want to do an extra RPC call - initializing = false; - } - - if (!initOutputPrinted) { - SessionState ss = SessionState.get(); - - String logMapper; - String logReducer; - - TaskReport[] mappers = jc.getMapTaskReports(rj.getJobID()); - if (mappers == null) { - logMapper = "no information for number of mappers; "; - } else { - numMap = mappers.length; - if (ss != null) { - ss.getHiveHistory().setTaskProperty(SessionState.get().getQueryId(), getId(), - Keys.TASK_NUM_MAPPERS, Integer.toString(numMap)); - } - logMapper = "number of mappers: " + numMap + "; "; - } - - TaskReport[] reducers = jc.getReduceTaskReports(rj.getJobID()); - if (reducers == null) { - logReducer = "no information for number of reducers. "; - } else { - numReduce = reducers.length; - if (ss != null) { - ss.getHiveHistory().setTaskProperty(SessionState.get().getQueryId(), getId(), - Keys.TASK_NUM_REDUCERS, Integer.toString(numReduce)); - } - logReducer = "number of reducers: " + numReduce; - } - - console - .printInfo("Hadoop job information for " + getId() + ": " + logMapper + logReducer); - initOutputPrinted = true; - } - - RunningJob newRj = jc.getJob(rj.getJobID()); - if (newRj == null) { - // under exceptional load, hadoop may not be able to look up status - // of finished jobs (because it has purged them from memory). From - // hive's perspective - it's equivalent to the job having failed. - // So raise a meaningful exception - throw new IOException("Could not find status of job:" + rj.getJobID()); - } else { - th.setRunningJob(newRj); - rj = newRj; - } - - // If fatal errors happen we should kill the job immediately rather than - // let the job retry several times, which eventually lead to failure. - if (fatal) { - continue; // wait until rj.isComplete - } - - Counters ctrs = th.getCounters(); - - if (fatal = checkFatalErrors(ctrs, errMsg)) { - console.printError("[Fatal Error] " + errMsg.toString() + ". Killing the job."); - rj.killJob(); - continue; - } - errMsg.setLength(0); - - updateCounters(ctrs, rj); - - // Prepare data for Client Stat Publishers (if any present) and execute them - if (clientStatPublishers.size() > 0 && ctrs != null) { - Map exctractedCounters = extractAllCounterValues(ctrs); - for (ClientStatsPublisher clientStatPublisher : clientStatPublishers) { - try { - clientStatPublisher.run(exctractedCounters, rj.getID().toString()); - } catch (RuntimeException runtimeException) { - LOG.error("Exception " + runtimeException.getClass().getCanonicalName() - + " thrown when running clientStatsPublishers. The stack trace is: ", - runtimeException); - } - } - } - - String report = " " + getId() + " map = " + mapProgress + "%, reduce = " + reduceProgress - + "%"; - - - if (!report.equals(lastReport) - || System.currentTimeMillis() >= reportTime + maxReportInterval) { - // find out CPU msecs - // In the case that we can't find out this number, we just skip the step to print - // it out. - if (ctrs != null) { - Counter counterCpuMsec = ctrs.findCounter("org.apache.hadoop.mapred.Task$Counter", - "CPU_MILLISECONDS"); - if (counterCpuMsec != null) { - long newCpuMSec = counterCpuMsec.getValue(); - if (newCpuMSec > 0) { - cpuMsec = newCpuMSec; - report += ", Cumulative CPU " - + (cpuMsec / 1000D) + " sec"; - } - } - } - - // write out serialized plan with counters to log file - // LOG.info(queryPlan); - String output = dateFormat.format(Calendar.getInstance().getTime()) + report; - SessionState ss = SessionState.get(); - if (ss != null) { - ss.getHiveHistory().setTaskCounters(SessionState.get().getQueryId(), getId(), ctrs); - ss.getHiveHistory().setTaskProperty(SessionState.get().getQueryId(), getId(), - Keys.TASK_HADOOP_PROGRESS, output); - if (ss.getConf().getBoolVar(HiveConf.ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS)) { - ss.getHiveHistory().progressTask(SessionState.get().getQueryId(), this.task); - this.callBackObj.logPlanProgress(ss); - } - } - console.printInfo(output); - lastReport = report; - reportTime = System.currentTimeMillis(); - } - } - - if (cpuMsec > 0) { - console.printInfo("MapReduce Total cumulative CPU time: " - + Utilities.formatMsecToStr(cpuMsec)); - } - - boolean success; - - Counters ctrs = th.getCounters(); - if (fatal) { - success = false; - } else { - // check for fatal error again in case it occurred after - // the last check before the job is completed - if (checkFatalErrors(ctrs, errMsg)) { - console.printError("[Fatal Error] " + errMsg.toString()); - success = false; - } else { - SessionState ss = SessionState.get(); - if (ss != null) { - ss.getHiveHistory().setTaskCounters(SessionState.get().getQueryId(), getId(), ctrs); - } - success = rj.isSuccessful(); - } - } - - if (ctrs != null) { - Counter counterCpuMsec = ctrs.findCounter("org.apache.hadoop.mapred.Task$Counter", - "CPU_MILLISECONDS"); - if (counterCpuMsec != null) { - long newCpuMSec = counterCpuMsec.getValue(); - if (newCpuMSec > cpuMsec) { - cpuMsec = newCpuMSec; - } - } - } - - MapRedStats mapRedStats = new MapRedStats(numMap, numReduce, cpuMsec, success, rj.getID().toString()); - mapRedStats.setCounters(ctrs); - - // update based on the final value of the counters - updateCounters(ctrs, rj); - - SessionState ss = SessionState.get(); - if (ss != null) { - this.callBackObj.logPlanProgress(ss); - } - // LOG.info(queryPlan); - return mapRedStats; - } - - private String getId() { - return this.task.getId(); - } - - /** - * from StreamJob.java. - */ - public void jobInfo(RunningJob rj) { - if (ShimLoader.getHadoopShims().isLocalMode(job)) { - console.printInfo("Job running in-process (local Hadoop)"); - } else { - if (SessionState.get() != null) { - SessionState.get().getHiveHistory().setTaskProperty(SessionState.get().getQueryId(), - getId(), Keys.TASK_HADOOP_ID, rj.getJobID()); - } - console.printInfo(getJobStartMsg(rj.getJobID()) + ", Tracking URL = " - + rj.getTrackingURL()); - console.printInfo("Kill Command = " + HiveConf.getVar(job, HiveConf.ConfVars.HADOOPBIN) - + " job -kill " + rj.getJobID()); - } - } - - /** - * This class contains the state of the running task Going forward, we will return this handle - * from execute and Driver can split execute into start, monitorProgess and postProcess. - */ - private static class ExecDriverTaskHandle extends TaskHandle { - JobClient jc; - RunningJob rj; - - JobClient getJobClient() { - return jc; - } - - RunningJob getRunningJob() { - return rj; - } - - public ExecDriverTaskHandle(JobClient jc, RunningJob rj) { - this.jc = jc; - this.rj = rj; - } - - public void setRunningJob(RunningJob job) { - rj = job; - } - - @Override - public Counters getCounters() throws IOException { - return rj.getCounters(); - } - } - - - public void localJobDebugger(int exitVal, String taskId) { - StringBuilder sb = new StringBuilder(); - sb.append("\n"); - sb.append("Task failed!\n"); - sb.append("Task ID:\n " + taskId + "\n\n"); - sb.append("Logs:\n"); - console.printError(sb.toString()); - - for (Appender a : Collections.list((Enumeration) - LogManager.getRootLogger().getAllAppenders())) { - if (a instanceof FileAppender) { - console.printError((new Path(((FileAppender)a).getFile())).toUri().getPath()); - } - } - } - - public int progressLocal(Process runningJob, String taskId) { - int exitVal = -101; - try { - exitVal = runningJob.waitFor(); //TODO: poll periodically - } catch (InterruptedException e) { - } - - if (exitVal != 0) { - console.printError("Execution failed with exit status: " + exitVal); - console.printError("Obtaining error information"); - if (HiveConf.getBoolVar(job, HiveConf.ConfVars.SHOW_JOB_FAIL_DEBUG_INFO)) { - // Since local jobs are run sequentially, all relevant information is already available - // Therefore, no need to fetch job debug info asynchronously - localJobDebugger(exitVal, taskId); - } - } else { - console.printInfo("Execution completed successfully"); - console.printInfo("Mapred Local Task Succeeded . Convert the Join into MapJoin"); - } - return exitVal; - } - - - public int progress(RunningJob rj, JobClient jc) throws IOException { - jobId = rj.getJobID(); - - int returnVal = 0; - - // remove the pwd from conf file so that job tracker doesn't show this - // logs - String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); - if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); - } - - // replace it back - if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, pwd); - } - - // add to list of running jobs to kill in case of abnormal shutdown - - runningJobKillURIs.put(rj.getJobID(), rj.getTrackingURL() + "&action=kill"); - - ExecDriverTaskHandle th = new ExecDriverTaskHandle(jc, rj); - jobInfo(rj); - MapRedStats mapRedStats = progress(th); - - this.task.taskHandle = th; - // Not always there is a SessionState. Sometimes ExeDriver is directly invoked - // for special modes. In that case, SessionState.get() is empty. - if (SessionState.get() != null) { - SessionState.get().getLastMapRedStatsList().add(mapRedStats); - - // Computes the skew for all the MapReduce irrespective - // of Success or Failure - if (this.task.getQueryPlan() != null) { - computeReducerTimeStatsPerJob(rj); - } - } - - boolean success = mapRedStats.isSuccess(); - - String statusMesg = getJobEndMsg(rj.getJobID()); - if (!success) { - statusMesg += " with errors"; - returnVal = 2; - console.printError(statusMesg); - if (HiveConf.getBoolVar(job, HiveConf.ConfVars.SHOW_JOB_FAIL_DEBUG_INFO) || - HiveConf.getBoolVar(job, HiveConf.ConfVars.JOB_DEBUG_CAPTURE_STACKTRACES)) { - try { - JobDebugger jd; - if (SessionState.get() != null) { - jd = new JobDebugger(job, rj, console, SessionState.get().getStackTraces()); - } else { - jd = new JobDebugger(job, rj, console); - } - Thread t = new Thread(jd); - t.start(); - t.join(HiveConf.getIntVar(job, HiveConf.ConfVars.JOB_DEBUG_TIMEOUT)); - int ec = jd.getErrorCode(); - if (ec > 0) { - returnVal = ec; - } - } catch (InterruptedException e) { - console.printError("Timed out trying to grab more detailed job failure" - + " information, please check jobtracker for more info"); - } - } - } else { - console.printInfo(statusMesg); - } - - return returnVal; - } - - - private void computeReducerTimeStatsPerJob(RunningJob rj) throws IOException { - TaskCompletionEvent[] taskCompletions = rj.getTaskCompletionEvents(0); - List reducersRunTimes = new ArrayList(); - - for (TaskCompletionEvent taskCompletion : taskCompletions) { - String[] taskJobIds = ShimLoader.getHadoopShims().getTaskJobIDs(taskCompletion); - if (taskJobIds == null) { - // Task attempt info is unavailable in this Hadoop version"); - continue; - } - String taskId = taskJobIds[0]; - if (!taskCompletion.isMapTask()) { - reducersRunTimes.add(new Integer(taskCompletion.getTaskRunTime())); - } - } - // Compute the reducers run time statistics for the job - ReducerTimeStatsPerJob reducerTimeStatsPerJob = new ReducerTimeStatsPerJob(reducersRunTimes, - new String(this.jobId)); - // Adding the reducers run time statistics for the job in the QueryPlan - this.task.getQueryPlan().getReducerTimeStatsPerJobList().add(reducerTimeStatsPerJob); - return; - } - - - private Map extractAllCounterValues(Counters counters) { - Map exctractedCounters = new HashMap(); - for (Counters.Group cg : counters) { - for (Counter c : cg) { - exctractedCounters.put(cg.getName() + "::" + c.getName(), new Double(c.getCounter())); - } - } - return exctractedCounters; - } - - private List getClientStatPublishers() { - List clientStatsPublishers = new ArrayList(); - String confString = HiveConf.getVar(job, HiveConf.ConfVars.CLIENTSTATSPUBLISHERS); - confString = confString.trim(); - if (confString.equals("")) { - return clientStatsPublishers; - } - - String[] clientStatsPublisherClasses = confString.split(","); - - for (String clientStatsPublisherClass : clientStatsPublisherClasses) { - try { - clientStatsPublishers.add((ClientStatsPublisher) Class.forName( - clientStatsPublisherClass.trim(), true, JavaUtils.getClassLoader()).newInstance()); - } catch (Exception e) { - LOG.warn(e.getClass().getName() + " occured when trying to create class: " - + clientStatsPublisherClass.trim() + " implementing ClientStatsPublisher interface"); - LOG.warn("The exception message is: " + e.getMessage()); - LOG.warn("Program will continue, but without this ClientStatsPublisher working"); - } - } - return clientStatsPublishers; - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java (working copy) @@ -74,7 +74,7 @@ } objToSendToUDTF = new Object[inputFields.size()]; - MapredContext context = MapredContext.get(); + UDFContext context = UDFContext.get(); if (context != null) { context.setup(genericUDTF); } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHook.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHook.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/HadoopJobExecHook.java (working copy) @@ -1,34 +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.IOException; - -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.mapred.Counters; -import org.apache.hadoop.mapred.RunningJob; - -@SuppressWarnings("deprecation") -public interface HadoopJobExecHook { - - public void updateCounters(Counters ctrs, RunningJob rj) throws IOException; - public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg); - public void logPlanProgress(SessionState ss) throws IOException; - -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (working copy) @@ -31,6 +31,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.lib.Node; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.parse.SemanticException; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecMapperContext.java (working copy) @@ -1,154 +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.util.Map; - -import org.apache.commons.logging.Log; -import org.apache.hadoop.hive.ql.io.IOContext; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.mapred.JobConf; - -public class ExecMapperContext { - - public static final Log l4j = ExecMapper.l4j; - - // lastInputFile should be changed by the root of the operator tree ExecMapper.map() - // but kept unchanged throughout the operator tree for one row - private String lastInputFile = null; - - // currentInputFile will be updated only by inputFileChanged(). If inputFileChanged() - // is not called throughout the opertor tree, currentInputFile won't be used anyways - // so it won't be updated. - private String currentInputFile = null; - private boolean inputFileChecked = false; - - // for SMB join, replaced with number part of task-id , making output file name - // if big alias is not partitioned table, it's bucket number - // if big alias is partitioned table, it's partition spec + bucket number - private String fileId = null; - private MapredLocalWork localWork = null; - private Map fetchOperators; - private JobConf jc; - - private IOContext ioCxt; - - private String currentBigBucketFile=null; - - public String getCurrentBigBucketFile() { - return currentBigBucketFile; - } - - public void setCurrentBigBucketFile(String currentBigBucketFile) { - this.currentBigBucketFile = currentBigBucketFile; - } - - public ExecMapperContext() { - ioCxt = IOContext.get(); - } - - public void clear() { - IOContext.clear(); - ioCxt = null; - } - - /** - * For CompbineFileInputFormat, the mapper's input file will be changed on the - * fly, and the input file name is passed to jobConf by shims/initNextRecordReader. - * If the map local work has any mapping depending on the current - * mapper's input file, the work need to clear context and re-initialization - * after the input file changed. This is first introduced to process bucket - * map join. - * - * @return is the input file changed? - */ - public boolean inputFileChanged() { - if (!inputFileChecked) { - currentInputFile = this.ioCxt.getInputFile(); - inputFileChecked = true; - } - return lastInputFile == null || !lastInputFile.equals(currentInputFile); - } - - /** - * Reset the execution context for each new row. This function should be called only - * once at the root of the operator tree -- ExecMapper.map(). - * Note: this function should be kept minimum since it is called for each input row. - */ - public void resetRow() { - // Update the lastInputFile with the currentInputFile. - lastInputFile = currentInputFile; - inputFileChecked = false; - } - - public String getLastInputFile() { - return lastInputFile; - } - - public void setLastInputFile(String lastInputFile) { - this.lastInputFile = lastInputFile; - } - - public String getCurrentInputFile() { - currentInputFile = this.ioCxt.getInputFile(); - return currentInputFile; - } - - public void setCurrentInputFile(String currentInputFile) { - this.currentInputFile = currentInputFile; - } - - public JobConf getJc() { - return jc; - } - public void setJc(JobConf jc) { - this.jc = jc; - } - - public MapredLocalWork getLocalWork() { - return localWork; - } - - public void setLocalWork(MapredLocalWork localWork) { - this.localWork = localWork; - } - - public String getFileId() { - return fileId; - } - - public void setFileId(String fileId) { - this.fileId = fileId; - } - - public Map getFetchOperators() { - return fetchOperators; - } - - public void setFetchOperators(Map fetchOperators) { - this.fetchOperators = fetchOperators; - } - - public IOContext getIoCxt() { - return ioCxt; - } - - public void setIoCxt(IOContext ioCxt) { - this.ioCxt = ioCxt; - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (working copy) @@ -1,1016 +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.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.management.ManagementFactory; -import java.lang.management.MemoryMXBean; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Enumeration; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; - -import org.apache.commons.lang.StringUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.filecache.DistributedCache; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.common.CompressionUtils; -import org.apache.hadoop.hive.common.LogUtils; -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; -import org.apache.hadoop.hive.ql.Context; -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.exec.FileSinkOperator.RecordWriter; -import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; -import org.apache.hadoop.hive.ql.io.HiveKey; -import org.apache.hadoop.hive.ql.io.HiveOutputFormat; -import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; -import org.apache.hadoop.hive.ql.io.IOPrepareCache; -import org.apache.hadoop.hive.ql.io.OneNullRowInputFormat; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.FetchWork; -import org.apache.hadoop.hive.ql.plan.FileSinkDesc; -import org.apache.hadoop.hive.ql.plan.MapredLocalWork; -import org.apache.hadoop.hive.ql.plan.MapredWork; -import org.apache.hadoop.hive.ql.plan.OperatorDesc; -import org.apache.hadoop.hive.ql.plan.PartitionDesc; -import org.apache.hadoop.hive.ql.plan.TableDesc; -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.hive.ql.stats.StatsFactory; -import org.apache.hadoop.hive.ql.stats.StatsPublisher; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.Counters; -import org.apache.hadoop.mapred.FileInputFormat; -import org.apache.hadoop.mapred.InputFormat; -import org.apache.hadoop.mapred.JobClient; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.Partitioner; -import org.apache.hadoop.mapred.RunningJob; -import org.apache.hadoop.mapred.lib.TotalOrderPartitioner; -import org.apache.log4j.Appender; -import org.apache.log4j.BasicConfigurator; -import org.apache.log4j.FileAppender; -import org.apache.log4j.LogManager; -import org.apache.log4j.varia.NullAppender; - -/** - * ExecDriver. - * - */ -public class ExecDriver extends Task implements Serializable, HadoopJobExecHook { - - private static final long serialVersionUID = 1L; - private static final String JOBCONF_FILENAME = "jobconf.xml"; - - protected transient JobConf job; - public static MemoryMXBean memoryMXBean; - protected HadoopJobExecHelper jobExecHelper; - - protected static transient final Log LOG = LogFactory.getLog(ExecDriver.class); - - private RunningJob rj; - - /** - * Constructor when invoked from QL. - */ - public ExecDriver() { - super(); - console = new LogHelper(LOG); - this.jobExecHelper = new HadoopJobExecHelper(job, console, this, this); - } - - @Override - public boolean requireLock() { - return true; - } - - private void initializeFiles(String prop, String files) { - if (files != null && files.length() > 0) { - job.set(prop, files); - ShimLoader.getHadoopShims().setTmpFiles(prop, files); - } - } - - /** - * Initialization when invoked from QL. - */ - @Override - public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) { - super.initialize(conf, queryPlan, driverContext); - - job = new JobConf(conf, ExecDriver.class); - - // NOTE: initialize is only called if it is in non-local mode. - // In case it's in non-local mode, we need to move the SessionState files - // and jars to jobConf. - // In case it's in local mode, MapRedTask will set the jobConf. - // - // "tmpfiles" and "tmpjars" are set by the method ExecDriver.execute(), - // which will be called by both local and NON-local mode. - String addedFiles = Utilities.getResourceFiles(job, SessionState.ResourceType.FILE); - if (StringUtils.isNotBlank(addedFiles)) { - HiveConf.setVar(job, ConfVars.HIVEADDEDFILES, addedFiles); - } - String addedJars = Utilities.getResourceFiles(job, SessionState.ResourceType.JAR); - if (StringUtils.isNotBlank(addedJars)) { - HiveConf.setVar(job, ConfVars.HIVEADDEDJARS, addedJars); - } - String addedArchives = Utilities.getResourceFiles(job, SessionState.ResourceType.ARCHIVE); - if (StringUtils.isNotBlank(addedArchives)) { - HiveConf.setVar(job, ConfVars.HIVEADDEDARCHIVES, addedArchives); - } - this.jobExecHelper = new HadoopJobExecHelper(job, console, this, this); - } - - /** - * Constructor/Initialization for invocation as independent utility. - */ - public ExecDriver(MapredWork plan, JobConf job, boolean isSilent) throws HiveException { - setWork(plan); - this.job = job; - console = new LogHelper(LOG, isSilent); - this.jobExecHelper = new HadoopJobExecHelper(job, console, this, this); - } - - /** - * Fatal errors are those errors that cannot be recovered by retries. These are application - * dependent. Examples of fatal errors include: - the small table in the map-side joins is too - * large to be feasible to be handled by one mapper. The job should fail and the user should be - * warned to use regular joins rather than map-side joins. Fatal errors are indicated by counters - * that are set at execution time. If the counter is non-zero, a fatal error occurred. The value - * of the counter indicates the error type. - * - * @return true if fatal errors happened during job execution, false otherwise. - */ - public boolean checkFatalErrors(Counters ctrs, StringBuilder errMsg) { - for (Operator op : work.getAliasToWork().values()) { - if (op.checkFatalErrors(ctrs, errMsg)) { - return true; - } - } - if (work.getReducer() != null) { - if (work.getReducer().checkFatalErrors(ctrs, errMsg)) { - return true; - } - } - return false; - } - - protected void createTmpDirs() throws IOException { - // fix up outputs - Map> pa = work.getPathToAliases(); - if (pa != null) { - List> opList = - new ArrayList>(); - - if (work.getReducer() != null) { - opList.add(work.getReducer()); - } - - for (List ls : pa.values()) { - for (String a : ls) { - opList.add(work.getAliasToWork().get(a)); - - while (!opList.isEmpty()) { - Operator op = opList.remove(0); - - if (op instanceof FileSinkOperator) { - FileSinkDesc fdesc = ((FileSinkOperator) op).getConf(); - String tempDir = fdesc.getDirName(); - - if (tempDir != null) { - Path tempPath = Utilities.toTempPath(new Path(tempDir)); - LOG.info("Making Temp Directory: " + tempDir); - FileSystem fs = tempPath.getFileSystem(job); - fs.mkdirs(tempPath); - } - } - - if (op.getChildOperators() != null) { - opList.addAll(op.getChildOperators()); - } - } - } - } - } - } - - /** - * Execute a query plan using Hadoop. - */ - @Override - public int execute(DriverContext driverContext) { - - IOPrepareCache ioPrepareCache = IOPrepareCache.get(); - ioPrepareCache.clear(); - - boolean success = true; - - String invalidReason = work.isInvalid(); - if (invalidReason != null) { - throw new RuntimeException("Plan invalid, Reason: " + invalidReason); - } - - Context ctx = driverContext.getCtx(); - boolean ctxCreated = false; - String emptyScratchDirStr; - Path emptyScratchDir; - - try { - if (ctx == null) { - ctx = new Context(job); - ctxCreated = true; - } - - emptyScratchDirStr = ctx.getMRTmpFileURI(); - emptyScratchDir = new Path(emptyScratchDirStr); - FileSystem fs = emptyScratchDir.getFileSystem(job); - fs.mkdirs(emptyScratchDir); - } catch (IOException e) { - e.printStackTrace(); - console.printError("Error launching map-reduce job", "\n" - + org.apache.hadoop.util.StringUtils.stringifyException(e)); - return 5; - } - - ShimLoader.getHadoopShims().prepareJobOutput(job); - //See the javadoc on HiveOutputFormatImpl and HadoopShims.prepareJobOutput() - job.setOutputFormat(HiveOutputFormatImpl.class); - job.setMapperClass(ExecMapper.class); - - job.setMapOutputKeyClass(HiveKey.class); - job.setMapOutputValueClass(BytesWritable.class); - - try { - job.setPartitionerClass((Class) (Class.forName(HiveConf.getVar(job, - HiveConf.ConfVars.HIVEPARTITIONER)))); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e.getMessage()); - } - - if (work.getNumMapTasks() != null) { - job.setNumMapTasks(work.getNumMapTasks().intValue()); - } - - if (work.getMaxSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, work.getMaxSplitSize().longValue()); - } - - if (work.getMinSplitSize() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work.getMinSplitSize().longValue()); - } - - if (work.getMinSplitSizePerNode() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, work.getMinSplitSizePerNode().longValue()); - } - - if (work.getMinSplitSizePerRack() != null) { - HiveConf.setLongVar(job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, work.getMinSplitSizePerRack().longValue()); - } - - job.setNumReduceTasks(work.getNumReduceTasks().intValue()); - job.setReducerClass(ExecReducer.class); - - // set input format information if necessary - setInputAttributes(job); - - // Turn on speculative execution for reducers - boolean useSpeculativeExecReducers = HiveConf.getBoolVar(job, - HiveConf.ConfVars.HIVESPECULATIVEEXECREDUCERS); - HiveConf.setBoolVar(job, HiveConf.ConfVars.HADOOPSPECULATIVEEXECREDUCERS, - useSpeculativeExecReducers); - - String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT); - if ((inpFormat == null) || (!StringUtils.isNotBlank(inpFormat))) { - inpFormat = ShimLoader.getHadoopShims().getInputFormatClassName(); - } - - if (getWork().isUseBucketizedHiveInputFormat()) { - inpFormat = BucketizedHiveInputFormat.class.getName(); - } - - LOG.info("Using " + inpFormat); - - try { - job.setInputFormat((Class) (Class.forName(inpFormat))); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e.getMessage()); - } - - - // No-Op - we don't really write anything here .. - job.setOutputKeyClass(Text.class); - job.setOutputValueClass(Text.class); - - // Transfer HIVEAUXJARS and HIVEADDEDJARS to "tmpjars" so hadoop understands - // it - String auxJars = HiveConf.getVar(job, HiveConf.ConfVars.HIVEAUXJARS); - String addedJars = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDJARS); - if (StringUtils.isNotBlank(auxJars) || StringUtils.isNotBlank(addedJars)) { - String allJars = StringUtils.isNotBlank(auxJars) ? (StringUtils.isNotBlank(addedJars) ? addedJars - + "," + auxJars - : auxJars) - : addedJars; - LOG.info("adding libjars: " + allJars); - initializeFiles("tmpjars", allJars); - } - - // Transfer HIVEADDEDFILES to "tmpfiles" so hadoop understands it - String addedFiles = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDFILES); - if (StringUtils.isNotBlank(addedFiles)) { - initializeFiles("tmpfiles", addedFiles); - } - int returnVal = 0; - boolean noName = StringUtils.isEmpty(HiveConf.getVar(job, HiveConf.ConfVars.HADOOPJOBNAME)); - - if (noName) { - // This is for a special case to ensure unit tests pass - HiveConf.setVar(job, HiveConf.ConfVars.HADOOPJOBNAME, "JOB" + Utilities.randGen.nextInt()); - } - String addedArchives = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDARCHIVES); - // Transfer HIVEADDEDARCHIVES to "tmparchives" so hadoop understands it - if (StringUtils.isNotBlank(addedArchives)) { - initializeFiles("tmparchives", addedArchives); - } - - try{ - MapredLocalWork localwork = work.getMapLocalWork(); - if (localwork != null) { - if (!ShimLoader.getHadoopShims().isLocalMode(job)) { - Path localPath = new Path(localwork.getTmpFileURI()); - Path hdfsPath = new Path(work.getTmpHDFSFileURI()); - - FileSystem hdfs = hdfsPath.getFileSystem(job); - FileSystem localFS = localPath.getFileSystem(job); - FileStatus[] hashtableFiles = localFS.listStatus(localPath); - int fileNumber = hashtableFiles.length; - String[] fileNames = new String[fileNumber]; - - for ( int i = 0; i < fileNumber; i++){ - fileNames[i] = hashtableFiles[i].getPath().getName(); - } - - //package and compress all the hashtable files to an archive file - String parentDir = localPath.toUri().getPath(); - String stageId = this.getId(); - String archiveFileURI = Utilities.generateTarURI(parentDir, stageId); - String archiveFileName = Utilities.generateTarFileName(stageId); - localwork.setStageID(stageId); - - CompressionUtils.tar(parentDir, fileNames,archiveFileName); - Path archivePath = new Path(archiveFileURI); - LOG.info("Archive "+ hashtableFiles.length+" hash table files to " + archiveFileURI); - - //upload archive file to hdfs - String hdfsFile =Utilities.generateTarURI(hdfsPath, stageId); - Path hdfsFilePath = new Path(hdfsFile); - short replication = (short) job.getInt("mapred.submit.replication", 10); - hdfs.setReplication(hdfsFilePath, replication); - hdfs.copyFromLocalFile(archivePath, hdfsFilePath); - LOG.info("Upload 1 archive file from" + archivePath + " to: " + hdfsFilePath); - - //add the archive file to distributed cache - DistributedCache.createSymlink(job); - DistributedCache.addCacheArchive(hdfsFilePath.toUri(), job); - LOG.info("Add 1 archive file to distributed cache. Archive file: " + hdfsFilePath.toUri()); - } - } - work.configureJobConf(job); - addInputPaths(job, work, emptyScratchDirStr, ctx); - - Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); - - if (work.getSamplingType() > 0 && work.getNumReduceTasks() > 1) { - try { - handleSampling(driverContext, work, job, new HiveConf(conf)); - job.setPartitionerClass(HiveTotalOrderPartitioner.class); - } catch (Exception e) { - console.printInfo("Not enough sampling data.. Rolling back to single reducer task"); - work.setNumReduceTasks(1); - job.setNumReduceTasks(1); - } - } - - // remove the pwd from conf file so that job tracker doesn't show this - // logs - String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); - if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); - } - JobClient jc = new JobClient(job); - // make this client wait if job trcker is not behaving well. - Throttle.checkJobTracker(job, LOG); - - if (work.isGatheringStats()) { - // initialize stats publishing table - StatsPublisher statsPublisher; - String statsImplementationClass = HiveConf.getVar(job, HiveConf.ConfVars.HIVESTATSDBCLASS); - if (StatsFactory.setImplementation(statsImplementationClass, job)) { - statsPublisher = StatsFactory.getStatsPublisher(); - if (!statsPublisher.init(job)) { // creating stats table if not exists - if (HiveConf.getBoolVar(job, HiveConf.ConfVars.HIVE_STATS_RELIABLE)) { - throw - new HiveException(ErrorMsg.STATSPUBLISHER_INITIALIZATION_ERROR.getErrorCodedMsg()); - } - } - } - } - - this.createTmpDirs(); - - // Finally SUBMIT the JOB! - rj = jc.submitJob(job); - // replace it back - if (pwd != null) { - HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, pwd); - } - - returnVal = jobExecHelper.progress(rj, jc); - success = (returnVal == 0); - } catch (Exception e) { - e.printStackTrace(); - String mesg = " with exception '" + Utilities.getNameMessage(e) + "'"; - if (rj != null) { - mesg = "Ended Job = " + rj.getJobID() + mesg; - } else { - mesg = "Job Submission failed" + mesg; - } - - // Has to use full name to make sure it does not conflict with - // org.apache.commons.lang.StringUtils - console.printError(mesg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - - success = false; - returnVal = 1; - } finally { - Utilities.clearMapRedWork(job); - try { - if (ctxCreated) { - ctx.clear(); - } - - if (rj != null) { - if (returnVal != 0) { - rj.killJob(); - } - HadoopJobExecHelper.runningJobKillURIs.remove(rj.getJobID()); - jobID = rj.getID().toString(); - } - } catch (Exception e) { - } - } - - // get the list of Dynamic partition paths - try { - if (rj != null) { - JobCloseFeedBack feedBack = new JobCloseFeedBack(); - if (work.getAliasToWork() != null) { - for (Operator op : work.getAliasToWork().values()) { - op.jobClose(job, success, feedBack); - } - } - if (work.getReducer() != null) { - work.getReducer().jobClose(job, success, feedBack); - } - } - } catch (Exception e) { - // jobClose needs to execute successfully otherwise fail task - if (success) { - success = false; - returnVal = 3; - String mesg = "Job Commit failed with exception '" + Utilities.getNameMessage(e) + "'"; - console.printError(mesg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); - } - } - - return (returnVal); - } - - private void handleSampling(DriverContext context, MapredWork work, JobConf job, HiveConf conf) - throws Exception { - assert work.getAliasToWork().keySet().size() == 1; - - String alias = work.getAliases().get(0); - Operator topOp = work.getAliasToWork().get(alias); - PartitionDesc partDesc = work.getAliasToPartnInfo().get(alias); - - ArrayList paths = work.getPaths(); - ArrayList parts = work.getPartitionDescs(); - - Path onePath = new Path(paths.get(0)); - String tmpPath = context.getCtx().getExternalTmpFileURI(onePath.toUri()); - - Path partitionFile = new Path(tmpPath, ".partitions"); - TotalOrderPartitioner.setPartitionFile(job, partitionFile); - - PartitionKeySampler sampler = new PartitionKeySampler(); - - if (work.getSamplingType() == MapredWork.SAMPLING_ON_PREV_MR) { - console.printInfo("Use sampling data created in previous MR"); - // merges sampling data from previous MR and make paritition keys for total sort - for (String path : paths) { - Path inputPath = new Path(path); - FileSystem fs = inputPath.getFileSystem(job); - for (FileStatus status : fs.globStatus(new Path(inputPath, ".sampling*"))) { - sampler.addSampleFile(status.getPath(), job); - } - } - } else if (work.getSamplingType() == MapredWork.SAMPLING_ON_START) { - console.printInfo("Creating sampling data.."); - assert topOp instanceof TableScanOperator; - TableScanOperator ts = (TableScanOperator) topOp; - - FetchWork fetchWork; - if (!partDesc.isPartitioned()) { - assert paths.size() == 1; - fetchWork = new FetchWork(paths.get(0), partDesc.getTableDesc()); - } else { - fetchWork = new FetchWork(paths, parts, partDesc.getTableDesc()); - } - fetchWork.setSource(ts); - - // random sampling - FetchOperator fetcher = PartitionKeySampler.createSampler(fetchWork, conf, job, ts); - try { - ts.initialize(conf, new ObjectInspector[]{fetcher.getOutputObjectInspector()}); - ts.setOutputCollector(sampler); - while (fetcher.pushRow()) { } - } finally { - fetcher.clearFetchContext(); - } - } else { - throw new IllegalArgumentException("Invalid sampling type " + work.getSamplingType()); - } - sampler.writePartitionKeys(partitionFile, job); - } - - /** - * Set hive input format, and input format file if necessary. - */ - protected void setInputAttributes(Configuration conf) { - if (work.getInputformat() != null) { - HiveConf.setVar(conf, HiveConf.ConfVars.HIVEINPUTFORMAT, work.getInputformat()); - } - if (work.getIndexIntermediateFile() != null) { - conf.set("hive.index.compact.file", work.getIndexIntermediateFile()); - conf.set("hive.index.blockfilter.file", work.getIndexIntermediateFile()); - } - - // Intentionally overwrites anything the user may have put here - conf.setBoolean("hive.input.format.sorted", work.isInputFormatSorted()); - } - - public boolean mapStarted() { - return this.jobExecHelper.mapStarted(); - } - - public boolean reduceStarted() { - return this.jobExecHelper.reduceStarted(); - } - - public boolean mapDone() { - return this.jobExecHelper.mapDone(); - } - - public boolean reduceDone() { - return this.jobExecHelper.reduceDone(); - } - - private static void printUsage() { - System.err.println("ExecDriver -plan [-jobconffile ]" - + "[-files [,] ...]"); - System.exit(1); - } - - /** - * we are running the hadoop job via a sub-command. this typically happens when we are running - * jobs in local mode. the log4j in this mode is controlled as follows: 1. if the admin provides a - * log4j properties file especially for execution mode - then we pick that up 2. otherwise - we - * default to the regular hive log4j properties if one is supplied 3. if none of the above two - * apply - we don't do anything - the log4j properties would likely be determined by hadoop. - * - * The intention behind providing a separate option #1 is to be able to collect hive run time logs - * generated in local mode in a separate (centralized) location if desired. This mimics the - * behavior of hive run time logs when running against a hadoop cluster where they are available - * on the tasktracker nodes. - */ - - private static void setupChildLog4j(Configuration conf) { - try { - LogUtils.initHiveExecLog4j(); - } catch (LogInitializationException e) { - System.err.println(e.getMessage()); - } - } - - public static void main(String[] args) throws IOException, HiveException { - - String planFileName = null; - String jobConfFileName = null; - boolean noLog = false; - String files = null; - boolean localtask = false; - try { - for (int i = 0; i < args.length; i++) { - if (args[i].equals("-plan")) { - planFileName = args[++i]; - } else if (args[i].equals("-jobconffile")) { - jobConfFileName = args[++i]; - } else if (args[i].equals("-nolog")) { - noLog = true; - } else if (args[i].equals("-files")) { - files = args[++i]; - } else if (args[i].equals("-localtask")) { - localtask = true; - } - } - } catch (IndexOutOfBoundsException e) { - System.err.println("Missing argument to option"); - printUsage(); - } - - JobConf conf; - if (localtask) { - conf = new JobConf(MapredLocalTask.class); - } else { - conf = new JobConf(ExecDriver.class); - } - - if (jobConfFileName != null) { - conf.addResource(new Path(jobConfFileName)); - } - - if (files != null) { - conf.set("tmpfiles", files); - } - - boolean isSilent = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVESESSIONSILENT); - - if (noLog) { - // If started from main(), and noLog is on, we should not output - // any logs. To turn the log on, please set -Dtest.silent=false - BasicConfigurator.resetConfiguration(); - BasicConfigurator.configure(new NullAppender()); - } else { - setupChildLog4j(conf); - } - - Log LOG = LogFactory.getLog(ExecDriver.class.getName()); - LogHelper console = new LogHelper(LOG, isSilent); - - if (planFileName == null) { - console.printError("Must specify Plan File Name"); - printUsage(); - } - - // print out the location of the log file for the user so - // that it's easy to find reason for local mode execution failures - for (Appender appender : Collections.list((Enumeration) LogManager.getRootLogger() - .getAllAppenders())) { - if (appender instanceof FileAppender) { - console.printInfo("Execution log at: " + ((FileAppender) appender).getFile()); - } - } - - // the plan file should always be in local directory - Path p = new Path(planFileName); - FileSystem fs = FileSystem.getLocal(conf); - InputStream pathData = fs.open(p); - - // this is workaround for hadoop-17 - libjars are not added to classpath of the - // child process. so we add it here explicitly - - String auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS); - String addedJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEADDEDJARS); - try { - // see also - code in CliDriver.java - ClassLoader loader = conf.getClassLoader(); - if (StringUtils.isNotBlank(auxJars)) { - loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")); - } - if (StringUtils.isNotBlank(addedJars)) { - loader = Utilities.addToClassPath(loader, StringUtils.split(addedJars, ",")); - } - conf.setClassLoader(loader); - // Also set this to the Thread ContextClassLoader, so new threads will - // inherit - // this class loader, and propagate into newly created Configurations by - // those - // new threads. - Thread.currentThread().setContextClassLoader(loader); - } catch (Exception e) { - throw new HiveException(e.getMessage(), e); - } - int ret; - if (localtask) { - memoryMXBean = ManagementFactory.getMemoryMXBean(); - MapredLocalWork plan = Utilities.deserializeMapRedLocalWork(pathData, conf); - MapredLocalTask ed = new MapredLocalTask(plan, conf, isSilent); - ret = ed.executeFromChildJVM(new DriverContext()); - - } else { - MapredWork plan = Utilities.deserializeMapRedWork(pathData, conf); - ExecDriver ed = new ExecDriver(plan, conf, isSilent); - ret = ed.execute(new DriverContext()); - } - - if (ret != 0) { - System.exit(ret); - } - } - - /** - * Given a Hive Configuration object - generate a command line fragment for passing such - * configuration information to ExecDriver. - */ - public static String generateCmdLine(HiveConf hconf, Context ctx) - throws IOException { - HiveConf tempConf = new HiveConf(); - Path hConfFilePath = new Path(ctx.getLocalTmpFileURI(), JOBCONF_FILENAME); - OutputStream out = null; - - Properties deltaP = hconf.getChangedProperties(); - boolean hadoopLocalMode = ShimLoader.getHadoopShims().isLocalMode(hconf); - String hadoopSysDir = "mapred.system.dir"; - String hadoopWorkDir = "mapred.local.dir"; - - for (Object one : deltaP.keySet()) { - String oneProp = (String) one; - - if (hadoopLocalMode && (oneProp.equals(hadoopSysDir) || oneProp.equals(hadoopWorkDir))) { - continue; - } - - tempConf.set(oneProp, deltaP.getProperty(oneProp)); - } - - // Multiple concurrent local mode job submissions can cause collisions in - // working dirs and system dirs - // Workaround is to rename map red working dir to a temp dir in such cases - if (hadoopLocalMode) { - tempConf.set(hadoopSysDir, hconf.get(hadoopSysDir) + "/" + Utilities.randGen.nextInt()); - tempConf.set(hadoopWorkDir, hconf.get(hadoopWorkDir) + "/" + Utilities.randGen.nextInt()); - } - - try { - out = FileSystem.getLocal(hconf).create(hConfFilePath); - tempConf.writeXml(out); - } finally { - if (out != null) { - out.close(); - } - } - return " -jobconffile " + hConfFilePath.toString(); - } - - @Override - public boolean isMapRedTask() { - return true; - } - - @Override - public Collection> getTopOperators() { - return getWork().getAliasToWork().values(); - } - - @Override - public boolean hasReduce() { - MapredWork w = getWork(); - return w.getReducer() != null; - } - - /** - * Handle a empty/null path for a given alias. - */ - private static int addInputPath(String path, JobConf job, MapredWork work, String hiveScratchDir, - int numEmptyPaths, boolean isEmptyPath, String alias) throws Exception { - // either the directory does not exist or it is empty - assert path == null || isEmptyPath; - - // The input file does not exist, replace it by a empty file - Class outFileFormat = null; - boolean nonNative = true; - boolean oneRow = false; - Properties props; - if (isEmptyPath) { - PartitionDesc partDesc = work.getPathToPartitionInfo().get(path); - props = partDesc.getProperties(); - outFileFormat = partDesc.getOutputFileFormatClass(); - nonNative = partDesc.getTableDesc().isNonNative(); - oneRow = partDesc.getInputFileFormatClass() == OneNullRowInputFormat.class; - } else { - TableDesc tableDesc = work.getAliasToPartnInfo().get(alias).getTableDesc(); - props = tableDesc.getProperties(); - outFileFormat = tableDesc.getOutputFileFormatClass(); - nonNative = tableDesc.isNonNative(); - } - - if (nonNative) { - FileInputFormat.addInputPaths(job, path); - LOG.info("Add a non-native table " + path); - return numEmptyPaths; - } - - // create a dummy empty file in a new directory - String newDir = hiveScratchDir + File.separator + (++numEmptyPaths); - Path newPath = new Path(newDir); - FileSystem fs = newPath.getFileSystem(job); - fs.mkdirs(newPath); - //Qualify the path against the filesystem. The user configured path might contain default port which is skipped - //in the file status. This makes sure that all paths which goes into PathToPartitionInfo are always listed status - //filepath. - newPath = fs.makeQualified(newPath); - String newFile = newDir + File.separator + "emptyFile"; - Path newFilePath = new Path(newFile); - - LOG.info("Changed input file to " + newPath.toString()); - - // toggle the work - - LinkedHashMap> pathToAliases = work.getPathToAliases(); - - if (isEmptyPath) { - assert path != null; - pathToAliases.put(newPath.toUri().toString(), pathToAliases.get(path)); - pathToAliases.remove(path); - } else { - assert path == null; - ArrayList newList = new ArrayList(); - newList.add(alias); - pathToAliases.put(newPath.toUri().toString(), newList); - } - - work.setPathToAliases(pathToAliases); - - LinkedHashMap pathToPartitionInfo = work.getPathToPartitionInfo(); - if (isEmptyPath) { - pathToPartitionInfo.put(newPath.toUri().toString(), pathToPartitionInfo.get(path)); - pathToPartitionInfo.remove(path); - } else { - PartitionDesc pDesc = work.getAliasToPartnInfo().get(alias).clone(); - pathToPartitionInfo.put(newPath.toUri().toString(), pDesc); - } - work.setPathToPartitionInfo(pathToPartitionInfo); - - String onefile = newPath.toString(); - RecordWriter recWriter = outFileFormat.newInstance().getHiveRecordWriter(job, newFilePath, - Text.class, false, props, null); - if (oneRow) { - // empty files are ommited at CombineHiveInputFormat. - // for metadata only query, it effectively makes partition columns disappear.. - // this could be fixed by other methods, but this seemed to be the most easy (HIVEV-2955) - recWriter.write(new Text("empty")); // written via HiveIgnoreKeyTextOutputFormat - } - recWriter.close(false); - FileInputFormat.addInputPaths(job, onefile); - return numEmptyPaths; - } - - public static void addInputPaths(JobConf job, MapredWork work, String hiveScratchDir, Context ctx) - throws Exception { - int numEmptyPaths = 0; - - Set pathsProcessed = new HashSet(); - List pathsToAdd = new LinkedList(); - // AliasToWork contains all the aliases - for (String oneAlias : work.getAliasToWork().keySet()) { - LOG.info("Processing alias " + oneAlias); - List emptyPaths = new ArrayList(); - - // The alias may not have any path - String path = null; - for (String onefile : work.getPathToAliases().keySet()) { - List aliases = work.getPathToAliases().get(onefile); - if (aliases.contains(oneAlias)) { - path = onefile; - - // Multiple aliases can point to the same path - it should be - // processed only once - if (pathsProcessed.contains(path)) { - continue; - } - - pathsProcessed.add(path); - - LOG.info("Adding input file " + path); - if (Utilities.isEmptyPath(job, path, ctx)) { - emptyPaths.add(path); - } else { - pathsToAdd.add(path); - } - } - } - - // Create a empty file if the directory is empty - for (String emptyPath : emptyPaths) { - numEmptyPaths = addInputPath(emptyPath, job, work, hiveScratchDir, numEmptyPaths, true, - oneAlias); - } - - // If the query references non-existent partitions - // We need to add a empty file, it is not acceptable to change the - // operator tree - // Consider the query: - // select * from (select count(1) from T union all select count(1) from - // T2) x; - // If T is empty and T2 contains 100 rows, the user expects: 0, 100 (2 - // rows) - if (path == null) { - numEmptyPaths = addInputPath(null, job, work, hiveScratchDir, numEmptyPaths, false, - oneAlias); - } - } - setInputPaths(job, pathsToAdd); - } - - private static void setInputPaths(JobConf job, List pathsToAdd) { - Path[] addedPaths = FileInputFormat.getInputPaths(job); - List toAddPathList = new ArrayList(); - if(addedPaths != null) { - for(Path added: addedPaths) { - toAddPathList.add(added); - } - } - for(String toAdd: pathsToAdd) { - toAddPathList.add(new Path(toAdd)); - } - FileInputFormat.setInputPaths(job, toAddPathList.toArray(new Path[0])); - } - - @Override - public StageType getType() { - return StageType.MAPRED; - } - - @Override - public String getName() { - return "MAPRED"; - } - - @Override - public void updateCounters(Counters ctrs, RunningJob rj) throws IOException { - for (Operator op : work.getAliasToWork().values()) { - op.updateCounters(ctrs); - } - if (work.getReducer() != null) { - work.getReducer().updateCounters(ctrs); - } - } - - @Override - public void logPlanProgress(SessionState ss) throws IOException { - ss.getHiveHistory().logPlanProgress(queryPlan); - } - - @Override - public void shutdown() { - super.shutdown(); - if (rj != null) { - try { - rj.killJob(); - } catch (Exception e) { - LOG.warn("failed to kill job " + rj.getID(), e); - } - rj = null; - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Throttle.java (working copy) @@ -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.hadoop.hive.ql.exec; - -import java.io.IOException; -import java.io.InputStream; -import java.net.URL; -import java.util.regex.Pattern; - -import org.apache.commons.logging.Log; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.mapred.JobConf; - -/** - * Intelligence to make clients wait if the cluster is in a bad state. - */ -public final class Throttle { - - // The percentage of maximum allocated memory that triggers GC - // on job tracker. This could be overridden thru the jobconf. - // The default is such that there is no throttling. - private static final int DEFAULT_MEMORY_GC_PERCENT = 100; - - // sleep this many seconds between each retry. - // This could be overridden thru the jobconf. - private static final int DEFAULT_RETRY_PERIOD = 60; - - /** - * Fetch http://tracker.om:/gc.jsp?threshold=period. - */ - public static void checkJobTracker(JobConf conf, Log LOG) { - - try { - byte[] buffer = new byte[1024]; - int threshold = conf.getInt("mapred.throttle.threshold.percent", DEFAULT_MEMORY_GC_PERCENT); - int retry = conf.getInt("mapred.throttle.retry.period", DEFAULT_RETRY_PERIOD); - - // If the threshold is 100 percent, then there is no throttling - if (threshold == 100) { - return; - } - - // This is the Job Tracker URL - String tracker = JobTrackerURLResolver.getURL(conf) + "/gc.jsp?threshold=" + threshold; - - while (true) { - // read in the first 1K characters from the URL - URL url = new URL(tracker); - LOG.debug("Throttle: URL " + tracker); - InputStream in = null; - try { - in = url.openStream(); - in.read(buffer); - in.close(); - in = null; - } finally { - IOUtils.closeStream(in); - } - String fetchString = new String(buffer); - - // fetch the xml tag xxx - Pattern dowait = Pattern.compile("", Pattern.CASE_INSENSITIVE - | Pattern.DOTALL | Pattern.MULTILINE); - String[] results = dowait.split(fetchString); - if (results.length != 2) { - throw new IOException("Throttle: Unable to parse response of URL " - + url + ". Get retuned " + fetchString); - } - dowait = Pattern.compile("", Pattern.CASE_INSENSITIVE - | Pattern.DOTALL | Pattern.MULTILINE); - results = dowait.split(results[1]); - if (results.length < 1) { - throw new IOException("Throttle: Unable to parse response of URL " - + url + ". Get retuned " + fetchString); - } - - // if the jobtracker signalled that the threshold is not exceeded, - // then we return immediately. - if (results[0].trim().compareToIgnoreCase("false") == 0) { - return; - } - - // The JobTracker has exceeded its threshold and is doing a GC. - // The client has to wait and retry. - LOG.warn("Job is being throttled because of resource crunch on the " - + "JobTracker. Will retry in " + retry + " seconds.."); - Thread.sleep(retry * 1000L); - } - } catch (Exception e) { - LOG.warn("Job is not being throttled. " + e); - } - } - - private Throttle() { - // prevent instantiation - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (working copy) @@ -97,6 +97,8 @@ import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryPlan; import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; +import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.io.ContentSummaryInputFormat; import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; @@ -1404,7 +1406,7 @@ jc = new JobConf(hconf); } else { // test code path - jc = new JobConf(hconf, ExecDriver.class); + jc = new JobConf(hconf); } HiveOutputFormat hiveOutputFormat = null; Class outputClass = null; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java (working copy) @@ -40,6 +40,8 @@ import org.apache.hadoop.hive.metastore.api.Order; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; +import org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask; import org.apache.hadoop.hive.ql.hooks.LineageInfo.DataContainer; import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/ExecReducer.java (working copy) @@ -1,324 +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.IOException; -import java.lang.management.ManagementFactory; -import java.lang.management.MemoryMXBean; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.exec.ExecMapper.reportStats; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.plan.MapredWork; -import org.apache.hadoop.hive.ql.plan.TableDesc; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeUtils; -import org.apache.hadoop.hive.serde2.io.ByteWritable; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.MapReduceBase; -import org.apache.hadoop.mapred.OutputCollector; -import org.apache.hadoop.mapred.Reducer; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.StringUtils; - -/** - * ExecReducer. - * - */ -public class ExecReducer extends MapReduceBase implements Reducer { - - private JobConf jc; - private OutputCollector oc; - private Operator reducer; - private Reporter rp; - private boolean abort = false; - private boolean isTagged = false; - private long cntr = 0; - private long nextCntr = 1; - - private static String[] fieldNames; - public static final Log l4j = LogFactory.getLog("ExecReducer"); - private boolean isLogInfoEnabled = false; - - // used to log memory usage periodically - private MemoryMXBean memoryMXBean; - - // TODO: move to DynamicSerDe when it's ready - private Deserializer inputKeyDeserializer; - // Input value serde needs to be an array to support different SerDe - // for different tags - private final SerDe[] inputValueDeserializer = new SerDe[Byte.MAX_VALUE]; - static { - ArrayList fieldNameArray = new ArrayList(); - for (Utilities.ReduceField r : Utilities.ReduceField.values()) { - fieldNameArray.add(r.toString()); - } - fieldNames = fieldNameArray.toArray(new String[0]); - } - - TableDesc keyTableDesc; - TableDesc[] valueTableDesc; - - ObjectInspector[] rowObjectInspector; - - @Override - public void configure(JobConf job) { - rowObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; - ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; - ObjectInspector keyObjectInspector; - - // Allocate the bean at the beginning - - memoryMXBean = ManagementFactory.getMemoryMXBean(); - l4j.info("maximum memory = " + memoryMXBean.getHeapMemoryUsage().getMax()); - - isLogInfoEnabled = l4j.isInfoEnabled(); - - try { - l4j.info("conf classpath = " - + Arrays.asList(((URLClassLoader) job.getClassLoader()).getURLs())); - l4j.info("thread classpath = " - + Arrays.asList(((URLClassLoader) Thread.currentThread() - .getContextClassLoader()).getURLs())); - } catch (Exception e) { - l4j.info("cannot get classpath: " + e.getMessage()); - } - jc = job; - MapredWork gWork = Utilities.getMapRedWork(job); - reducer = gWork.getReducer(); - reducer.setParentOperators(null); // clear out any parents as reducer is the - // root - isTagged = gWork.getNeedsTagging(); - try { - keyTableDesc = gWork.getKeyDesc(); - inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc - .getDeserializerClass(), null); - inputKeyDeserializer.initialize(null, keyTableDesc.getProperties()); - keyObjectInspector = inputKeyDeserializer.getObjectInspector(); - valueTableDesc = new TableDesc[gWork.getTagToValueDesc().size()]; - 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] = (SerDe) ReflectionUtils.newInstance( - valueTableDesc[tag].getDeserializerClass(), null); - inputValueDeserializer[tag].initialize(null, valueTableDesc[tag] - .getProperties()); - valueObjectInspector[tag] = inputValueDeserializer[tag] - .getObjectInspector(); - - ArrayList ois = new ArrayList(); - ois.add(keyObjectInspector); - ois.add(valueObjectInspector[tag]); - ois.add(PrimitiveObjectInspectorFactory.writableByteObjectInspector); - rowObjectInspector[tag] = ObjectInspectorFactory - .getStandardStructObjectInspector(Arrays.asList(fieldNames), ois); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - - MapredContext.init(false, new JobConf(jc)); - - // initialize reduce operator tree - try { - l4j.info(reducer.dump(0)); - reducer.initialize(jc, rowObjectInspector); - } 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); - } - } - } - - private Object keyObject; - private final Object[] valueObject = new Object[Byte.MAX_VALUE]; - - private BytesWritable groupKey; - - ArrayList row = new ArrayList(3); - ByteWritable tag = new ByteWritable(); - - public void reduce(Object key, Iterator values, OutputCollector output, - Reporter reporter) throws IOException { - if (reducer.getDone()) { - return; - } - if (oc == null) { - // propagete reporter and output collector to all operators - oc = output; - rp = reporter; - reducer.setOutputCollector(oc); - reducer.setReporter(rp); - MapredContext.get().setReporter(reporter); - } - - try { - BytesWritable keyWritable = (BytesWritable) key; - tag.set((byte) 0); - if (isTagged) { - // remove the tag - int size = keyWritable.getSize() - 1; - tag.set(keyWritable.get()[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 - l4j.trace("End Group"); - reducer.endGroup(); - } - - try { - keyObject = inputKeyDeserializer.deserialize(keyWritable); - } catch (Exception e) { - throw new HiveException( - "Hive Runtime Error: Unable to deserialize reduce input key from " - + Utilities.formatBinaryString(keyWritable.get(), 0, - keyWritable.getSize()) + " with properties " - + keyTableDesc.getProperties(), e); - } - - groupKey.set(keyWritable.get(), 0, keyWritable.getSize()); - l4j.trace("Start Group"); - reducer.startGroup(); - reducer.setGroupKeyObject(keyObject); - } - // System.err.print(keyObject.toString()); - while (values.hasNext()) { - BytesWritable valueWritable = (BytesWritable) values.next(); - // System.err.print(who.getHo().toString()); - try { - valueObject[tag.get()] = inputValueDeserializer[tag.get()] - .deserialize(valueWritable); - } catch (SerDeException e) { - throw new HiveException( - "Hive Runtime Error: Unable to deserialize reduce input value (tag=" - + tag.get() - + ") from " - + Utilities.formatBinaryString(valueWritable.get(), 0, - valueWritable.getSize()) + " with properties " - + valueTableDesc[tag.get()].getProperties(), e); - } - row.clear(); - row.add(keyObject); - row.add(valueObject[tag.get()]); - // The tag is not used any more, we should remove it. - row.add(tag); - if (isLogInfoEnabled) { - cntr++; - if (cntr == nextCntr) { - long used_memory = memoryMXBean.getHeapMemoryUsage().getUsed(); - l4j.info("ExecReducer: processing " + cntr - + " rows: used memory = " + used_memory); - nextCntr = getNextCntr(cntr); - } - } - try { - reducer.process(row, tag.get()); - } catch (Exception e) { - String rowString = null; - try { - rowString = SerDeUtils.getJSONString(row, rowObjectInspector[tag.get()]); - } catch (Exception e2) { - rowString = "[Error getting row data with exception " + - StringUtils.stringifyException(e2) + " ]"; - } - throw new HiveException("Hive Runtime Error while processing row (tag=" - + tag.get() + ") " + rowString, e); - } - } - - } 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 { - l4j.fatal(StringUtils.stringifyException(e)); - throw new RuntimeException(e); - } - } - } - - private long getNextCntr(long cntr) { - // 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 (cntr >= 1000000) { - return cntr + 1000000; - } - - return 10 * cntr; - } - - @Override - public void close() { - - // No row was processed - if (oc == null) { - l4j.trace("Close called no row"); - } - - try { - if (groupKey != null) { - // If a operator wants to do some work at the end of a group - l4j.trace("End Group"); - reducer.endGroup(); - } - if (isLogInfoEnabled) { - l4j.info("ExecReducer: processed " + cntr + " rows: used memory = " - + memoryMXBean.getHeapMemoryUsage().getUsed()); - } - - reducer.close(abort); - reportStats rps = new reportStats(rp); - reducer.preorderMap(rps); - - } catch (Exception e) { - if (!abort) { - // signal new failure to map-reduce - l4j.error("Hit error while closing operators - failing tree"); - throw new RuntimeException("Hive Runtime Error while closing operators: " - + e.getMessage(), e); - } - } finally { - MapredContext.close(); - } - } -} Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobDebugger.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.lang.Exception; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.File; import java.io.IOException; @@ -54,6 +54,15 @@ 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.exec.FetchOperator; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.HiveTotalOrderPartitioner; +import org.apache.hadoop.hive.ql.exec.JobCloseFeedBack; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.PartitionKeySampler; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter; import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveKey; @@ -93,8 +102,13 @@ import org.apache.log4j.varia.NullAppender; /** - * ExecDriver. + * ExecDriver is the central class in co-ordinating execution of any map-reduce task. + * It's main responsabilities are: * + * - Converting the plan (MapredWork) into a MR Job (JobConf) + * - Submitting a MR job to the cluster via JobClient and ExecHelper + * - Executing MR job in local execution mode (where applicable) + * */ public class ExecDriver extends Task implements Serializable, HadoopJobExecHook { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/Throttle.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/Throttle.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/Throttle.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.io.InputStream; @@ -76,18 +76,18 @@ // fetch the xml tag xxx Pattern dowait = Pattern.compile("", Pattern.CASE_INSENSITIVE - | Pattern.DOTALL | Pattern.MULTILINE); + | Pattern.DOTALL | Pattern.MULTILINE); String[] results = dowait.split(fetchString); if (results.length != 2) { throw new IOException("Throttle: Unable to parse response of URL " - + url + ". Get retuned " + fetchString); + + url + ". Get retuned " + fetchString); } dowait = Pattern.compile("", Pattern.CASE_INSENSITIVE - | Pattern.DOTALL | Pattern.MULTILINE); + | Pattern.DOTALL | Pattern.MULTILINE); results = dowait.split(results[1]); if (results.length < 1) { throw new IOException("Throttle: Unable to parse response of URL " - + url + ". Get retuned " + fetchString); + + url + ". Get retuned " + fetchString); } // if the jobtracker signalled that the threshold is not exceeded, @@ -99,7 +99,7 @@ // The JobTracker has exceeded its threshold and is doing a GC. // The client has to wait and retry. LOG.warn("Job is being throttled because of resource crunch on the " - + "JobTracker. Will retry in " + retry + " seconds.."); + + "JobTracker. Will retry in " + retry + " seconds.."); Thread.sleep(retry * 1000L); } } catch (Exception e) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java (working copy) @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.File; import java.io.IOException; @@ -43,6 +43,13 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; +import org.apache.hadoop.hive.ql.exec.BucketMatcher; +import org.apache.hadoop.hive.ql.exec.FetchOperator; +import org.apache.hadoop.hive.ql.exec.HashTableSinkOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.TableScanOperator; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; import org.apache.hadoop.hive.ql.exec.persistence.AbstractMapJoinKey; import org.apache.hadoop.hive.ql.exec.persistence.HashMapWrapper; @@ -63,6 +70,15 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.ReflectionUtils; + +/** + * MapredLocalTask represents any local work (i.e.: client side work) that hive needs to + * execute. E.g.: This is used for generating Hashtables for Mapjoins on the client + * before the Join is executed on the cluster. + * + * MapRedLocalTask does not actually execute the work in process, but rather generates + * a command using ExecDriver. ExecDriver is what will finally drive processing the records. + */ public class MapredLocalTask extends Task implements Serializable { private Map fetchOperators; @@ -202,7 +218,7 @@ // This will be used by hadoop only in unsecure(/non kerberos) mode HadoopShims shim = ShimLoader.getHadoopShims(); String endUserName = shim.getShortUserName(shim.getUGIForConf(job)); - console.printInfo("setting HADOOP_USER_NAME\t" + endUserName); + LOG.debug("setting HADOOP_USER_NAME\t" + endUserName); variables.put("HADOOP_USER_NAME", endUserName); if (variables.containsKey(HADOOP_OPTS_KEY)) { Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -28,6 +28,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.FetchOperator; +import org.apache.hadoop.hive.ql.exec.MapOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.UDFContext; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -38,8 +43,16 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.util.StringUtils; + /** - * ExecMapper. + * ExecMapper is the generic Map class for Hive. Together with ExecReducer it is + * the bridge between the map-reduce framework and the Hive operator pipeline at + * execution time. It's main responsabilities 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 ExecMapper extends MapReduceBase implements Mapper { @@ -50,7 +63,7 @@ private JobConf jc; private boolean abort = false; private Reporter rp; - public static final Log l4j = LogFactory.getLog("ExecMapper"); + public static final Log l4j = LogFactory.getLog(ExecMapper.class); private static boolean done; // used to log memory usage periodically @@ -93,7 +106,7 @@ localWork = mrwork.getMapLocalWork(); execContext.setLocalWork(localWork); - MapredContext.init(true, new JobConf(jc)); + UDFContext.init(); mo.setExecContext(execContext); mo.initializeLocalWork(jc); @@ -130,7 +143,6 @@ rp = reporter; mo.setOutputCollector(oc); mo.setReporter(rp); - MapredContext.get().setReporter(reporter); } // reset the execContext for each new row execContext.resetRow(); @@ -227,7 +239,7 @@ throw new RuntimeException("Hive Runtime Error while closing operators", e); } } finally { - MapredContext.close(); + UDFContext.close(); } } Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.io.Serializable; @@ -35,6 +35,9 @@ import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.MapRedStats; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.exec.TaskHandle; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.Operator.ProgressCounter; import org.apache.hadoop.hive.ql.history.HiveHistory.Keys; import org.apache.hadoop.hive.ql.plan.ReducerTimeStatsPerJob; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.File; import java.io.IOException; @@ -36,6 +36,8 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.Utilities; import org.apache.hadoop.hive.ql.exec.Utilities.StreamPrinter; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.MapredWork; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHook.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHook.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHook.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobTrackerURLResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobTrackerURLResolver.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/JobTrackerURLResolver.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.net.InetSocketAddress; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapperContext.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapperContext.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapperContext.java (working copy) @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.util.Map; import org.apache.commons.logging.Log; +import org.apache.hadoop.hive.ql.exec.FetchOperator; import org.apache.hadoop.hive.ql.io.IOContext; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; import org.apache.hadoop.mapred.JobConf; Index: ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java (revision 0) +++ ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecReducer.java (working copy) @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hive.ql.exec; +package org.apache.hadoop.hive.ql.exec.mr; import java.io.IOException; import java.lang.management.ManagementFactory; @@ -25,11 +25,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; -import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.exec.ExecMapper.reportStats; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.hive.ql.exec.UDFContext; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.reportStats; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.MapredWork; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -51,7 +53,14 @@ import org.apache.hadoop.util.StringUtils; /** - * ExecReducer. + * ExecReducer is the generic Reducer class for Hive. Together with ExecMapper it is + * the bridge between the map-reduce framework and the Hive operator pipeline at + * execution time. It's main responsabilities 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 ExecReducer extends MapReduceBase implements Reducer { @@ -145,7 +154,7 @@ throw new RuntimeException(e); } - MapredContext.init(false, new JobConf(jc)); + UDFContext.init(); // initialize reduce operator tree try { @@ -181,7 +190,6 @@ rp = reporter; reducer.setOutputCollector(oc); reducer.setReporter(rp); - MapredContext.get().setReporter(reporter); } try { @@ -318,7 +326,7 @@ + e.getMessage(), e); } } finally { - MapredContext.close(); + UDFContext.close(); } } } Index: ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveInputSplit.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveInputSplit.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveInputSplit.java (working copy) @@ -37,7 +37,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hive.ql.exec.ExecMapper; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.exec.Utilities; Index: ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/CombineHiveRecordReader.java (working copy) @@ -21,7 +21,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.exec.ExecMapper; +import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.CombineHiveInputSplit; import org.apache.hadoop.hive.shims.HadoopShims.InputSplitShim; import org.apache.hadoop.io.Writable; Index: ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/HiveRecordReader.java (working copy) @@ -20,7 +20,7 @@ import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.ExecMapper; +import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.JobConf; Index: ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/BlockMergeTask.java (working copy) @@ -35,11 +35,11 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHook; import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.Throttle; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHook; +import org.apache.hadoop.hive.ql.exec.mr.Throttle; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; import org.apache.hadoop.hive.ql.plan.api.StageType; Index: ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java (working copy) @@ -28,11 +28,11 @@ import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.DriverContext; import org.apache.hadoop.hive.ql.QueryPlan; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHelper; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHook; import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.Throttle; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHook; +import org.apache.hadoop.hive.ql.exec.mr.Throttle; import org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; import org.apache.hadoop.hive.ql.plan.api.StageType; Index: ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java (working copy) @@ -36,11 +36,11 @@ 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.exec.HadoopJobExecHelper; -import org.apache.hadoop.hive.ql.exec.HadoopJobExecHook; import org.apache.hadoop.hive.ql.exec.Task; -import org.apache.hadoop.hive.ql.exec.Throttle; import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper; +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHook; +import org.apache.hadoop.hive.ql.exec.mr.Throttle; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormatImpl; import org.apache.hadoop.hive.ql.metadata.HiveException; Index: ql/src/java/org/apache/hadoop/hive/ql/Driver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/Driver.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/Driver.java (working copy) @@ -46,7 +46,6 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Schema; import org.apache.hadoop.hive.ql.exec.ConditionalTask; -import org.apache.hadoop.hive.ql.exec.ExecDriver; import org.apache.hadoop.hive.ql.exec.FetchTask; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.TableScanOperator; @@ -193,7 +192,7 @@ public ClusterStatus getClusterStatus() throws Exception { ClusterStatus cs; try { - JobConf job = new JobConf(conf, ExecDriver.class); + JobConf job = new JobConf(conf); JobClient jc = new JobClient(job); cs = jc.getClusterStatus(); } catch (Exception e) { Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDF.java (working copy) @@ -21,8 +21,8 @@ import java.io.Closeable; import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.MapredContext; import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.exec.UDFContext; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.udf.UDFType; @@ -95,12 +95,12 @@ throws UDFArgumentException; /** - * Additionally setup GenericUDF with MapredContext before initializing. + * Additionally setup GenericUDF with UDFContext before initializing. * This is only called in runtime of MapRedTask. * * @param context context */ - public void configure(MapredContext context) { + public void configure(UDFContext context) { } /** Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTF.java (working copy) @@ -18,7 +18,7 @@ package org.apache.hadoop.hive.ql.udf.generic; -import org.apache.hadoop.hive.ql.exec.MapredContext; +import org.apache.hadoop.hive.ql.exec.UDFContext; import org.apache.hadoop.hive.ql.exec.UDFArgumentException; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -35,12 +35,12 @@ Collector collector = null; /** - * Additionally setup GenericUDTF with MapredContext before initializing. + * Additionally setup GenericUDTF with UDFContext before initializing. * This is only called in runtime of MapRedTask. * * @param context context */ - public void configure(MapredContext mapredContext) { + public void configure(UDFContext mapredContext) { } /** Index: ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFEvaluator.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFEvaluator.java (revision 1500544) +++ ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFEvaluator.java (working copy) @@ -21,7 +21,7 @@ import java.io.Closeable; import java.io.IOException; -import org.apache.hadoop.hive.ql.exec.MapredContext; +import org.apache.hadoop.hive.ql.exec.UDFContext; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.udf.UDFType; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -90,12 +90,12 @@ } /** - * Additionally setup GenericUDAFEvaluator with MapredContext before initializing. + * Additionally setup GenericUDAFEvaluator with UDFContext before initializing. * This is only called in runtime of MapRedTask. * * @param context context */ - public void configure(MapredContext mapredContext) { + public void configure(UDFContext mapredContext) { } /**