You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by su...@apache.org on 2015/07/20 22:12:12 UTC

[02/50] [abbrv] hive git commit: HIVE-11221: In Tez mode, alter table concatenate orc files can intermittently fail with NPE (Prasanth Jayachandran reviewed by Vikram Dixit)

HIVE-11221: In Tez mode, alter table concatenate orc files can intermittently fail with NPE (Prasanth Jayachandran reviewed by Vikram Dixit)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b61e6b52
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b61e6b52
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b61e6b52

Branch: refs/heads/spark
Commit: b61e6b52b54c9f8914aa6e4e042ff2921ce6a947
Parents: d89a7d1
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Fri Jul 10 00:19:37 2015 -0700
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Fri Jul 10 00:19:37 2015 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../ql/exec/tez/MergeFileRecordProcessor.java   |  42 +++--
 .../results/clientpositive/tez/orc_merge9.q.out | 186 +++++++++++++++++++
 3 files changed, 216 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b61e6b52/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 8773bd3..97715fc 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -139,6 +139,7 @@ minitez.query.files.shared=alter_merge_2_orc.q,\
   orc_merge6.q,\
   orc_merge7.q,\
   orc_merge8.q,\
+  orc_merge9.q,\
   orc_merge_incompat1.q,\
   orc_merge_incompat2.q,\
   orc_vectorization_ppd.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/b61e6b52/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java
index b95ab42..fce1523 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MergeFileRecordProcessor.java
@@ -17,9 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.exec.tez;
 
-import java.io.IOException;
+import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.concurrent.Callable;
 
 import org.apache.commons.logging.Log;
@@ -41,11 +40,14 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Input;
 import org.apache.tez.runtime.api.LogicalInput;
 import org.apache.tez.runtime.api.LogicalOutput;
 import org.apache.tez.runtime.api.ProcessorContext;
 import org.apache.tez.runtime.library.api.KeyValueReader;
 
+import com.google.common.collect.Lists;
+
 /**
  * Record processor for fast merging of files.
  */
