Details
-
Sub-task
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
None
-
None
-
None
-
None
Description
the groupbyoperator's logic for firstrow passes null for all parameters.
see here.
this could obstruct compute_stats operations because it has a constant argument.
affected unit test: -Dtest=TestCliDriver -Dqfile=acid_insert_overwrite.q
exception backtrace:
java.lang.Exception: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row (tag=0) {"key":{},"value":{"_col0":{"columntype":"String","maxlength":0, "sumlength":0,"count":0,"countnulls":1,"bitvector":FM^@^@},"_col1":{"columntype":"Long","min":null,"max":null,"countnulls":1,"bitvector":FM^@^@}}} at org.apache.hadoop.mapred.LocalJobRunner$Job.runTasks(LocalJobRunner.java:492) ~[hadoop-mapreduce-client-common-3.1.0.jar:?] at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:559) [hadoop-mapreduce-client-common-3.1.0.jar:?] Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row (tag=0) {"key":{},"value":{"_col0":{"columntype":"String","maxlength":0,"sumlength ":0,"count":0,"countnulls":1,"bitvector":FM^@^@},"_col1":{"columntype":"Long","min":null,"max":null,"countnulls":1,"bitvector":FM^@^@}}} at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.reduce(ExecReducer.java:255) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:445) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:393) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.LocalJobRunner$Job$ReduceTaskRunnable.run(LocalJobRunner.java:347) ~[hadoop-mapreduce-client-common-3.1.0.jar:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_181] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_181] at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_181] Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row (tag=0) {"key":{},"value":{"_col0":{"columntype":"String","maxlength":0,"sumlength":0,"count":0,"countnulls":1 ,"bitvector":FM^@^@},"_col1":{"columntype":"Long","min":null,"max":null,"countnulls":1,"bitvector":FM^@^@}}} at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.reduce(ExecReducer.java:243) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:445) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:393) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.LocalJobRunner$Job$ReduceTaskRunnable.run(LocalJobRunner.java:347) ~[hadoop-mapreduce-client-common-3.1.0.jar:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_181] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_181] at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_181] Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.ClassCastException: org.apache.hadoop.hive.common.ndv.fm.FMSketch cannot be cast to org.apache.hadoop.hive.common.ndv.hll.HyperLogLog at org.apache.hadoop.hive.ql.exec.GroupByOperator.process(GroupByOperator.java:795) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.reduce(ExecReducer.java:234) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:445) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:393) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.LocalJobRunner$Job$ReduceTaskRunnable.run(LocalJobRunner.java:347) ~[hadoop-mapreduce-client-common-3.1.0.jar:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_181] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_181] at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_181] Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.common.ndv.fm.FMSketch cannot be cast to org.apache.hadoop.hive.common.ndv.hll.HyperLogLog at org.apache.hadoop.hive.common.ndv.hll.HyperLogLog.mergeEstimators(HyperLogLog.java:650) ~[hive-standalone-metastore-server-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFComputeStats$GenericUDAFStringStatsEvaluator.merge(GenericUDAFComputeStats.java:964) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.aggregate(GenericUDAFEvaluator.java:215) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.GroupByOperator.updateAggregations(GroupByOperator.java:641) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.GroupByOperator.processAggr(GroupByOperator.java:880) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.GroupByOperator.processKey(GroupByOperator.java:724) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.GroupByOperator.process(GroupByOperator.java:790) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.reduce(ExecReducer.java:234) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:445) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:393) ~[hadoop-mapreduce-client-core-3.1.0.jar:?] at org.apache.hadoop.mapred.LocalJobRunner$Job$ReduceTaskRunnable.run(LocalJobRunner.java:347) ~[hadoop-mapreduce-client-common-3.1.0.jar:?] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[?:1.8.0_181] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_181] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_181] at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_181]