@@ -219,22 +221,36 @@ public class MergeFileRecordProcessor extends RecordProcessor {
   }
 
   private MRInputLegacy getMRInput(Map<String, LogicalInput> inputs) throws Exception {
-    // there should be only one MRInput
-    MRInputLegacy theMRInput = null;
-    LOG.info("VDK: the inputs are: " + inputs);
-    for (Entry<String, LogicalInput> inp : inputs.entrySet()) {
-      if (inp.getValue() instanceof MRInputLegacy) {
-        if (theMRInput != null) {
+    LOG.info("The inputs are: " + inputs);
+
+    // start the mr input and wait for ready event. number of MRInput is expected to be 1
+    List<Input> li = Lists.newArrayList();
+    int numMRInputs = 0;
+    for (LogicalInput inp : inputs.values()) {
+      if (inp instanceof MRInputLegacy) {
+        numMRInputs++;
+        if (numMRInputs > 1) {
           throw new IllegalArgumentException("Only one MRInput is expected");
         }
-        // a better logic would be to find the alias
-        theMRInput = (MRInputLegacy) inp.getValue();
+        inp.start();
+        li.add(inp);
       } else {
-        throw new IOException("Expecting only one input of type MRInputLegacy. Found type: "
-            + inp.getClass().getCanonicalName());
+        throw new IllegalArgumentException("Expecting only one input of type MRInputLegacy." +
+            " Found type: " + inp.getClass().getCanonicalName());
       }
     }
-    theMRInput.init();
+
+    // typically alter table .. concatenate is run on only one partition/one table,
+    // so it doesn't matter if we wait for all inputs or any input to be ready.
+    processorContext.waitForAnyInputReady(li);
+
+    final MRInputLegacy theMRInput;
+    if (li.size() == 1) {
+      theMRInput = (MRInputLegacy) li.get(0);
+      theMRInput.init();
+    } else {
+      throw new IllegalArgumentException("MRInputs count is expected to be 1");
+    }
 
     return theMRInput;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/b61e6b52/ql/src/test/results/clientpositive/tez/orc_merge9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_merge9.q.out b/ql/src/test/results/clientpositive/tez/orc_merge9.q.out
new file mode 100644
index 0000000..bdf0fd3
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/orc_merge9.q.out
@@ -0,0 +1,186 @@
+PREHOOK: query: create table ts_merge (
+userid bigint,
+string1 string,
+subtype double,
+decimal1 decimal(38,18),
+ts timestamp
+) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ts_merge
+POSTHOOK: query: create table ts_merge (
+userid bigint,
+string1 string,
+subtype double,
+decimal1 decimal(38,18),
+ts timestamp
+) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ts_merge
+PREHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' overwrite into table ts_merge
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@ts_merge
+POSTHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' overwrite into table ts_merge
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@ts_merge
+PREHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table ts_merge
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@ts_merge
+POSTHOOK: query: load data local inpath '../../data/files/orc_split_elim.orc' into table ts_merge
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@ts_merge
+Found 2 items
+#### A masked pattern was here ####
+PREHOOK: query: select count(*) from ts_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ts_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from ts_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ts_merge
+#### A masked pattern was here ####
+50000
+PREHOOK: query: alter table ts_merge concatenate
+PREHOOK: type: ALTER_TABLE_MERGE
+PREHOOK: Input: default@ts_merge
+PREHOOK: Output: default@ts_merge
+POSTHOOK: query: alter table ts_merge concatenate
+POSTHOOK: type: ALTER_TABLE_MERGE
+POSTHOOK: Input: default@ts_merge
+POSTHOOK: Output: default@ts_merge
+PREHOOK: query: select count(*) from ts_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ts_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from ts_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ts_merge
+#### A masked pattern was here ####
+50000
+Found 1 items
+#### A masked pattern was here ####
+PREHOOK: query: -- incompatible merge test (stripe statistics missing)
+
+create table a_merge like alltypesorc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: -- incompatible merge test (stripe statistics missing)
+
+create table a_merge like alltypesorc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@a_merge
+PREHOOK: query: insert overwrite table a_merge select * from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: insert overwrite table a_merge select * from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@a_merge
+POSTHOOK: Lineage: a_merge.cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: a_merge.cboolean1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
+POSTHOOK: Lineage: a_merge.cboolean2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean2, type:boolean, comment:null), ]
+POSTHOOK: Lineage: a_merge.cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: a_merge.cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: a_merge.cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: a_merge.csmallint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: a_merge.cstring1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: a_merge.cstring2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cstring2, type:string, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctimestamp1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctimestamp2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctinyint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+PREHOOK: query: load data local inpath '../../data/files/alltypesorc' into table a_merge
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: load data local inpath '../../data/files/alltypesorc' into table a_merge
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@a_merge
+Found 2 items
+#### A masked pattern was here ####
+PREHOOK: query: select count(*) from a_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from a_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+24576
+PREHOOK: query: alter table a_merge concatenate
+PREHOOK: type: ALTER_TABLE_MERGE
+PREHOOK: Input: default@a_merge
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: alter table a_merge concatenate
+POSTHOOK: type: ALTER_TABLE_MERGE
+POSTHOOK: Input: default@a_merge
+POSTHOOK: Output: default@a_merge
+PREHOOK: query: select count(*) from a_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from a_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+24576
+Found 2 items
+#### A masked pattern was here ####
+PREHOOK: query: insert into table a_merge select * from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: insert into table a_merge select * from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+POSTHOOK: Output: default@a_merge
+POSTHOOK: Lineage: a_merge.cbigint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cbigint, type:bigint, comment:null), ]
+POSTHOOK: Lineage: a_merge.cboolean1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean1, type:boolean, comment:null), ]
+POSTHOOK: Lineage: a_merge.cboolean2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cboolean2, type:boolean, comment:null), ]
+POSTHOOK: Lineage: a_merge.cdouble SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cdouble, type:double, comment:null), ]
+POSTHOOK: Lineage: a_merge.cfloat SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cfloat, type:float, comment:null), ]
+POSTHOOK: Lineage: a_merge.cint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cint, type:int, comment:null), ]
+POSTHOOK: Lineage: a_merge.csmallint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:csmallint, type:smallint, comment:null), ]
+POSTHOOK: Lineage: a_merge.cstring1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cstring1, type:string, comment:null), ]
+POSTHOOK: Lineage: a_merge.cstring2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:cstring2, type:string, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctimestamp1 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp1, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctimestamp2 SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctimestamp2, type:timestamp, comment:null), ]
+POSTHOOK: Lineage: a_merge.ctinyint SIMPLE [(alltypesorc)alltypesorc.FieldSchema(name:ctinyint, type:tinyint, comment:null), ]
+Found 3 items
+#### A masked pattern was here ####
+PREHOOK: query: select count(*) from a_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from a_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+36864
+PREHOOK: query: alter table a_merge concatenate
+PREHOOK: type: ALTER_TABLE_MERGE
+PREHOOK: Input: default@a_merge
+PREHOOK: Output: default@a_merge
+POSTHOOK: query: alter table a_merge concatenate
+POSTHOOK: type: ALTER_TABLE_MERGE
+POSTHOOK: Input: default@a_merge
+POSTHOOK: Output: default@a_merge
+PREHOOK: query: select count(*) from a_merge
+PREHOOK: type: QUERY
+PREHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from a_merge
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@a_merge
+#### A masked pattern was here ####
+36864
+Found 2 items
+#### A masked pattern was here ####