You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/01/19 03:47:35 UTC

[1/4] hive git commit: HIVE-12758 : Parallel compilation: Operator::resetId() is not thread-safe (Sergey Shelukhin, reviewed by Gopal V)

Repository: hive
Updated Branches:
  refs/heads/master 4eab2dfe8 -> 88fceacaa


http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
index 7ab94a2..eaeb66b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.WindowsPathUtil;
 import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
@@ -77,6 +78,7 @@ public class TestExecDriver extends TestCase {
   private static final Path tmppath;
   private static Hive db;
   private static FileSystem fs;
+  private static CompilationOpContext ctx = null;
 
   static {
     try {
@@ -153,6 +155,7 @@ public class TestExecDriver extends TestCase {
   @Override
   protected void setUp() {
     mr = PlanUtils.getMapRedWork();
+    ctx = new CompilationOpContext();
   }
 
   public static void addMapWork(MapredWork mr, Table tbl, String alias, Operator<?> work) {
@@ -210,10 +213,9 @@ public class TestExecDriver extends TestCase {
   @SuppressWarnings("unchecked")
   private void populateMapPlan1(Table src) throws Exception {
 
-    Operator<FileSinkDesc> op2 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op2 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapplan1.out"), Utilities.defaultTd, true));
-    Operator<FilterDesc> op1 = OperatorFactory.get(getTestFilterDesc("key"),
-        op2);
+    Operator<FilterDesc> op1 = OperatorFactory.get(getTestFilterDesc("key"), op2);
 
     addMapWork(mr, src, "a", op1);
   }
@@ -221,7 +223,7 @@ public class TestExecDriver extends TestCase {
   @SuppressWarnings("unchecked")
   private void populateMapPlan2(Table src) throws Exception {
 
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapplan2.out"), Utilities.defaultTd, false));
 
     Operator<ScriptDesc> op2 = OperatorFactory.get(new ScriptDesc("cat",
@@ -244,7 +246,7 @@ public class TestExecDriver extends TestCase {
       outputColumns.add("_col" + i);
     }
     // map-side work
-    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("key")),
         Utilities.makeList(getStringColumn("value")), outputColumns, true,
         -1, 1, -1, AcidUtils.Operation.NOT_ACID));
@@ -257,7 +259,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan1.out"), Utilities.defaultTd, false));
 
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
@@ -276,7 +278,7 @@ public class TestExecDriver extends TestCase {
       outputColumns.add("_col" + i);
     }
     // map-side work
-    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("key")),
         Utilities
         .makeList(getStringColumn("key"), getStringColumn("value")),
@@ -290,7 +292,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op4 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan2.out"), Utilities.defaultTd, false));
 
     Operator<FilterDesc> op3 = OperatorFactory.get(getTestFilterDesc("0"), op4);
@@ -313,14 +315,14 @@ public class TestExecDriver extends TestCase {
       outputColumns.add("_col" + i);
     }
     // map-side work
-    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("key")),
         Utilities.makeList(getStringColumn("value")), outputColumns, true,
         Byte.valueOf((byte) 0), 1, -1, AcidUtils.Operation.NOT_ACID));
 
     addMapWork(mr, src, "a", op1);
 
-    Operator<ReduceSinkDesc> op2 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op2 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("key")),
         Utilities.makeList(getStringColumn("key")), outputColumns, true,
         Byte.valueOf((byte) 1), Integer.MAX_VALUE, -1, AcidUtils.Operation.NOT_ACID));
@@ -336,7 +338,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op2.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op4 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op4 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan3.out"), Utilities.defaultTd, false));
 
     Operator<SelectDesc> op5 = OperatorFactory.get(new SelectDesc(Utilities
@@ -356,7 +358,7 @@ public class TestExecDriver extends TestCase {
     for (int i = 0; i < 2; i++) {
       outputColumns.add("_col" + i);
     }
-    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("tkey")),
         Utilities.makeList(getStringColumn("tkey"),
         getStringColumn("tvalue")), outputColumns, false, -1, 1, -1, AcidUtils.Operation.NOT_ACID));
@@ -379,7 +381,7 @@ public class TestExecDriver extends TestCase {
     mr.setReduceWork(rWork);
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan4.out"), Utilities.defaultTd, false));
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
     cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0));
@@ -401,7 +403,7 @@ public class TestExecDriver extends TestCase {
     for (int i = 0; i < 2; i++) {
       outputColumns.add("_col" + i);
     }
-    Operator<ReduceSinkDesc> op0 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op0 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("0")), Utilities
         .makeList(getStringColumn("0"), getStringColumn("1")),
         outputColumns, false, -1, 1, -1, AcidUtils.Operation.NOT_ACID));
@@ -418,7 +420,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op0.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan5.out"), Utilities.defaultTd, false));
 
     List<ExprNodeDesc> cols = new ArrayList<ExprNodeDesc>();
@@ -436,7 +438,7 @@ public class TestExecDriver extends TestCase {
     for (int i = 0; i < 2; i++) {
       outputColumns.add("_col" + i);
     }
-    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(PlanUtils
+    Operator<ReduceSinkDesc> op1 = OperatorFactory.get(ctx, PlanUtils
         .getReduceSinkDesc(Utilities.makeList(getStringColumn("tkey")),
         Utilities.makeList(getStringColumn("tkey"),
         getStringColumn("tvalue")), outputColumns, false, -1, 1, -1, AcidUtils.Operation.NOT_ACID));
@@ -460,7 +462,7 @@ public class TestExecDriver extends TestCase {
     rWork.getTagToValueDesc().add(op1.getConf().getValueSerializeInfo());
 
     // reduce side work
-    Operator<FileSinkDesc> op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator
+    Operator<FileSinkDesc> op3 = OperatorFactory.get(ctx, new FileSinkDesc(new Path(tmpdir + File.separator
         + "mapredplan6.out"), Utilities.defaultTd, false));
 
     Operator<FilterDesc> op2 = OperatorFactory.get(getTestFilterDesc("0"), op3);
@@ -478,7 +480,7 @@ public class TestExecDriver extends TestCase {
     MapRedTask mrtask = new MapRedTask();
     DriverContext dctx = new DriverContext ();
     mrtask.setWork(mr);
-    mrtask.initialize(conf, null, dctx);
+    mrtask.initialize(conf, null, dctx, null);
     int exitVal =  mrtask.execute(dctx);
 
     if (exitVal != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
index 68c598a..a8d7c9c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestFileSinkOperator.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.ValidTxnList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
@@ -293,7 +294,8 @@ public class TestFileSinkOperator {
     if (txnId > 0) desc.setTransactionId(txnId);
     if (writeType != AcidUtils.Operation.NOT_ACID) desc.setTransactionId(1L);
 
-    FileSinkOperator op = (FileSinkOperator)OperatorFactory.get(FileSinkDesc.class);
+    FileSinkOperator op = (FileSinkOperator)OperatorFactory.get(
+        new CompilationOpContext(), FileSinkDesc.class);
     op.setConf(desc);
     op.initialize(jc, new ObjectInspector[]{inspector});
     return op;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
index 000b46b..2ccb05a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestOperators.java
@@ -31,6 +31,7 @@ import junit.framework.TestCase;
 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.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.io.IOContextMap;
 import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory;
@@ -155,7 +156,7 @@ public class TestOperators extends TestCase {
    */
   public void testScriptOperatorEnvVarsProcessing() throws Throwable {
     try {
-      ScriptOperator scriptOperator = new ScriptOperator();
+      ScriptOperator scriptOperator = new ScriptOperator(new CompilationOpContext());
 
       //Environment Variables name
       assertEquals("a_b_c", scriptOperator.safeEnvVarName("a.b.c"));
@@ -194,7 +195,7 @@ public class TestOperators extends TestCase {
   }
 
   public void testScriptOperatorBlacklistedEnvVarsProcessing() {
-    ScriptOperator scriptOperator = new ScriptOperator();
+    ScriptOperator scriptOperator = new ScriptOperator(new CompilationOpContext());
 
     Configuration hconf = new JobConf(ScriptOperator.class);
 
@@ -229,7 +230,7 @@ public class TestOperators extends TestCase {
         outputCols.add("_col" + i);
       }
       SelectDesc selectCtx = new SelectDesc(earr, outputCols);
-      Operator<SelectDesc> op = OperatorFactory.get(SelectDesc.class);
+      Operator<SelectDesc> op = OperatorFactory.get(new CompilationOpContext(), SelectDesc.class);
       op.setConf(selectCtx);
 
       // scriptOperator to echo the output of the select
@@ -245,8 +246,7 @@ public class TestOperators extends TestCase {
 
       // Collect operator to observe the output of the script
       CollectDesc cd = new CollectDesc(Integer.valueOf(10));
-      CollectOperator cdop = (CollectOperator) OperatorFactory.getAndMakeChild(
-          cd, sop);
+      CollectOperator cdop = (CollectOperator) OperatorFactory.getAndMakeChild(cd, sop);
 
       op.initialize(new JobConf(TestOperators.class),
           new ObjectInspector[]{r[0].oi});
@@ -308,12 +308,13 @@ public class TestOperators extends TestCase {
       pathToPartitionInfo.put("hdfs:///testDir", pd);
 
       // initialize aliasToWork
+      CompilationOpContext ctx = new CompilationOpContext();
       CollectDesc cd = new CollectDesc(Integer.valueOf(1));
       CollectOperator cdop1 = (CollectOperator) OperatorFactory
-          .get(CollectDesc.class);
+          .get(ctx, CollectDesc.class);
       cdop1.setConf(cd);
       CollectOperator cdop2 = (CollectOperator) OperatorFactory
-          .get(CollectDesc.class);
+          .get(ctx, CollectDesc.class);
       cdop2.setConf(cd);
       LinkedHashMap<String, Operator<? extends OperatorDesc>> aliasToWork =
         new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
@@ -327,7 +328,7 @@ public class TestOperators extends TestCase {
       mrwork.getMapWork().setAliasToWork(aliasToWork);
 
       // get map operator and initialize it
-      MapOperator mo = new MapOperator();
+      MapOperator mo = new MapOperator(new CompilationOpContext());
       mo.initializeAsRoot(hconf, mrwork.getMapWork());
 
       Text tw = new Text();

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
index 64db486..06e5e07 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestPlan.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.parse.TypeCheckProcFactory;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -58,7 +59,7 @@ public class TestPlan extends TestCase {
           .getFuncExprNodeDesc("==", expr1, expr2);
 
       FilterDesc filterCtx = new FilterDesc(filterExpr, false);
-      Operator<FilterDesc> op = OperatorFactory.get(FilterDesc.class);
+      Operator<FilterDesc> op = OperatorFactory.get(new CompilationOpContext(), FilterDesc.class);
       op.setConf(filterCtx);
 
       ArrayList<String> aliasList = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
index 3968c50..d3bb84d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorFilterOperator.java
@@ -24,6 +24,7 @@ import java.util.List;
 import junit.framework.Assert;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.FilterExprAndExpr;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongColEqualDoubleScalar;
@@ -91,7 +92,7 @@ public class TestVectorFilterOperator {
     VectorizationContext vc = new VectorizationContext("name", columns);
     FilterDesc fdesc = new FilterDesc();
     fdesc.setPredicate(col1Expr);
-    return new VectorFilterOperator(vc, fdesc);
+    return new VectorFilterOperator(new CompilationOpContext(), vc, fdesc);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
index fdcf103..451947b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
@@ -38,6 +38,7 @@ import java.util.Set;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeCaptureOutputOperator;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromConcat;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromLongIterables;
@@ -186,9 +187,10 @@ public class TestVectorGroupByOperator {
     float treshold = 100.0f*1024.0f/maxMemory;
     desc.setMemoryThreshold(treshold);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     this.outputRowCount = 0;
@@ -1729,9 +1731,10 @@ public class TestVectorGroupByOperator {
     desc.setAggregators(aggs);
     desc.setKeys(keysDesc);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
     out.setOutputInspector(new FakeCaptureOutputOperator.OutputInspector() {
 
@@ -1843,9 +1846,10 @@ public class TestVectorGroupByOperator {
     keysDesc.add(keyExp);
     desc.setKeys(keysDesc);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
     out.setOutputInspector(new FakeCaptureOutputOperator.OutputInspector() {
 
@@ -2239,9 +2243,10 @@ public class TestVectorGroupByOperator {
 
     GroupByDesc desc = buildGroupByDescCountStar (ctx);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit: data) {
@@ -2269,10 +2274,10 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildGroupByDescType(ctx, "count", "A", TypeInfoFactory.longTypeInfo);
     VectorGroupByDesc vectorDesc = desc.getVectorDesc();
     vectorDesc.setIsReduceMergePartial(true);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
-
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit: data) {
@@ -2301,9 +2306,10 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildGroupByDescType(ctx, aggregateName, "A",
         TypeInfoFactory.stringTypeInfo);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit: data) {
@@ -2332,9 +2338,10 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc =
         buildGroupByDescType(ctx, aggregateName, "A", TypeInfoFactory.getDecimalTypeInfo(30, 4));
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit : data) {
@@ -2364,9 +2371,10 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildGroupByDescType (ctx, aggregateName, "A",
         TypeInfoFactory.doubleTypeInfo);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit: data) {
@@ -2394,9 +2402,10 @@ public class TestVectorGroupByOperator {
 
     GroupByDesc desc = buildGroupByDescType(ctx, aggregateName, "A", TypeInfoFactory.longTypeInfo);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(null, null);
 
     for (VectorizedRowBatch unit: data) {
@@ -2428,13 +2437,13 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildKeyGroupByDesc (ctx, aggregateName, "Value",
         TypeInfoFactory.longTypeInfo, "Key", TypeInfoFactory.longTypeInfo);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
     out.setOutputInspector(new FakeCaptureOutputOperator.OutputInspector() {
 
-      private int rowIndex;
       private String aggregateName;
       private HashMap<Object,Object> expected;
       private Set<Object> keys;
@@ -2494,9 +2503,10 @@ public class TestVectorGroupByOperator {
     GroupByDesc desc = buildKeyGroupByDesc (ctx, aggregateName, "Value",
        dataTypeInfo, "Key", TypeInfoFactory.stringTypeInfo);
 
-    VectorGroupByOperator vgo = new VectorGroupByOperator(ctx, desc);
+    CompilationOpContext cCtx = new CompilationOpContext();
+    VectorGroupByOperator vgo = new VectorGroupByOperator(cCtx, ctx, desc);
 
-    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(vgo);
+    FakeCaptureOutputOperator out = FakeCaptureOutputOperator.addCaptureOutputChild(cCtx, vgo);
     vgo.initialize(hconf, null);
     out.setOutputInspector(new FakeCaptureOutputOperator.OutputInspector() {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java
index aa1d89d..428f456 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorLimitOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromObjectIterables;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.LimitDesc;
@@ -63,7 +64,7 @@ public class TestVectorLimitOperator {
 
     // Create limit desc with limit value
     LimitDesc ld = new LimitDesc(limit);
-    VectorLimitOperator lo = new VectorLimitOperator(null, ld);
+    VectorLimitOperator lo = new VectorLimitOperator(new CompilationOpContext(), null, ld);
     lo.initialize(new Configuration(), null);
 
     // Process the batch

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
index 59961c5..779177a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSelectOperator.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.util.VectorizedRowGroupGenUtil;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
@@ -47,9 +48,9 @@ public class TestVectorSelectOperator {
 
     private static final long serialVersionUID = 1L;
 
-    public ValidatorVectorSelectOperator(VectorizationContext ctxt, OperatorDesc conf)
-        throws HiveException {
-      super(ctxt, conf);
+    public ValidatorVectorSelectOperator(CompilationOpContext ctx,
+        VectorizationContext ctxt, OperatorDesc conf) throws HiveException {
+      super(ctx, ctxt, conf);
       initializeOp(null);
     }
 
@@ -114,7 +115,8 @@ public class TestVectorSelectOperator {
     outputColNames.add("_col1");
     selDesc.setOutputColumnNames(outputColNames);
 
-    ValidatorVectorSelectOperator vso = new ValidatorVectorSelectOperator(vc, selDesc);
+    ValidatorVectorSelectOperator vso = new ValidatorVectorSelectOperator(
+        new CompilationOpContext(), vc, selDesc);
 
     VectorizedRowBatch vrg = VectorizedRowGroupGenUtil.getVectorizedRowBatch(
         VectorizedRowBatch.DEFAULT_SIZE, 4, 17);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeCaptureOutputOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeCaptureOutputOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeCaptureOutputOperator.java
index 7454b01..74e077b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeCaptureOutputOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeCaptureOutputOperator.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -54,9 +55,9 @@ public class FakeCaptureOutputOperator extends Operator<FakeCaptureOutputDesc>
 
   private transient List<Object> rows;
 
-  public static FakeCaptureOutputOperator addCaptureOutputChild(
+  public static FakeCaptureOutputOperator addCaptureOutputChild(CompilationOpContext ctx,
       Operator<? extends OperatorDesc> op) {
-    FakeCaptureOutputOperator out = new FakeCaptureOutputOperator();
+    FakeCaptureOutputOperator out = new FakeCaptureOutputOperator(ctx);
     List<Operator<? extends OperatorDesc>> listParents =
         new ArrayList<Operator<? extends OperatorDesc>>(1);
     listParents.add(op);
@@ -73,6 +74,15 @@ public class FakeCaptureOutputOperator extends Operator<FakeCaptureOutputDesc>
     return rows;
   }
 
+  /** Kryo ctor. */
+  protected FakeCaptureOutputOperator() {
+    super();
+  }
+
+  public FakeCaptureOutputOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void initializeOp(Configuration conf) throws HiveException {
     super.initializeOp(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorDataSourceOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorDataSourceOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorDataSourceOperator.java
index 3bea307..d06d214 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorDataSourceOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorDataSourceOperator.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -44,7 +45,8 @@ public class FakeVectorDataSourceOperator extends Operator<FakeVectorDataSourceO
   public static FakeVectorDataSourceOperator addFakeVectorDataSourceParent(
       Iterable<VectorizedRowBatch> source,
       Operator<? extends OperatorDesc> op) {
-    FakeVectorDataSourceOperator parent = new FakeVectorDataSourceOperator(source);
+    FakeVectorDataSourceOperator parent = new FakeVectorDataSourceOperator(
+        new CompilationOpContext(), source);
     List<Operator<? extends OperatorDesc>> listParents =
         new ArrayList<Operator<? extends OperatorDesc>>(1);
     listParents.add(parent);
@@ -56,11 +58,21 @@ public class FakeVectorDataSourceOperator extends Operator<FakeVectorDataSourceO
     return parent;
   }
 
-  public FakeVectorDataSourceOperator(
+  public FakeVectorDataSourceOperator(CompilationOpContext ctx,
     Iterable<VectorizedRowBatch> source) {
+    super(ctx);
     this.source = source;
   }
 
+  /** Kryo ctor. */
+  protected FakeVectorDataSourceOperator() {
+    super();
+  }
+
+  public FakeVectorDataSourceOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void initializeOp(Configuration conf) throws HiveException {
     super.initializeOp(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
index f9a0e79..5628959 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/physical/TestVectorizer.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.*;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator;
@@ -104,7 +105,7 @@ public class TestVectorizer {
     grpByKeys.add(colExprB);
     desc.setKeys(grpByKeys);
 
-    GroupByOperator gbyOp = new GroupByOperator();
+    GroupByOperator gbyOp = new GroupByOperator(new CompilationOpContext());
     gbyOp.setConf(desc);
 
     Vectorizer v = new Vectorizer();
@@ -181,7 +182,7 @@ public class TestVectorizer {
   */
   @Test
   public void testValidateMapJoinOperator() {
-    MapJoinOperator map = new MapJoinOperator();
+    MapJoinOperator map = new MapJoinOperator(new CompilationOpContext());
     MapJoinDesc mjdesc = new MapJoinDesc();
     
     prepareAbstractMapJoin(map, mjdesc);
@@ -197,7 +198,7 @@ public class TestVectorizer {
   */
   @Test
   public void testValidateSMBJoinOperator() {
-      SMBMapJoinOperator map = new SMBMapJoinOperator();
+      SMBMapJoinOperator map = new SMBMapJoinOperator(new CompilationOpContext());
       SMBJoinDesc mjdesc = new SMBJoinDesc();
       
       prepareAbstractMapJoin(map, mjdesc);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/parse/TestGenTezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestGenTezWork.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestGenTezWork.java
index 9e5db23..be51edc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestGenTezWork.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestGenTezWork.java
@@ -28,6 +28,7 @@ import java.util.Collections;
 import java.util.LinkedHashMap;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
@@ -85,11 +86,12 @@ public class TestGenTezWork {
       }
     });
 
-    fs = new FileSinkOperator();
+    CompilationOpContext cCtx = new CompilationOpContext();
+    fs = new FileSinkOperator(cCtx);
     fs.setConf(new FileSinkDesc());
-    rs = new ReduceSinkOperator();
+    rs = new ReduceSinkOperator(cCtx);
     rs.setConf(new ReduceSinkDesc());
-    ts = new TableScanOperator();
+    ts = new TableScanOperator(cCtx);
     ts.setConf(new TableScanDesc(null));
     ts.getChildOperators().add(rs);
     rs.getParentOperators().add(ts);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
index 2b33da1..d795324 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestUpdateDeleteSemanticAnalyzer.java
@@ -297,7 +297,7 @@ public class TestUpdateDeleteSemanticAnalyzer {
         sem.getFetchTask(), astTree, sem, true, false, false, false, false, false, null);
     ExplainTask task = new ExplainTask();
     task.setWork(work);
-    task.initialize(conf, plan, null);
+    task.initialize(conf, plan, null, null);
     task.execute(null);
     FSDataInputStream in = fs.open(tmp);
     StringBuilder builder = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java b/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
index f7a45f7..5e29d13 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/testutil/BaseScalarUdfTest.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import junit.framework.TestCase;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.CollectOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -83,7 +84,7 @@ public abstract class BaseScalarUdfTest extends TestCase {
     List<ExprNodeDesc> expressionList = getExpressionList();
     SelectDesc selectCtx = new SelectDesc(expressionList,
         OperatorTestUtils.createOutputColumnNames(expressionList));
-    Operator<SelectDesc> op = OperatorFactory.get(SelectDesc.class);
+    Operator<SelectDesc> op = OperatorFactory.get(new CompilationOpContext(), SelectDesc.class);
     op.setConf(selectCtx);
     CollectDesc cd = new CollectDesc(Integer.valueOf(10));
     CollectOperator cdop = (CollectOperator) OperatorFactory.getAndMakeChild(cd, op);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/results/clientpositive/auto_join0.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join0.q.out b/ql/src/test/results/clientpositive/auto_join0.q.out
index 10859b8..957a458 100644
--- a/ql/src/test/results/clientpositive/auto_join0.q.out
+++ b/ql/src/test/results/clientpositive/auto_join0.q.out
@@ -1,5 +1,5 @@
-Warning: Map Join MAPJOIN[45][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[33][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[12][tables = [src1, src2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: -- SORT_QUERY_RESULTS
 
@@ -254,8 +254,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[45][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[33][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[12][tables = [src1, src2]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: select sum(hash(a.k1,a.v1,a.k2, a.v2))
 from (

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out b/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
index ff3b9f6..f1707eb 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_cross_product_check_2.q.out
@@ -454,8 +454,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[43][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[36][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[31][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[30][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[20][tables = [, ]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain select * from 
 (select A.key from A group by key) ss join 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/results/clientpositive/cross_product_check_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cross_product_check_2.q.out b/ql/src/test/results/clientpositive/cross_product_check_2.q.out
index 625a465..a36560f 100644
--- a/ql/src/test/results/clientpositive/cross_product_check_2.q.out
+++ b/ql/src/test/results/clientpositive/cross_product_check_2.q.out
@@ -446,8 +446,8 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Map Join MAPJOIN[47][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
-Warning: Map Join MAPJOIN[40][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Stage-7:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[34][bigTable=?] in task 'Stage-6:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[23][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain select * from 
 (select A.key from A group by key) ss join 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
index 9cba332..279843b 100644
--- a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.7.out
@@ -490,7 +490,7 @@ POSTHOOK: Input: default@src_5
 199	val_199
 199	val_199
 2	val_2
-Warning: Map Join MAPJOIN[62][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[55][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[31][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain
 from src b 
@@ -801,7 +801,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-Warning: Map Join MAPJOIN[62][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[55][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[31][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: from src b 
 INSERT OVERWRITE TABLE src_4 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.8.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.8.out b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.8.out
index 4c1d06c..899723f 100644
--- a/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.8.out
+++ b/ql/src/test/results/clientpositive/subquery_multiinsert.q.java1.8.out
@@ -103,16 +103,14 @@ STAGE PLANS:
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
             predicate: (_col0 = 0) (type: boolean)
-            Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: 0 (type: bigint)
-              outputColumnNames: _col0
-              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: bigint)
+                keys: 0 (type: bigint)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -139,7 +137,7 @@ STAGE PLANS:
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -254,21 +252,21 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
-              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
-                    Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -333,10 +331,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-10:MAPRED
 RUN: Stage-2:MAPRED
 RUN: Stage-3:MAPRED
@@ -492,7 +490,7 @@ POSTHOOK: Input: default@src_5
 199	val_199
 199	val_199
 2	val_2
-Warning: Map Join MAPJOIN[107][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[55][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[31][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain
 from src b 
@@ -566,16 +564,14 @@ STAGE PLANS:
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           Filter Operator
             predicate: (_col0 = 0) (type: boolean)
-            Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: 0 (type: bigint)
-              outputColumnNames: _col0
-              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
-                keys: _col0 (type: bigint)
+                keys: 0 (type: bigint)
                 mode: hash
                 outputColumnNames: _col0
-                Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+                Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -605,6 +601,7 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: b
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Map Join Operator
               condition map:
                    Left Semi Join 0 to 1
@@ -612,6 +609,7 @@ STAGE PLANS:
                 0 
                 1 
               outputColumnNames: _col0, _col1
+              Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
                 table:
@@ -716,16 +714,16 @@ STAGE PLANS:
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: ((key > '9') and value is not null) (type: boolean)
-              Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: string), value (type: string)
                 outputColumnNames: _col0, _col1
-                Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   keys: _col0 (type: string), _col1 (type: string)
                   mode: hash
                   outputColumnNames: _col0, _col1
-                  Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   HashTable Sink Operator
                     keys:
                       0 key (type: string), value (type: string)
@@ -786,7 +784,7 @@ STAGE PLANS:
           TableScan
             Reduce Output Operator
               sort order: 
-              Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+              Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -803,7 +801,7 @@ STAGE PLANS:
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
 
-Warning: Map Join MAPJOIN[107][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
+Warning: Map Join MAPJOIN[55][bigTable=b] in task 'Stage-13:MAPRED' is a cross product
 Warning: Shuffle Join JOIN[31][tables = [b, sq_2_notin_nullcheck]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: from src b 
 INSERT OVERWRITE TABLE src_4 
@@ -837,10 +835,10 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@src_4
 POSTHOOK: Output: default@src_5
-POSTHOOK: Lineage: src_4.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_4.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.key EXPRESSION [(src)b.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: src_5.value EXPRESSION [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_4.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.key SIMPLE [(src)b.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_5.value SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
 RUN: Stage-10:MAPRED
 RUN: Stage-14:CONDITIONAL
 RUN: Stage-17:MAPREDLOCAL


[2/4] hive git commit: HIVE-12758 : Parallel compilation: Operator::resetId() is not thread-safe (Sergey Shelukhin, reviewed by Gopal V)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
index e4107ff..8ed1ed4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterStringOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -67,12 +68,18 @@ public class VectorMapJoinOuterStringOperator extends VectorMapJoinOuterGenerate
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinOuterStringOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOuterStringOperator() {
     super();
   }
 
-  public VectorMapJoinOuterStringOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinOuterStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinOuterStringOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
index a79a649..7bdd11a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
@@ -25,6 +25,7 @@ import java.util.Properties;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TerminalOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -143,13 +144,18 @@ public abstract class VectorReduceSinkCommonOperator extends TerminalOperator<Re
 
   //---------------------------------------------------------------------------
 
-  public VectorReduceSinkCommonOperator() {
+  /** Kryo ctor. */
+  protected VectorReduceSinkCommonOperator() {
     super();
   }
 
-  public VectorReduceSinkCommonOperator(VectorizationContext vContext, OperatorDesc conf)
-          throws HiveException {
-    super();
+  public VectorReduceSinkCommonOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorReduceSinkCommonOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
 
     ReduceSinkDesc desc = (ReduceSinkDesc) conf;
     this.conf = desc;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java
index cec5660..325f773 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkLongOperator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.reducesink;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.keyseries.VectorKeySeriesLongSerialized;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -49,13 +50,18 @@ public class VectorReduceSinkLongOperator extends VectorReduceSinkCommonOperator
   // Pass-thru constructors.
   //
 
-  public VectorReduceSinkLongOperator() {
+  /** Kryo ctor. */
+  protected VectorReduceSinkLongOperator() {
     super();
   }
 
-  public VectorReduceSinkLongOperator(VectorizationContext vContext, OperatorDesc conf)
-          throws HiveException {
-    super(vContext, conf);
+  public VectorReduceSinkLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorReduceSinkLongOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java
index a4ef66b..2027187 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkMultiKeyOperator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.reducesink;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.keyseries.VectorKeySeriesMultiSerialized;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -45,13 +46,18 @@ public class VectorReduceSinkMultiKeyOperator extends VectorReduceSinkCommonOper
   // Pass-thru constructors.
   //
 
-  public VectorReduceSinkMultiKeyOperator() {
+  /** Kryo ctor. */
+  protected VectorReduceSinkMultiKeyOperator() {
     super();
   }
 
-  public VectorReduceSinkMultiKeyOperator(VectorizationContext vContext, OperatorDesc conf)
-          throws HiveException {
-    super(vContext, conf);
+  public VectorReduceSinkMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorReduceSinkMultiKeyOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java
index b6cb527..b655e6e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkStringOperator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.reducesink;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.keyseries.VectorKeySeriesBytesSerialized;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -48,13 +49,18 @@ public class VectorReduceSinkStringOperator extends VectorReduceSinkCommonOperat
   // Pass-thru constructors.
   //
 
-  public VectorReduceSinkStringOperator() {
+  /** Kryo ctor. */
+  protected VectorReduceSinkStringOperator() {
     super();
   }
 
-  public VectorReduceSinkStringOperator(VectorizationContext vContext, OperatorDesc conf)
-          throws HiveException {
-    super(vContext, conf);
+  public VectorReduceSinkStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorReduceSinkStringOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
index 38b6b5d..0be8b3c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java
@@ -135,7 +135,7 @@ public class ATSHook implements ExecuteWithHookContext {
             );
               @SuppressWarnings("unchecked")
               ExplainTask explain = (ExplainTask) TaskFactory.get(work, conf);
-              explain.initialize(conf, plan, null);
+              explain.initialize(conf, plan, null, null);
               String query = plan.getQueryStr();
               JSONObject explainPlan = explain.getJSONPlan(null, work);
               String logID = conf.getLogIdVar(SessionState.get().getSessionId());

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
index e23a969..82629c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/merge/MergeFileTask.java
@@ -22,6 +22,7 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -60,8 +61,8 @@ public class MergeFileTask extends Task<MergeFileWork> implements Serializable,
 
   @Override
   public void initialize(HiveConf conf, QueryPlan queryPlan,
-      DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+      DriverContext driverContext, CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
     job = new JobConf(conf, MergeFileTask.class);
     jobExecHelper = new HadoopJobExecHelper(job, this.console, this, this);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java
index 829a9f6..71371a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/stats/PartialScanTask.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.common.StatsSetupConst.StatDB;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -84,8 +85,8 @@ public class PartialScanTask extends Task<PartialScanWork> implements
 
   @Override
   public void initialize(HiveConf conf, QueryPlan queryPlan,
-      DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+      DriverContext driverContext, CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
     job = new JobConf(conf, PartialScanTask.class);
     jobExecHelper = new HadoopJobExecHelper(job, this.console, this, this);
   }
@@ -351,7 +352,7 @@ public class PartialScanTask extends Task<PartialScanWork> implements
     PartialScanWork mergeWork = new PartialScanWork(inputPaths);
     DriverContext driverCxt = new DriverContext();
     PartialScanTask taskExec = new PartialScanTask();
-    taskExec.initialize(hiveConf, null, driverCxt);
+    taskExec.initialize(hiveConf, null, driverCxt, new CompilationOpContext());
     taskExec.setWork(mergeWork);
     int ret = taskExec.execute(driverCxt);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java
index 34c067a..bc21da0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateTask.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 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.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -59,8 +60,8 @@ public class ColumnTruncateTask extends Task<ColumnTruncateWork> implements Seri
 
   @Override
   public void initialize(HiveConf conf, QueryPlan queryPlan,
-      DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+      DriverContext driverContext, CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
     job = new JobConf(conf, ColumnTruncateTask.class);
     jobExecHelper = new HadoopJobExecHelper(job, this.console, this, this);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
index 1da0dda..b57dc77 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
@@ -185,7 +185,7 @@ abstract public class AbstractSMBJoinProc extends AbstractBucketJoinProc impleme
         par.getChildOperators().add(index, smbJop);
       }
       else {
-        DummyStoreOperator dummyStoreOp = new DummyStoreOperator();
+        DummyStoreOperator dummyStoreOp = new DummyStoreOperator(par.getCompilationOpContext());
         par.getChildOperators().add(index, dummyStoreOp);
 
         List<Operator<? extends OperatorDesc>> childrenOps =

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index ea89cf0..00bc193 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -245,8 +245,9 @@ public class ConvertJoinMapJoin implements NodeProcessor {
     }
 
     CommonMergeJoinOperator mergeJoinOp =
-        (CommonMergeJoinOperator) OperatorFactory.get(new CommonMergeJoinDesc(numBuckets,
-            mapJoinConversionPos, mapJoinDesc), joinOp.getSchema());
+        (CommonMergeJoinOperator) OperatorFactory.get(joinOp.getCompilationOpContext(),
+            new CommonMergeJoinDesc(numBuckets, mapJoinConversionPos, mapJoinDesc),
+            joinOp.getSchema());
     OpTraits opTraits =
         new OpTraits(joinOp.getOpTraits().getBucketColNames(), numBuckets, joinOp.getOpTraits()
             .getSortCols());
@@ -295,7 +296,8 @@ public class ConvertJoinMapJoin implements NodeProcessor {
         }
 
         // insert the dummy store operator here
-        DummyStoreOperator dummyStoreOp = new TezDummyStoreOperator();
+        DummyStoreOperator dummyStoreOp = new TezDummyStoreOperator(
+            mergeJoinOp.getCompilationOpContext());
         dummyStoreOp.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>());
         dummyStoreOp.setChildOperators(new ArrayList<Operator<? extends OperatorDesc>>());
         dummyStoreOp.getChildOperators().add(mergeJoinOp);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
index 8d232ac..60240bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/DynamicPartitionPruningOptimization.java
@@ -315,8 +315,8 @@ public class DynamicPartitionPruningOptimization implements NodeProcessor {
             new ArrayList<AggregationDesc>(), false, groupByMemoryUsage, memoryThreshold,
             null, false, 0, true);
 
-    GroupByOperator groupByOp =
-        (GroupByOperator) OperatorFactory.getAndMakeChild(groupBy, selectOp);
+    GroupByOperator groupByOp = (GroupByOperator) OperatorFactory.getAndMakeChild(
+        groupBy, selectOp);
 
     Map<String, ExprNodeDesc> colMap = new HashMap<String, ExprNodeDesc>();
     colMap.put(outputNames.get(0), groupByExpr);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
index 2160e01..9c979be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
@@ -204,7 +204,7 @@ public class GenMRTableScan1 implements NodeProcessor {
     // partial scan task
     DriverContext driverCxt = new DriverContext();
     Task<PartialScanWork> psTask = TaskFactory.get(scanWork, parseCtx.getConf());
-    psTask.initialize(parseCtx.getConf(), null, driverCxt);
+    psTask.initialize(parseCtx.getConf(), null, driverCxt, op.getCompilationOpContext());
     psTask.setWork(scanWork);
 
     // task dependency

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
index 9cbd496..3500711 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
@@ -993,9 +994,10 @@ public final class GenMapRedUtils {
     return mrWork;
   }
 
-  public static TableScanOperator createTemporaryTableScanOperator(RowSchema rowSchema) {
+  public static TableScanOperator createTemporaryTableScanOperator(
+      CompilationOpContext ctx, RowSchema rowSchema) {
     TableScanOperator tableScanOp =
-        (TableScanOperator) OperatorFactory.get(new TableScanDesc(null), rowSchema);
+        (TableScanOperator) OperatorFactory.get(ctx, new TableScanDesc(null), rowSchema);
     // Set needed columns for this dummy TableScanOperator
     List<Integer> neededColumnIds = new ArrayList<Integer>();
     List<String> neededColumnNames = new ArrayList<String>();
@@ -1038,7 +1040,7 @@ public final class GenMapRedUtils {
           HiveConf.ConfVars.COMPRESSINTERMEDIATETYPE));
     }
     Operator<? extends OperatorDesc> fileSinkOp = OperatorFactory.get(
-            desc, parent.getSchema());
+        parent.getCompilationOpContext(), desc, parent.getSchema());
 
     // Connect parent to fileSinkOp
     parent.replaceChild(child, fileSinkOp);
@@ -1046,7 +1048,7 @@ public final class GenMapRedUtils {
 
     // Create a dummy TableScanOperator for the file generated through fileSinkOp
     TableScanOperator tableScanOp = createTemporaryTableScanOperator(
-            parent.getSchema());
+        parent.getCompilationOpContext(), parent.getSchema());
 
     // Connect this TableScanOperator to child.
     tableScanOp.setChildOperators(Utilities.makeList(child));
@@ -1272,8 +1274,8 @@ public final class GenMapRedUtils {
 
     // Create a TableScan operator
     RowSchema inputRS = fsInput.getSchema();
-    TableScanOperator tsMerge =
-        GenMapRedUtils.createTemporaryTableScanOperator(inputRS);
+    TableScanOperator tsMerge = GenMapRedUtils.createTemporaryTableScanOperator(
+        fsInput.getCompilationOpContext(), inputRS);
 
     // Create a FileSink operator
     TableDesc ts = (TableDesc) fsInputDesc.getTableInfo().clone();
@@ -1324,7 +1326,7 @@ public final class GenMapRedUtils {
             fsInputDesc.getTableInfo().getInputFileFormatClass().equals(OrcInputFormat.class))) {
 
       cplan = GenMapRedUtils.createMergeTask(fsInputDesc, finalName,
-          dpCtx != null && dpCtx.getNumDPCols() > 0);
+          dpCtx != null && dpCtx.getNumDPCols() > 0, fsInput.getCompilationOpContext());
       if (conf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("tez")) {
         work = new TezWork(conf.getVar(HiveConf.ConfVars.HIVEQUERYID));
         cplan.setName("File Merge");
@@ -1560,12 +1562,13 @@ public final class GenMapRedUtils {
    *
    * @param fsInputDesc
    * @param finalName
+   * @param ctx
    * @param inputFormatClass
    * @return MergeWork if table is stored as RCFile or ORCFile,
    *         null otherwise
    */
-  public static MapWork createMergeTask(FileSinkDesc fsInputDesc,
-      Path finalName, boolean hasDynamicPartitions) throws SemanticException {
+  public static MapWork createMergeTask(FileSinkDesc fsInputDesc, Path finalName,
+      boolean hasDynamicPartitions, CompilationOpContext ctx) throws SemanticException {
 
     Path inputDir = fsInputDesc.getFinalDirName();
     TableDesc tblDesc = fsInputDesc.getTableInfo();
@@ -1621,7 +1624,7 @@ public final class GenMapRedUtils {
     int lbLevel = work.getListBucketingCtx() == null ? 0 :
       work.getListBucketingCtx().calculateListBucketingLevel();
     fmd.setListBucketingDepth(lbLevel);
-    mergeOp = OperatorFactory.get(fmd);
+    mergeOp = OperatorFactory.get(ctx, fmd);
     aliasToWork.put(inputDir.toString(), mergeOp);
     work.setAliasToWork(aliasToWork);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
index 8f3ce60..3233157 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GroupByOptimizer.java
@@ -571,8 +571,8 @@ public class GroupByOptimizer extends Transform {
         colName.add(cInfo.getInternalName());
         columnExprMap.put(cInfo.getInternalName(), column);
       }
-      return OperatorFactory.getAndMakeChild(new SelectDesc(columns, colName), new RowSchema(currOp
-          .getSchema().getSignature()), columnExprMap, parentOp);
+      return OperatorFactory.getAndMakeChild(new SelectDesc(columns, colName),
+          new RowSchema(currOp.getSchema().getSignature()), columnExprMap, parentOp);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index 2e3f930..e577e19 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -80,6 +80,8 @@ import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
 
+import com.clearspring.analytics.util.Lists;
+
 /**
  * Implementation of one of the rule-based map join optimization. User passes hints to specify
  * map-joins and during this optimization, all user specified map joins are converted to MapJoins -
@@ -376,7 +378,8 @@ public class MapJoinProcessor extends Transform {
     RowSchema outputRS = op.getSchema();
 
     MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory.getAndMakeChild(
-        mapJoinDescriptor, new RowSchema(outputRS.getSignature()), op.getParentOperators());
+        op.getCompilationOpContext(), mapJoinDescriptor,
+        new RowSchema(outputRS.getSignature()), op.getParentOperators());
 
     mapJoinOp.getConf().setReversedExprs(op.getConf().getReversedExprs());
     Map<String, ExprNodeDesc> colExprMap = op.getColumnExprMap();
@@ -438,7 +441,8 @@ public class MapJoinProcessor extends Transform {
     RowSchema joinRS = smbJoinOp.getSchema();
     // The mapjoin has the same schema as the join operator
     MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory.getAndMakeChild(
-        mapJoinDesc, joinRS, new ArrayList<Operator<? extends OperatorDesc>>());
+        smbJoinOp.getCompilationOpContext(), mapJoinDesc, joinRS,
+        new ArrayList<Operator<? extends OperatorDesc>>());
 
     // change the children of the original join operator to point to the map
     // join operator
@@ -601,8 +605,8 @@ public class MapJoinProcessor extends Transform {
 
     SelectDesc select = new SelectDesc(exprs, outputs, false);
 
-    SelectOperator sel = (SelectOperator) OperatorFactory.getAndMakeChild(select,
-            new RowSchema(outputRS), input);
+    SelectOperator sel = (SelectOperator) OperatorFactory.getAndMakeChild(
+        select, new RowSchema(outputRS), input);
 
     sel.setColumnExprMap(colExprMap);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index d5c3a2d..c38c6d7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -297,6 +297,7 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
           LOG.debug("Cloning reduce sink for multi-child broadcast edge");
           // we've already set this one up. Need to clone for the next work.
           r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
+              parentRS.getCompilationOpContext(),
               (ReduceSinkDesc) parentRS.getConf().clone(),
               new RowSchema(parentRS.getSchema()),
               parentRS.getParentOperators());
@@ -334,7 +335,8 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     // create an new operator: HashTableDummyOperator, which share the table desc
     HashTableDummyDesc desc = new HashTableDummyDesc();
     @SuppressWarnings("unchecked")
-    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(desc);
+    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(
+        parentRS.getCompilationOpContext(), desc);
     TableDesc tbl;
 
     // need to create the correct table descriptor for key/value

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
index f71fd3f..fbcb779 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchAggregation.java
@@ -121,7 +121,8 @@ public class SimpleFetchAggregation extends Transform {
 
       // Create a file sink operator for this file name
       FileSinkDesc desc = new FileSinkDesc(fileName, tsDesc, false);
-      FileSinkOperator newFS = (FileSinkOperator) OperatorFactory.get(desc, parent.getSchema());
+      FileSinkOperator newFS = (FileSinkOperator) OperatorFactory.get(
+          parent.getCompilationOpContext(), desc, parent.getSchema());
 
       newFS.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>());
       newFS.getParentOperators().add(parent);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
index 719dfff..b5ceb14 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SimpleFetchOptimizer.java
@@ -474,7 +474,8 @@ public class SimpleFetchOptimizer extends Transform {
 
   public static ListSinkOperator replaceFSwithLS(Operator<?> fileSink, String nullFormat) {
     ListSinkDesc desc = new ListSinkDesc(nullFormat);
-    ListSinkOperator sink = (ListSinkOperator) OperatorFactory.get(desc);
+    ListSinkOperator sink = (ListSinkOperator) OperatorFactory.get(
+        fileSink.getCompilationOpContext(), desc);
 
     sink.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>());
     Operator<? extends OperatorDesc> parent = fileSink.getParentOperators().get(0);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
index 3f1b277..5e30910 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SkewJoinOptimizer.java
@@ -219,7 +219,7 @@ public class SkewJoinOptimizer extends Transform {
       oplist.add(currOp);
       oplist.add(currOpClone);
       Operator<? extends OperatorDesc> unionOp =
-        OperatorFactory.getAndMakeChild(
+        OperatorFactory.getAndMakeChild(currOp.getCompilationOpContext(),
           new UnionDesc(), new RowSchema(currOp.getSchema().getSignature()), oplist);
 
       // Introduce a select after the union
@@ -228,8 +228,7 @@ public class SkewJoinOptimizer extends Transform {
       unionList.add(unionOp);
 
       Operator<? extends OperatorDesc> selectUnionOp =
-        OperatorFactory.getAndMakeChild(
-          new SelectDesc(true),
+        OperatorFactory.getAndMakeChild(currOp.getCompilationOpContext(), new SelectDesc(true),
           new RowSchema(unionOp.getSchema().getSignature()), unionList);
 
       // add the finalOp after the union
@@ -475,8 +474,7 @@ public class SkewJoinOptimizer extends Transform {
 
       Operator<FilterDesc> filter = OperatorFactory.getAndMakeChild(
         new FilterDesc(filterExpr, false),
-        new RowSchema(tableScanOp.getSchema().getSignature()),
-        tableScanOp);
+        new RowSchema(tableScanOp.getSchema().getSignature()), tableScanOp);
       OperatorFactory.makeChild(filter, currChild);
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveGBOpConvUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveGBOpConvUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveGBOpConvUtil.java
index a6d809b..7fbf8cd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveGBOpConvUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveGBOpConvUtil.java
@@ -685,9 +685,9 @@ public class HiveGBOpConvUtil {
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(PlanUtils
         .getReduceSinkDesc(reduceKeys, keyLength, reduceValues, gbInfo.distColIndices,
-            outputKeyColumnNames, outputValueColumnNames, true, -1,
-            getNumPartFieldsForMapSideRS(gbInfo), getParallelismForMapSideRS(gbInfo),
-            AcidUtils.Operation.NOT_ACID), new RowSchema(colInfoLst), mapGB);
+        outputKeyColumnNames, outputValueColumnNames, true, -1, getNumPartFieldsForMapSideRS(
+        gbInfo), getParallelismForMapSideRS(gbInfo), AcidUtils.Operation.NOT_ACID),
+        new RowSchema(colInfoLst), mapGB);
 
     rsOp.setColumnExprMap(colExprMap);
 
@@ -945,8 +945,8 @@ public class HiveGBOpConvUtil {
         && !(gbInfo.gbPhysicalPipelineMode == HIVEGBPHYSICALMODE.MAP_SIDE_GB_SKEW_GBKEYS_OR_DIST_UDAF_PRESENT);
     Operator rsGBOp = OperatorFactory.getAndMakeChild(new GroupByDesc(gbMode, outputColNames,
         gbKeys, aggregations, gbInfo.groupByMemoryUsage, gbInfo.memoryThreshold, gbInfo.grpSets,
-        includeGrpSetInGBDesc, groupingSetsColPosition,
-        gbInfo.containsDistinctAggr), new RowSchema(colInfoLst), rs);
+        includeGrpSetInGBDesc, groupingSetsColPosition, gbInfo.containsDistinctAggr),
+        new RowSchema(colInfoLst), rs);
 
     rsGBOp.setColumnExprMap(colExprMap);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index 00f1acb..c79b1be 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -253,7 +253,8 @@ public class HiveOpConverter {
     tsd.setNeededColumns(neededColumnNames);
 
     // 2. Setup TableScan
-    TableScanOperator ts = (TableScanOperator) OperatorFactory.get(tsd, new RowSchema(colInfos));
+    TableScanOperator ts = (TableScanOperator) OperatorFactory.get(
+        semanticAnalyzer.getOpContext(), tsd, new RowSchema(colInfos));
 
     topOps.put(tableAlias, ts);
 
@@ -488,8 +489,7 @@ public class HiveOpConverter {
           && semanticAnalyzer.getQB().getParseInfo() != null)
         this.semanticAnalyzer.getQB().getParseInfo().setOuterQueryLimit(limit);
       ArrayList<ColumnInfo> cinfoLst = createColInfos(resultOp);
-      resultOp = OperatorFactory.getAndMakeChild(limitDesc,
-          new RowSchema(cinfoLst), resultOp);
+      resultOp = OperatorFactory.getAndMakeChild(limitDesc, new RowSchema(cinfoLst), resultOp);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Generated " + resultOp + " with row schema: [" + resultOp.getSchema() + "]");
@@ -516,8 +516,8 @@ public class HiveOpConverter {
             filterRel.getCluster().getTypeFactory()));
     FilterDesc filDesc = new FilterDesc(filCondExpr, false);
     ArrayList<ColumnInfo> cinfoLst = createColInfos(inputOpAf.inputs.get(0));
-    FilterOperator filOp = (FilterOperator) OperatorFactory.getAndMakeChild(filDesc, new RowSchema(
-        cinfoLst), inputOpAf.inputs.get(0));
+    FilterOperator filOp = (FilterOperator) OperatorFactory.getAndMakeChild(filDesc,
+        new RowSchema(cinfoLst), inputOpAf.inputs.get(0));
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Generated " + filOp + " with row schema: [" + filOp.getSchema() + "]");
@@ -569,8 +569,8 @@ public class HiveOpConverter {
         children[i] = genInputSelectForUnion(op, cinfoLst);
       }
     }
-    Operator<? extends OperatorDesc> unionOp = OperatorFactory.getAndMakeChild(unionDesc,
-        new RowSchema(cinfoLst), children);
+    Operator<? extends OperatorDesc> unionOp = OperatorFactory.getAndMakeChild(
+        semanticAnalyzer.getOpContext(), unionDesc, new RowSchema(cinfoLst), children);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("Generated " + unionOp + " with row schema: [" + unionOp.getSchema() + "]");
@@ -662,8 +662,8 @@ public class HiveOpConverter {
           unparseTranslator);
       RowResolver ptfOpRR = ptfDesc.getFuncDef().getOutputShape().getRr();
 
-      Operator<?> ptfOp = OperatorFactory.getAndMakeChild(ptfDesc,
-          new RowSchema(ptfOpRR.getColumnInfos()), selectOp);
+      Operator<?> ptfOp = OperatorFactory.getAndMakeChild(
+          ptfDesc, new RowSchema(ptfOpRR.getColumnInfos()), selectOp);
 
       if (LOG.isDebugEnabled()) {
         LOG.debug("Generated " + ptfOp + " with row schema: [" + ptfOp.getSchema() + "]");
@@ -725,8 +725,8 @@ public class HiveOpConverter {
     SelectDesc selectDesc = new SelectDesc(new ArrayList<ExprNodeDesc>(descriptors.values()),
         new ArrayList<String>(descriptors.keySet()));
     ArrayList<ColumnInfo> cinfoLst = createColInfosSubset(input, keepColNames);
-    SelectOperator selectOp = (SelectOperator) OperatorFactory.getAndMakeChild(selectDesc,
-        new RowSchema(cinfoLst), rsOp);
+    SelectOperator selectOp = (SelectOperator) OperatorFactory.getAndMakeChild(
+        selectDesc, new RowSchema(cinfoLst), rsOp);
     selectOp.setColumnExprMap(descriptors);
 
     if (LOG.isDebugEnabled()) {
@@ -823,8 +823,8 @@ public class HiveOpConverter {
           partitionCols, order, numReducers, acidOperation);
     }
 
-    ReduceSinkOperator rsOp = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(rsDesc,
-        new RowSchema(outputColumns), input);
+    ReduceSinkOperator rsOp = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
+        rsDesc, new RowSchema(outputColumns), input);
 
     List<String> keyColNames = rsDesc.getOutputKeyColumnNames();
     for (int i = 0; i < keyColNames.size(); i++) {
@@ -976,8 +976,8 @@ public class HiveOpConverter {
     desc.setReversedExprs(reversedExprs);
     desc.setFilterMap(filterMap);
 
-    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(desc, new RowSchema(
-        outputColumns), childOps);
+    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(
+        childOps[0].getCompilationOpContext(), desc, new RowSchema(outputColumns), childOps);
     joinOp.setColumnExprMap(colExprMap);
     joinOp.setPosToAliasMap(posToAliasMap);
 
@@ -1241,8 +1241,8 @@ public class HiveOpConverter {
       columnExprMap.put(uInfo.getInternalName(), column);
     }
     if (needSelectOp) {
-      return OperatorFactory.getAndMakeChild(new SelectDesc(columns, colName), new RowSchema(
-          uColumnInfo), columnExprMap, origInputOp);
+      return OperatorFactory.getAndMakeChild(new SelectDesc(
+          columns, colName), new RowSchema(uColumnInfo), columnExprMap, origInputOp);
     } else {
       return origInputOp;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java
index 315a650..c8aa48c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/QueryPlanTreeTransformation.java
@@ -27,6 +27,7 @@ import java.util.Set;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.DemuxOperator;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -123,7 +124,11 @@ public class QueryPlanTreeTransformation {
     Map<ReduceSinkOperator, Integer> bottomRSToNewTag =
         new HashMap<ReduceSinkOperator, Integer>();
     int newTag = 0;
+    CompilationOpContext opCtx = null;
     for (ReduceSinkOperator rsop: bottomReduceSinkOperators) {
+      if (opCtx == null) {
+        opCtx = rsop.getCompilationOpContext();
+      }
       rsop.getConf().setNumReducers(numReducers);
       bottomRSToNewTag.put(rsop, newTag);
       parentRSsOfDemux.add(rsop);
@@ -150,7 +155,7 @@ public class QueryPlanTreeTransformation {
             childIndexToOriginalNumParents,
             keysSerializeInfos,
             valuessSerializeInfos);
-    Operator<? extends OperatorDesc> demuxOp = OperatorFactory.get(demuxDesc);
+    Operator<? extends OperatorDesc> demuxOp = OperatorFactory.get(opCtx, demuxDesc);
     demuxOp.setChildOperators(childrenOfDemux);
     demuxOp.setParentOperators(parentRSsOfDemux);
     for (Operator<? extends OperatorDesc> child: childrenOfDemux) {
@@ -199,7 +204,7 @@ public class QueryPlanTreeTransformation {
               CorrelationUtilities.getSingleParent(childOP, true);
           parentsOfMux.add(parentOp);
           Operator<? extends OperatorDesc> mux = OperatorFactory.get(
-              new MuxDesc(parentsOfMux));
+              childOP.getCompilationOpContext(), new MuxDesc(parentsOfMux));
           mux.setChildOperators(Utilities.makeList(childOP));
           mux.setParentOperators(parentsOfMux);
           childOP.setParentOperators(Utilities.makeList(mux));
@@ -229,7 +234,8 @@ public class QueryPlanTreeTransformation {
             }
           }
           MuxDesc muxDesc = new MuxDesc(siblingOPs);
-          Operator<? extends OperatorDesc> mux = OperatorFactory.get(muxDesc);
+          Operator<? extends OperatorDesc> mux = OperatorFactory.get(
+              rsop.getCompilationOpContext(), muxDesc);
           mux.setChildOperators(Utilities.makeList(childOP));
           mux.setParentOperators(parentsOfMux);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
index e9ca5fa..9ea0857 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinTaskDispatcher.java
@@ -177,7 +177,6 @@ public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher impleme
     MapRedTask newTask = (MapRedTask) TaskFactory.get(newWork, physicalContext
         .getParseContext().getConf());
     JoinOperator newJoinOp = getJoinOp(newTask);
-
     // optimize this newWork given the big table position
     MapJoinProcessor.genMapJoinOpAndLocalWork(physicalContext.getParseContext().getConf(),
         newWork, newJoinOp, bigTablePosition);
@@ -513,8 +512,7 @@ public class CommonJoinTaskDispatcher extends AbstractJoinTaskDispatcher impleme
         taskToAliases.put(newTask, aliases);
       }
     } catch (Exception e) {
-      e.printStackTrace();
-      throw new SemanticException("Generate Map Join Task Error: " + e.getMessage());
+      throw new SemanticException("Generate Map Join Task Error: " + e.getMessage(), e);
     }
 
     // insert current common join task to conditional task

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
index a71c474..9fbbd4c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenMRSkewJoinProcessor.java
@@ -259,7 +259,8 @@ public final class GenMRSkewJoinProcessor {
       Operator<? extends OperatorDesc>[] parentOps = new TableScanOperator[tags.length];
       for (int k = 0; k < tags.length; k++) {
         Operator<? extends OperatorDesc> ts =
-            GenMapRedUtils.createTemporaryTableScanOperator(rowSchemaList.get((byte)k));
+            GenMapRedUtils.createTemporaryTableScanOperator(
+                joinOp.getCompilationOpContext(), rowSchemaList.get((byte)k));
         ((TableScanOperator)ts).setTableDesc(tableDescList.get((byte)k));
         parentOps[k] = ts;
       }
@@ -310,8 +311,8 @@ public final class GenMRSkewJoinProcessor {
       newPlan.setMapRedLocalWork(localPlan);
 
       // construct a map join and set it as the child operator of tblScan_op
-      MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory
-          .getAndMakeChild(mapJoinDescriptor, (RowSchema) null, parentOps);
+      MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory.getAndMakeChild(
+          joinOp.getCompilationOpContext(), mapJoinDescriptor, (RowSchema) null, parentOps);
       // change the children of the original join operator to point to the map
       // join operator
       List<Operator<? extends OperatorDesc>> childOps = cloneJoinOp

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
index 4805162..11ec07a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/GenSparkSkewJoinProcessor.java
@@ -232,8 +232,8 @@ public class GenSparkSkewJoinProcessor {
       // create N TableScans
       Operator<? extends OperatorDesc>[] parentOps = new TableScanOperator[tags.length];
       for (int k = 0; k < tags.length; k++) {
-        Operator<? extends OperatorDesc> ts =
-            GenMapRedUtils.createTemporaryTableScanOperator(rowSchemaList.get((byte) k));
+        Operator<? extends OperatorDesc> ts = GenMapRedUtils.createTemporaryTableScanOperator(
+            joinOp.getCompilationOpContext(), rowSchemaList.get((byte) k));
         ((TableScanOperator) ts).setTableDesc(tableDescList.get((byte) k));
         parentOps[k] = ts;
       }
@@ -249,7 +249,7 @@ public class GenSparkSkewJoinProcessor {
       mapJoinDescriptor.setNullSafes(joinDescriptor.getNullSafes());
       // temporarily, mark it as child of all the TS
       MapJoinOperator mapJoinOp = (MapJoinOperator) OperatorFactory
-          .getAndMakeChild(mapJoinDescriptor, null, parentOps);
+          .getAndMakeChild(joinOp.getCompilationOpContext(), mapJoinDescriptor, null, parentOps);
 
       // clone the original join operator, and replace it with the MJ
       // this makes sure MJ has the same downstream operator plan as the original join
@@ -360,7 +360,8 @@ public class GenSparkSkewJoinProcessor {
     Preconditions.checkArgument(tableScan.getChildOperators().size() == 1
         && tableScan.getChildOperators().get(0) instanceof MapJoinOperator);
     HashTableDummyDesc desc = new HashTableDummyDesc();
-    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(desc);
+    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(
+        tableScan.getCompilationOpContext(), desc);
     dummyOp.getConf().setTbl(tableScan.getTableDesc());
     MapJoinOperator mapJoinOp = (MapJoinOperator) tableScan.getChildOperators().get(0);
     mapJoinOp.replaceParent(tableScan, dummyOp);
@@ -373,8 +374,8 @@ public class GenSparkSkewJoinProcessor {
     // mapjoin should not be affected by join reordering
     mjDesc.resetOrder();
     SparkHashTableSinkDesc hashTableSinkDesc = new SparkHashTableSinkDesc(mjDesc);
-    SparkHashTableSinkOperator hashTableSinkOp =
-        (SparkHashTableSinkOperator) OperatorFactory.get(hashTableSinkDesc);
+    SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator)OperatorFactory.get(
+            tableScan.getCompilationOpContext(), hashTableSinkDesc);
     int[] valueIndex = mjDesc.getValueIndex(tag);
     if (valueIndex != null) {
       List<ExprNodeDesc> newValues = new ArrayList<ExprNodeDesc>();

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
index df598e7..9ca815c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/LocalMapJoinProcFactory.java
@@ -141,7 +141,7 @@ public final class LocalMapJoinProcFactory {
 
       HashTableSinkDesc hashTableSinkDesc = new HashTableSinkDesc(mapJoinDesc);
       HashTableSinkOperator hashTableSinkOp = (HashTableSinkOperator) OperatorFactory
-          .get(hashTableSinkDesc);
+          .get(mapJoinOp.getCompilationOpContext(), hashTableSinkDesc);
 
       // get the last operator for processing big tables
       int bigTable = mapJoinDesc.getPosBigTable();
@@ -203,7 +203,8 @@ public final class LocalMapJoinProcFactory {
 
         // create new operator: HashTable DummyOperator, which share the table desc
         HashTableDummyDesc desc = new HashTableDummyDesc();
-        HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(desc);
+        HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(
+            parent.getCompilationOpContext(), desc);
         TableDesc tbl;
 
         if (parent.getSchema() == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 1629a5d..f462bd6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -1894,7 +1894,8 @@ public class Vectorizer implements PhysicalPlanResolver {
       break;
     }
 
-    vectorOp = OperatorFactory.getVectorOperator(opClass, op.getConf(), vContext);
+    vectorOp = OperatorFactory.getVectorOperator(
+        opClass, op.getCompilationOpContext(), op.getConf(), vContext);
     LOG.info("Vectorizer vectorizeOperator map join class " + vectorOp.getClass().getSimpleName());
 
     boolean minMaxEnabled = HiveConf.getBoolVar(hiveConf,
@@ -2030,7 +2031,8 @@ public class Vectorizer implements PhysicalPlanResolver {
     vectorDesc.setReduceSinkKeyType(reduceSinkKeyType);
     vectorDesc.setVectorReduceSinkInfo(vectorReduceSinkInfo);
 
-    vectorOp = OperatorFactory.getVectorOperator(opClass, op.getConf(), vContext);
+    vectorOp = OperatorFactory.getVectorOperator(
+        opClass, op.getCompilationOpContext(), op.getConf(), vContext);
     LOG.info("Vectorizer vectorizeOperator reduce sink class " + vectorOp.getClass().getSimpleName());
 
     return vectorOp;
@@ -2179,7 +2181,8 @@ public class Vectorizer implements PhysicalPlanResolver {
               opClass = VectorSMBMapJoinOperator.class;
             }
 
-            vectorOp = OperatorFactory.getVectorOperator(opClass, op.getConf(), vContext);
+            vectorOp = OperatorFactory.getVectorOperator(
+                opClass, op.getCompilationOpContext(), op.getConf(), vContext);
 
           } else {
 
@@ -2200,7 +2203,8 @@ public class Vectorizer implements PhysicalPlanResolver {
 
           if (!specialize) {
 
-            vectorOp = OperatorFactory.getVectorOperator(op.getConf(), vContext);
+            vectorOp = OperatorFactory.getVectorOperator(
+                op.getCompilationOpContext(), op.getConf(), vContext);
 
           } else {
 
@@ -2217,7 +2221,8 @@ public class Vectorizer implements PhysicalPlanResolver {
       case EXTRACT:
       case EVENT:
       case HASHTABLESINK:
-        vectorOp = OperatorFactory.getVectorOperator(op.getConf(), vContext);
+        vectorOp = OperatorFactory.getVectorOperator(
+            op.getCompilationOpContext(), op.getConf(), vContext);
         break;
       default:
         vectorOp = op;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
index 5b3125b..f48fac1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
@@ -197,7 +197,8 @@ public class SparkReduceSinkMapJoinProc implements NodeProcessor {
 
     // create an new operator: HashTableDummyOperator, which share the table desc
     HashTableDummyDesc desc = new HashTableDummyDesc();
-    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(desc);
+    HashTableDummyOperator dummyOp = (HashTableDummyOperator) OperatorFactory.get(
+        mapJoinOp.getCompilationOpContext(), desc);
     TableDesc tbl;
 
     // need to create the correct table descriptor for key/value
@@ -261,8 +262,8 @@ public class SparkReduceSinkMapJoinProc implements NodeProcessor {
     mjDesc.setHashTableMemoryUsage(hashtableMemoryUsage);
 
     SparkHashTableSinkDesc hashTableSinkDesc = new SparkHashTableSinkDesc(mjDesc);
-    SparkHashTableSinkOperator hashTableSinkOp =
-      (SparkHashTableSinkOperator) OperatorFactory.get(hashTableSinkDesc);
+    SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) OperatorFactory.get(
+        mapJoinOp.getCompilationOpContext(), hashTableSinkDesc);
 
     byte tag = (byte) pos;
     int[] valueIndex = mjDesc.getValueIndex(tag);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
index 94947d6..2a7f3d4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
@@ -196,8 +196,7 @@ public final class UnionProcFactory {
             Map<String, ExprNodeDesc> origColExprMap = originalOp.getColumnExprMap();
 
             Operator<? extends OperatorDesc> cloneOp =
-              OperatorFactory.getAndMakeChild(
-                cloneDesc, 
+              OperatorFactory.getAndMakeChild(cloneDesc,
                 origSchema == null ? null : new RowSchema(origSchema), 
                 origColExprMap == null ? null : new HashMap(origColExprMap), 
                 parents.get(p));

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index cead5ae..8a9411a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hive.metastore.HiveMetaStore;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryProperties;
@@ -90,6 +91,7 @@ public abstract class BaseSemanticAnalyzer {
   protected final Logger LOG;
   protected final LogHelper console;
 
+  protected CompilationOpContext cContext;
   protected Context ctx;
   protected HashMap<String, String> idToTableNameMap;
   protected QueryProperties queryProperties;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
index 2d365a9..c1e9ec1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
@@ -83,7 +83,7 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
     FetchTask fetchTask = sem.getFetchTask();
     if (fetchTask != null) {
       // Initialize fetch work such that operator tree will be constructed.
-      fetchTask.getWork().initializeForFetch();
+      fetchTask.getWork().initializeForFetch(ctx.getOpContext());
     }
 
     ParseContext pCtx = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
index 1a49de1..46d279e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezWork.java
@@ -290,9 +290,8 @@ public class GenTezWork implements NodeProcessor {
                   LOG.debug("Cloning reduce sink for multi-child broadcast edge");
                   // we've already set this one up. Need to clone for the next work.
                   r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
-                      (ReduceSinkDesc)r.getConf().clone(),
-                      new RowSchema(r.getSchema()),
-                      r.getParentOperators());
+                      r.getCompilationOpContext(), (ReduceSinkDesc)r.getConf().clone(),
+                      new RowSchema(r.getSchema()), r.getParentOperators());
                   context.clonedReduceSinks.add(r);
                 }
                 r.getConf().setOutputName(work.getName());

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
index 6f9948e..93b7a66 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ProcessAnalyzeTable.java
@@ -190,7 +190,8 @@ public class ProcessAnalyzeTable implements NodeProcessor {
     // partial scan task
     DriverContext driverCxt = new DriverContext();
     Task<PartialScanWork> partialScanTask = TaskFactory.get(scanWork, parseContext.getConf());
-    partialScanTask.initialize(parseContext.getConf(), null, driverCxt);
+    partialScanTask.initialize(parseContext.getConf(), null, driverCxt,
+        tableScan.getCompilationOpContext());
     partialScanTask.setWork(scanWork);
     statsWork.setSourceTask(partialScanTask);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index c38699d..a0251fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -66,6 +66,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryProperties;
 import org.apache.hadoop.hive.ql.exec.AbstractMapJoinOperator;
@@ -418,6 +419,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         analyzeRewrite, tableDesc, queryProperties);
   }
 
+  public CompilationOpContext getOpContext() {
+    return ctx.getOpContext();
+  }
+
   @SuppressWarnings("nls")
   public void doPhase1QBExpr(ASTNode ast, QBExpr qbexpr, String id, String alias)
       throws SemanticException {
@@ -3323,8 +3328,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         .getChild(inputRecordWriterNum));
     Class<? extends RecordReader> errRecordReader = getDefaultRecordReader();
 
-    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
-        new ScriptDesc(
+    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(new ScriptDesc(
             fetchFilesNotInLocalFilesystem(stripQuotes(trfm.getChild(execPos).getText())),
             inInfo, inRecordWriter, outInfo, outRecordReader, errRecordReader, errInfo),
         new RowSchema(out_rwsch.getColumnInfos()), input), out_rwsch);
@@ -5424,8 +5428,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       FilterDesc orFilterDesc = new FilterDesc(previous, false);
       orFilterDesc.setGenerated(true);
 
-      selectInput = putOpInsertMap(OperatorFactory.getAndMakeChild(
-          orFilterDesc, new RowSchema(
+      selectInput = putOpInsertMap(OperatorFactory.getAndMakeChild(orFilterDesc, new RowSchema(
               inputRR.getColumnInfos()), input), inputRR);
     }
 
@@ -6636,8 +6639,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       fileSinkDesc.setStaticSpec(dpCtx.getSPPath());
     }
 
-    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(fileSinkDesc,
-        fsRS, input), inputRR);
+    Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild(
+        fileSinkDesc, fsRS, input), inputRR);
 
     if (ltd != null && SessionState.get() != null) {
       SessionState.get().getLineageState()
@@ -7382,7 +7385,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     desc.setReversedExprs(reversedExprs);
     desc.setFilterMap(join.getFilterMap());
 
-    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(desc,
+    JoinOperator joinOp = (JoinOperator) OperatorFactory.getAndMakeChild(getOpContext(), desc,
         new RowSchema(outputRR.getColumnInfos()), rightOps);
     joinOp.setColumnExprMap(colExprMap);
     joinOp.setPosToAliasMap(posToAliasMap);
@@ -7499,8 +7502,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         reduceKeys.size(), numReds, AcidUtils.Operation.NOT_ACID);
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator) putOpInsertMap(
-        OperatorFactory.getAndMakeChild(rsDesc, new RowSchema(outputRR
-            .getColumnInfos()), child), outputRR);
+        OperatorFactory.getAndMakeChild(rsDesc, new RowSchema(outputRR.getColumnInfos()),
+            child), outputRR);
     List<String> keyColNames = rsDesc.getOutputKeyColumnNames();
     for (int i = 0 ; i < keyColNames.size(); i++) {
       colExprMap.put(Utilities.ReduceField.KEY + "." + keyColNames.get(i), reduceKeys.get(i));
@@ -9090,7 +9093,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
     // Create a new union operator
     Operator<? extends OperatorDesc> unionforward = OperatorFactory
-        .getAndMakeChild(new UnionDesc(), new RowSchema(unionoutRR
+        .getAndMakeChild(getOpContext(), new UnionDesc(), new RowSchema(unionoutRR
             .getColumnInfos()));
 
     // set union operator as child of each of leftOp and rightOp
@@ -9332,7 +9335,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         nameToSplitSample.remove(alias_id);
       }
 
-      top = (TableScanOperator) putOpInsertMap(OperatorFactory.get(tsDesc,
+      top = (TableScanOperator) putOpInsertMap(OperatorFactory.get(getOpContext(), tsDesc,
           new RowSchema(rwsch.getColumnInfos())), rwsch);
 
       // Add this to the list of top operators - we always start from a table
@@ -11988,8 +11991,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
 
         ptfDesc.setMapSide(true);
         input = putOpInsertMap(OperatorFactory.getAndMakeChild(ptfDesc,
-            new RowSchema(ptfMapRR.getColumnInfos()),
-            input), ptfMapRR);
+            new RowSchema(ptfMapRR.getColumnInfos()), input), ptfMapRR);
         rr = opParseCtx.get(input).getRowResolver();
       }
 
@@ -12052,8 +12054,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       PTFDesc ptfDesc = translator.translate(wSpec, this, conf, rr, unparseTranslator);
       RowResolver ptfOpRR = ptfDesc.getFuncDef().getOutputShape().getRr();
       input = putOpInsertMap(OperatorFactory.getAndMakeChild(ptfDesc,
-          new RowSchema(ptfOpRR.getColumnInfos()),
-          input), ptfOpRR);
+          new RowSchema(ptfOpRR.getColumnInfos()), input), ptfOpRR);
       input = genSelectAllDesc(input);
       rr = ptfOpRR;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
index ea5e414..afbeccb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkWork.java
@@ -185,7 +185,8 @@ public class GenSparkWork implements NodeProcessor {
                   LOG.debug("Cloning reduce sink for multi-child broadcast edge");
                   // we've already set this one up. Need to clone for the next work.
                   r = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(
-                      (ReduceSinkDesc)r.getConf().clone(), r.getParentOperators());
+                      r.getCompilationOpContext(), (ReduceSinkDesc)r.getConf().clone(),
+                      r.getParentOperators());
                 }
                 r.getConf().setOutputName(work.getName());
               }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java
index fa8a53a..3f31fb5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -53,6 +54,15 @@ public class SparkPartitionPruningSinkOperator extends Operator<SparkPartitionPr
   protected transient DataOutputBuffer buffer;
   protected static final Logger LOG = LoggerFactory.getLogger(SparkPartitionPruningSinkOperator.class);
 
+  /** Kryo ctor. */
+  protected SparkPartitionPruningSinkOperator() {
+    super();
+  }
+
+  public SparkPartitionPruningSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @SuppressWarnings("deprecation")
   public void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
index e4e7c98..80ccb28 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkProcessAnalyzeTable.java
@@ -187,7 +187,8 @@ public class SparkProcessAnalyzeTable implements NodeProcessor {
 
     @SuppressWarnings("unchecked")
     Task<PartialScanWork> partialScanTask = TaskFactory.get(scanWork, parseContext.getConf());
-    partialScanTask.initialize(parseContext.getConf(), null, driverCxt);
+    partialScanTask.initialize(parseContext.getConf(), null, driverCxt,
+        tableScan.getCompilationOpContext());
     partialScanTask.setWork(scanWork);
     statsWork.setSourceTask(partialScanTask);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
index 2bce69e..ad34b98 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java
@@ -109,6 +109,7 @@ public abstract class BaseWork extends AbstractOperatorDesc {
   public abstract void replaceRoots(Map<Operator<?>, Operator<?>> replacementMap);
 
   public abstract Set<Operator<? extends OperatorDesc>> getAllRootOperators();
+  public abstract Operator<? extends OperatorDesc> getAnyRootOperator();
 
   public Set<Operator<?>> getAllOperators() {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/ColumnStatsWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ColumnStatsWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ColumnStatsWork.java
index 1d2c24d..76811b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ColumnStatsWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ColumnStatsWork.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.plan;
 
 import java.io.Serializable;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
 
@@ -72,8 +73,8 @@ public class ColumnStatsWork implements Serializable {
     return fWork.getSink();
   }
 
-  public void initializeForFetch() {
-    fWork.initializeForFetch();
+  public void initializeForFetch(CompilationOpContext ctx) {
+    fWork.initializeForFetch(ctx);
   }
 
   public int getLeastNumRows() {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
index edd9cac..d68c64c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FetchWork.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.TreeMap;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.ListSinkOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
@@ -100,10 +101,10 @@ public class FetchWork implements Serializable {
     this.limit = limit;
   }
 
-  public void initializeForFetch() {
+  public void initializeForFetch(CompilationOpContext ctx) {
     if (source == null) {
       ListSinkDesc desc = new ListSinkDesc(serializationNullFormat);
-      sink = (ListSinkOperator) OperatorFactory.get(desc);
+      sink = (ListSinkOperator) OperatorFactory.get(ctx, desc);
       source = sink;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
index ffaf129..3ef50fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java
@@ -399,6 +399,11 @@ public class MapWork extends BaseWork {
     return opSet;
   }
 
+  @Override
+  public Operator<? extends OperatorDesc> getAnyRootOperator() {
+    return aliasToWork.isEmpty() ? null : aliasToWork.values().iterator().next();
+  }
+
   public void mergeAliasedInput(String alias, String pathDir, PartitionDesc partitionInfo) {
     ArrayList<String> aliases = pathToAliases.get(pathDir);
     if (aliases == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java
index da1010b..aa7f6ed 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MapredWork.java
@@ -82,4 +82,9 @@ public class MapredWork extends AbstractOperatorDesc {
 
     return ops;
   }
-}
+
+  public Operator<?> getAnyOperator() {
+    Operator<?> result = mapWork.getAnyRootOperator();
+    if (result != null) return result;
+    return (reduceWork != null) ? reduceWork.getAnyRootOperator() : null;
+  }}

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java
index b088326..a5527dc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java
@@ -52,6 +52,11 @@ public class MergeJoinWork extends BaseWork {
   }
 
   @Override
+  public Operator<?> getAnyRootOperator() {
+    return getMainWork().getAnyRootOperator();
+  }
+
+  @Override
   public void configureJobConf(JobConf job) {
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
index 0ac625f..1c31962 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceWork.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorUtils;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
@@ -197,6 +198,11 @@ public class ReduceWork extends BaseWork {
     return opSet;
   }
 
+  @Override
+  public Operator<? extends OperatorDesc> getAnyRootOperator() {
+    return getReducer();
+  }
+
   /**
    * If the number of reducers is -1, the runtime will automatically figure it
    * out by input data size.

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/plan/UnionWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/UnionWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/UnionWork.java
index 3fecff3..5e30ece 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/UnionWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/UnionWork.java
@@ -64,6 +64,11 @@ public class UnionWork extends BaseWork {
     return new HashSet<Operator<?>>();
   }
 
+  @Override
+  public Operator<? extends OperatorDesc> getAnyRootOperator() {
+    return null;
+  }
+
   public void addUnionOperators(Collection<UnionOperator> unions) {
     unionOperators.addAll(unions);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
index 4702f01..5390ba7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java
@@ -885,8 +885,7 @@ public final class OpProcFactory {
         .getChildOperators();
     op.setChildOperators(null);
     Operator<FilterDesc> output = OperatorFactory.getAndMakeChild(
-        new FilterDesc(condn, false), new RowSchema(inputRS.getSignature()),
-        op);
+        new FilterDesc(condn, false), new RowSchema(inputRS.getSignature()), op);
     output.setChildOperators(originalChilren);
     for (Operator<? extends OperatorDesc> ch : originalChilren) {
       List<Operator<? extends OperatorDesc>> parentOperators = ch

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index b091ebc..8066292 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -106,8 +106,8 @@ public class PredicateTransitivePropagate extends Transform {
   // insert filter operator between target(child) and input(parent)
   private Operator<FilterDesc> createFilter(Operator<?> target, Operator<?> parent,
       RowSchema parentRS, ExprNodeDesc filterExpr) {
-    Operator<FilterDesc> filter = OperatorFactory.get(new FilterDesc(filterExpr, false),
-        new RowSchema(parentRS.getSignature()));
+    Operator<FilterDesc> filter = OperatorFactory.get(parent.getCompilationOpContext(),
+        new FilterDesc(filterExpr, false), new RowSchema(parentRS.getSignature()));
     filter.getParentOperators().add(parent);
     filter.getChildOperators().add(target);
     parent.replaceChild(target, filter);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
index 5d5f02d..71c7310 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
@@ -107,8 +107,8 @@ public class SyntheticJoinPredicate extends Transform {
       RowSchema parentRS, ExprNodeDesc filterExpr) {
     FilterDesc filterDesc = new FilterDesc(filterExpr, false);
     filterDesc.setSyntheticJoinPredicate(true);
-    Operator<FilterDesc> filter = OperatorFactory.get(filterDesc,
-        new RowSchema(parentRS.getSignature()));
+    Operator<FilterDesc> filter = OperatorFactory.get(parent.getCompilationOpContext(),
+        filterDesc, new RowSchema(parentRS.getSignature()));
     filter.getParentOperators().add(parent);
     filter.getChildOperators().add(target);
     parent.replaceChild(target, filter);


[4/4] hive git commit: HIVE-12758 : Parallel compilation: Operator::resetId() is not thread-safe (Sergey Shelukhin, reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-12758 : Parallel compilation: Operator::resetId() is not thread-safe (Sergey Shelukhin, reviewed by Gopal V)


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

Branch: refs/heads/master
Commit: 88fceacaa1f4d9d2d4e56850006d4d1ddfdbf102
Parents: 4eab2df
Author: Sergey Shelukhin <se...@apache.org>
Authored: Mon Jan 18 18:47:17 2016 -0800
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Mon Jan 18 18:47:17 2016 -0800

----------------------------------------------------------------------
 .../mapreduce/TestHCatMultiOutputFormat.java    |   4 +-
 .../hadoop/hive/ql/CompilationOpContext.java    |  36 +++
 .../java/org/apache/hadoop/hive/ql/Context.java |   5 +
 .../java/org/apache/hadoop/hive/ql/Driver.java  |  10 +-
 .../hive/ql/exec/AbstractFileMergeOperator.java |  10 +
 .../hive/ql/exec/AbstractMapJoinOperator.java   |   9 +-
 .../hive/ql/exec/AppMasterEventOperator.java    |  10 +
 .../hadoop/hive/ql/exec/CollectOperator.java    |  10 +
 .../hadoop/hive/ql/exec/ColumnStatsTask.java    |   8 +-
 .../hive/ql/exec/ColumnStatsUpdateTask.java     |   6 +-
 .../hadoop/hive/ql/exec/CommonJoinOperator.java |  12 +-
 .../hive/ql/exec/CommonMergeJoinOperator.java   |   8 +-
 .../hadoop/hive/ql/exec/ConditionalTask.java    |   5 -
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  19 +-
 .../hadoop/hive/ql/exec/DemuxOperator.java      |  10 +
 .../hadoop/hive/ql/exec/DummyStoreOperator.java |   8 +-
 .../apache/hadoop/hive/ql/exec/FetchTask.java   |   8 +-
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  10 +
 .../hadoop/hive/ql/exec/FilterOperator.java     |   8 +-
 .../hadoop/hive/ql/exec/ForwardOperator.java    |  10 +
 .../hadoop/hive/ql/exec/FunctionTask.java       |   6 +-
 .../hadoop/hive/ql/exec/GroupByOperator.java    |  10 +
 .../hive/ql/exec/HashTableDummyOperator.java    |  10 +
 .../hive/ql/exec/HashTableSinkOperator.java     |  12 +-
 .../hadoop/hive/ql/exec/JoinOperator.java       |  10 +
 .../ql/exec/LateralViewForwardOperator.java     |  10 +
 .../hive/ql/exec/LateralViewJoinOperator.java   |  10 +
 .../hadoop/hive/ql/exec/LimitOperator.java      |  10 +
 .../hadoop/hive/ql/exec/ListSinkOperator.java   |  10 +
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |   9 +-
 .../apache/hadoop/hive/ql/exec/MapOperator.java |  10 +
 .../apache/hadoop/hive/ql/exec/MuxOperator.java |  10 +
 .../apache/hadoop/hive/ql/exec/Operator.java    |  57 ++--
 .../hadoop/hive/ql/exec/OperatorFactory.java    | 287 ++++++++++---------
 .../hive/ql/exec/OrcFileMergeOperator.java      |  10 +
 .../apache/hadoop/hive/ql/exec/PTFOperator.java |  10 +
 .../hive/ql/exec/RCFileMergeOperator.java       |  11 +
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java |  10 +
 .../hadoop/hive/ql/exec/SMBMapJoinOperator.java |   9 +-
 .../hadoop/hive/ql/exec/ScriptOperator.java     |  10 +
 .../hadoop/hive/ql/exec/SelectOperator.java     |  10 +
 .../hive/ql/exec/SerializationUtilities.java    |  51 +++-
 .../ql/exec/SparkHashTableSinkOperator.java     |  12 +-
 .../hadoop/hive/ql/exec/StatsNoJobTask.java     |   6 +-
 .../hadoop/hive/ql/exec/TableScanOperator.java  |  10 +
 .../org/apache/hadoop/hive/ql/exec/Task.java    |   4 +-
 .../hive/ql/exec/TemporaryHashSinkOperator.java |   4 +-
 .../hadoop/hive/ql/exec/TerminalOperator.java   |   9 +
 .../hive/ql/exec/TezDummyStoreOperator.java     |  10 +
 .../hadoop/hive/ql/exec/UDTFOperator.java       |  10 +
 .../hadoop/hive/ql/exec/UnionOperator.java      |  12 +-
 .../hadoop/hive/ql/exec/mr/ExecDriver.java      |   6 +-
 .../hadoop/hive/ql/exec/mr/ExecMapper.java      |   6 +-
 .../hadoop/hive/ql/exec/mr/HashTableLoader.java |   3 +-
 .../hadoop/hive/ql/exec/mr/MapredLocalTask.java |   6 +-
 .../hive/ql/exec/spark/HashTableLoader.java     |   3 +-
 .../ql/exec/spark/SparkMapRecordHandler.java    |   6 +-
 .../hadoop/hive/ql/exec/spark/SparkTask.java    |   6 +-
 .../hive/ql/exec/tez/MapRecordProcessor.java    |  10 +-
 .../vector/VectorAppMasterEventOperator.java    |  15 +-
 .../ql/exec/vector/VectorFileSinkOperator.java  |  14 +-
 .../ql/exec/vector/VectorFilterOperator.java    |  14 +-
 .../ql/exec/vector/VectorGroupByOperator.java   |  15 +-
 .../ql/exec/vector/VectorLimitOperator.java     |  12 +-
 .../exec/vector/VectorMapJoinBaseOperator.java  |  14 +-
 .../ql/exec/vector/VectorMapJoinOperator.java   |  14 +-
 .../VectorMapJoinOuterFilteredOperator.java     |  14 +-
 .../hive/ql/exec/vector/VectorMapOperator.java  |  10 +
 .../exec/vector/VectorReduceSinkOperator.java   |  14 +-
 .../exec/vector/VectorSMBMapJoinOperator.java   |  14 +-
 .../ql/exec/vector/VectorSelectOperator.java    |  14 +-
 .../VectorSparkHashTableSinkOperator.java       |  14 +-
 ...VectorSparkPartitionPruningSinkOperator.java |  15 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    |  14 +-
 .../VectorMapJoinGenerateResultOperator.java    |  14 +-
 ...pJoinInnerBigOnlyGenerateResultOperator.java |  14 +-
 .../VectorMapJoinInnerBigOnlyLongOperator.java  |  13 +-
 ...ctorMapJoinInnerBigOnlyMultiKeyOperator.java |  13 +-
 ...VectorMapJoinInnerBigOnlyStringOperator.java |  13 +-
 ...ectorMapJoinInnerGenerateResultOperator.java |  14 +-
 .../mapjoin/VectorMapJoinInnerLongOperator.java |  13 +-
 .../VectorMapJoinInnerMultiKeyOperator.java     |  13 +-
 .../VectorMapJoinInnerStringOperator.java       |  13 +-
 ...orMapJoinLeftSemiGenerateResultOperator.java |  14 +-
 .../VectorMapJoinLeftSemiLongOperator.java      |  13 +-
 .../VectorMapJoinLeftSemiMultiKeyOperator.java  |  13 +-
 .../VectorMapJoinLeftSemiStringOperator.java    |  13 +-
 ...ectorMapJoinOuterGenerateResultOperator.java |  14 +-
 .../mapjoin/VectorMapJoinOuterLongOperator.java |  13 +-
 .../VectorMapJoinOuterMultiKeyOperator.java     |  13 +-
 .../VectorMapJoinOuterStringOperator.java       |  13 +-
 .../VectorReduceSinkCommonOperator.java         |  14 +-
 .../VectorReduceSinkLongOperator.java           |  14 +-
 .../VectorReduceSinkMultiKeyOperator.java       |  14 +-
 .../VectorReduceSinkStringOperator.java         |  14 +-
 .../apache/hadoop/hive/ql/hooks/ATSHook.java    |   2 +-
 .../hadoop/hive/ql/io/merge/MergeFileTask.java  |   5 +-
 .../ql/io/rcfile/stats/PartialScanTask.java     |   7 +-
 .../io/rcfile/truncate/ColumnTruncateTask.java  |   5 +-
 .../hive/ql/optimizer/AbstractSMBJoinProc.java  |   2 +-
 .../hive/ql/optimizer/ConvertJoinMapJoin.java   |   8 +-
 .../DynamicPartitionPruningOptimization.java    |   4 +-
 .../hive/ql/optimizer/GenMRTableScan1.java      |   2 +-
 .../hive/ql/optimizer/GenMapRedUtils.java       |  23 +-
 .../hive/ql/optimizer/GroupByOptimizer.java     |   4 +-
 .../hive/ql/optimizer/MapJoinProcessor.java     |  12 +-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |   4 +-
 .../ql/optimizer/SimpleFetchAggregation.java    |   3 +-
 .../hive/ql/optimizer/SimpleFetchOptimizer.java |   3 +-
 .../hive/ql/optimizer/SkewJoinOptimizer.java    |   8 +-
 .../calcite/translator/HiveGBOpConvUtil.java    |  10 +-
 .../calcite/translator/HiveOpConverter.java     |  34 +--
 .../QueryPlanTreeTransformation.java            |  12 +-
 .../physical/CommonJoinTaskDispatcher.java      |   4 +-
 .../physical/GenMRSkewJoinProcessor.java        |   7 +-
 .../physical/GenSparkSkewJoinProcessor.java     |  13 +-
 .../physical/LocalMapJoinProcFactory.java       |   5 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |  15 +-
 .../spark/SparkReduceSinkMapJoinProc.java       |   7 +-
 .../optimizer/unionproc/UnionProcFactory.java   |   3 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |   2 +
 .../hive/ql/parse/ExplainSemanticAnalyzer.java  |   2 +-
 .../apache/hadoop/hive/ql/parse/GenTezWork.java |   5 +-
 .../hive/ql/parse/ProcessAnalyzeTable.java      |   3 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  31 +-
 .../hive/ql/parse/spark/GenSparkWork.java       |   3 +-
 .../SparkPartitionPruningSinkOperator.java      |  10 +
 .../parse/spark/SparkProcessAnalyzeTable.java   |   3 +-
 .../apache/hadoop/hive/ql/plan/BaseWork.java    |   1 +
 .../hadoop/hive/ql/plan/ColumnStatsWork.java    |   5 +-
 .../apache/hadoop/hive/ql/plan/FetchWork.java   |   5 +-
 .../org/apache/hadoop/hive/ql/plan/MapWork.java |   5 +
 .../apache/hadoop/hive/ql/plan/MapredWork.java  |   7 +-
 .../hadoop/hive/ql/plan/MergeJoinWork.java      |   5 +
 .../apache/hadoop/hive/ql/plan/ReduceWork.java  |   6 +
 .../apache/hadoop/hive/ql/plan/UnionWork.java   |   5 +
 .../hadoop/hive/ql/ppd/OpProcFactory.java       |   3 +-
 .../ql/ppd/PredicateTransitivePropagate.java    |   4 +-
 .../hive/ql/ppd/SyntheticJoinPredicate.java     |   4 +-
 .../hadoop/hive/ql/exec/TestExecDriver.java     |  38 +--
 .../hive/ql/exec/TestFileSinkOperator.java      |   4 +-
 .../hadoop/hive/ql/exec/TestOperators.java      |  17 +-
 .../apache/hadoop/hive/ql/exec/TestPlan.java    |   3 +-
 .../exec/vector/TestVectorFilterOperator.java   |   3 +-
 .../exec/vector/TestVectorGroupByOperator.java  |  58 ++--
 .../ql/exec/vector/TestVectorLimitOperator.java |   3 +-
 .../exec/vector/TestVectorSelectOperator.java   |  10 +-
 .../vector/util/FakeCaptureOutputOperator.java  |  14 +-
 .../util/FakeVectorDataSourceOperator.java      |  16 +-
 .../ql/optimizer/physical/TestVectorizer.java   |   7 +-
 .../hadoop/hive/ql/parse/TestGenTezWork.java    |   8 +-
 .../parse/TestUpdateDeleteSemanticAnalyzer.java |   2 +-
 .../hive/ql/testutil/BaseScalarUdfTest.java     |   3 +-
 .../results/clientpositive/auto_join0.q.out     |   8 +-
 .../cbo_rp_cross_product_check_2.q.out          |   4 +-
 .../clientpositive/cross_product_check_2.q.out  |   4 +-
 .../subquery_multiinsert.q.java1.7.out          |   4 +-
 .../subquery_multiinsert.q.java1.8.out          |  60 ++--
 158 files changed, 1379 insertions(+), 598 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
index 8148faa..61efc1a 100644
--- a/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
+++ b/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatMultiOutputFormat.java
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -40,6 +39,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -392,7 +392,7 @@ public class TestHCatMultiOutputFormat {
     }
     FetchTask task = new FetchTask();
     task.setWork(work);
-    task.initialize(conf, null, null);
+    task.initialize(conf, null, null, new CompilationOpContext());
     task.fetch(temp);
     for (String str : temp) {
       results.add(str.replace("\t", ","));

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/CompilationOpContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/CompilationOpContext.java b/ql/src/java/org/apache/hadoop/hive/ql/CompilationOpContext.java
new file mode 100644
index 0000000..949f873
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/CompilationOpContext.java
@@ -0,0 +1,36 @@
+/**
+ * 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;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A subset of compilation context that is passed to operators to get rid of some globals.
+ * Perhaps this should be rolled into main Context; however, some code necessitates storing the
+ * context in the operators for now, so this may not be advisable given how much stuff the main
+ * Context class contains.
+ * For now, only the operator sequence ID lives here.
+ */
+public class CompilationOpContext {
+  private final AtomicInteger opSeqId = new AtomicInteger(0);
+
+  public int nextOperatorId() {
+    return opSeqId.getAndIncrement();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index affaec8..746456b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -67,6 +67,7 @@ public class Context {
   private int resDirFilesNum;
   boolean initialized;
   String originalTracker = null;
+  private final CompilationOpContext opContext;
   private final Map<String, ContentSummary> pathToCS = new ConcurrentHashMap<String, ContentSummary>();
 
   // scratch path to use for all non-local (ie. hdfs) file system tmp folders
@@ -133,6 +134,7 @@ public class Context {
     localScratchDir = new Path(SessionState.getLocalSessionPath(conf), executionId).toUri().getPath();
     scratchDirPermission = HiveConf.getVar(conf, HiveConf.ConfVars.SCRATCHDIRPERMISSION);
     stagingDir = HiveConf.getVar(conf, HiveConf.ConfVars.STAGINGDIR);
+    opContext = new CompilationOpContext();
   }
 
 
@@ -715,4 +717,7 @@ public class Context {
     this.cboSucceeded = cboSucceeded;
   }
 
+  public CompilationOpContext getOpContext() {
+    return opContext;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 12a7eea..75187cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -169,7 +169,7 @@ public class Driver implements CommandProcessor {
 
   @Override
   public void init() {
-    Operator.resetId();
+    // Nothing for now.
   }
 
   /**
@@ -494,7 +494,7 @@ public class Driver implements CommandProcessor {
 
       // initialize FetchTask right here
       if (plan.getFetchTask() != null) {
-        plan.getFetchTask().initialize(conf, plan, null);
+        plan.getFetchTask().initialize(conf, plan, null, ctx.getOpContext());
       }
 
       //do the authorization check
@@ -572,7 +572,7 @@ public class Driver implements CommandProcessor {
       ASTNode astTree) throws IOException {
     String ret = null;
     ExplainTask task = new ExplainTask();
-    task.initialize(conf, plan, null);
+    task.initialize(conf, plan, null, ctx.getOpContext());
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     PrintStream ps = new PrintStream(baos);
     try {
@@ -1775,7 +1775,7 @@ public class Driver implements CommandProcessor {
       cxt.incCurJobNo(1);
       console.printInfo("Launching Job " + cxt.getCurJobNo() + " out of " + jobs);
     }
-    tsk.initialize(conf, plan, cxt);
+    tsk.initialize(conf, plan, cxt, ctx.getOpContext());
     TaskResult tskRes = new TaskResult();
     TaskRunner tskRun = new TaskRunner(tsk, tskRes);
 
@@ -1865,7 +1865,7 @@ public class Driver implements CommandProcessor {
         throw new IOException("Error closing the current fetch task", e);
       }
       // FetchTask should not depend on the plan.
-      fetchTask.initialize(conf, null, null);
+      fetchTask.initialize(conf, null, null, ctx.getOpContext());
     } else {
       ctx.resetStream();
       resStream = null;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java
index a3ec0e1..f99bf11 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractFileMergeOperator.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
 import org.apache.hadoop.hive.ql.plan.FileMergeDesc;
@@ -61,6 +62,15 @@ public abstract class AbstractFileMergeOperator<T extends FileMergeDesc>
   protected Set<Path> incompatFileSet;
   protected transient DynamicPartitionCtx dpCtx;
 
+  /** Kryo ctor. */
+  protected AbstractFileMergeOperator() {
+    super();
+  }
+
+  public AbstractFileMergeOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java
index 7302688..69ba4a2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AbstractMapJoinOperator.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
@@ -50,7 +51,13 @@ public abstract class AbstractMapJoinOperator <T extends MapJoinDesc> extends Co
 
   transient int numMapRowsRead;
 
-  public AbstractMapJoinOperator() {
+  /** Kryo ctor. */
+  protected AbstractMapJoinOperator() {
+    super();
+  }
+
+  public AbstractMapJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   public AbstractMapJoinOperator(AbstractMapJoinOperator<? extends MapJoinDesc> mjop) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
index 7114177..743098b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/AppMasterEventOperator.java
@@ -27,6 +27,7 @@ import java.util.concurrent.Future;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.tez.TezContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
@@ -51,6 +52,15 @@ public class AppMasterEventOperator extends Operator<AppMasterEventDesc> {
   protected transient boolean hasReachedMaxSize = false;
   protected transient long MAX_SIZE;
 
+  /** Kryo ctor. */
+  protected AppMasterEventOperator() {
+    super();
+  }
+
+  public AppMasterEventOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java
index e2f4f58..27ddf13 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CollectOperator.java
@@ -24,6 +24,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.CollectDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -42,6 +43,15 @@ public class CollectOperator extends Operator<CollectDesc> implements
   protected transient ObjectInspector standardRowInspector;
   transient int maxSize;
 
+  /** Kryo ctor. */
+  protected CollectOperator() {
+    super();
+  }
+
+  public CollectOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
index f6fbe74..7914471 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsTask.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -81,9 +82,10 @@ public class ColumnStatsTask extends Task<ColumnStatsWork> implements Serializab
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
-    super.initialize(conf, queryPlan, ctx);
-    work.initializeForFetch();
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, ctx, opContext);
+    work.initializeForFetch(opContext);
     try {
       JobConf job = new JobConf(conf);
       ftOp = new FetchOperator(work.getfWork(), job);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
index dcbbe2e..a1b98f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnStatsUpdateTask.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -68,8 +69,9 @@ public class ColumnStatsUpdateTask extends Task<ColumnStatsUpdateWork> {
       .getLogger(ColumnStatsUpdateTask.class);
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
-    super.initialize(conf, queryPlan, ctx);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, ctx, opContext);
   }
 
   private ColumnStatistics constructColumnStatsFromInput()

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
index b0170f5..f8520f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
@@ -31,6 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.AbstractRowContainer;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -125,17 +126,23 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
   protected transient int heartbeatInterval;
   protected static final int NOTSKIPBIGTABLE = -1;
 
-  public CommonJoinOperator() {
+  /** Kryo ctor. */
+  protected CommonJoinOperator() {
+    super();
+  }
+
+  public CommonJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   public CommonJoinOperator(CommonJoinOperator<T> clone) {
+    super(clone.id, clone.cContext);
     this.joinEmitInterval = clone.joinEmitInterval;
     this.joinCacheSize = clone.joinCacheSize;
     this.nextSz = clone.nextSz;
     this.childOperators = clone.childOperators;
     this.parentOperators = clone.parentOperators;
     this.done = clone.done;
-    this.operatorId = clone.operatorId;
     this.storage = clone.storage;
     this.condn = clone.condn;
     this.conf = clone.getConf();
@@ -150,7 +157,6 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
     this.groupKeyObject = clone.groupKeyObject;
     this.handleSkewJoin = clone.handleSkewJoin;
     this.hconf = clone.hconf;
-    this.id = clone.id;
     this.inputObjInspectors = clone.inputObjInspectors;
     this.noOuterJoin = clone.noOuterJoin;
     this.numAliases = clone.numAliases;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java
index 1cbd13d..8693200 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java
@@ -33,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 import org.apache.hadoop.hive.ql.exec.tez.RecordSource;
 import org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource;
@@ -90,10 +91,15 @@ public class CommonMergeJoinOperator extends AbstractMapJoinOperator<CommonMerge
       new ArrayList<Operator<? extends OperatorDesc>>();
   transient Set<Integer> fetchInputAtClose;
 
-  public CommonMergeJoinOperator() {
+  /** Kryo ctor. */
+  protected CommonMergeJoinOperator() {
     super();
   }
 
+  public CommonMergeJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @SuppressWarnings("unchecked")
   @Override
   public void initializeOp(Configuration hconf) throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java
index 031331e..c96c813 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ConditionalTask.java
@@ -72,11 +72,6 @@ public class ConditionalTask extends Task<ConditionalWork> implements Serializab
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
-  }
-
-  @Override
   public int execute(DriverContext driverContext) {
     resTasks = resolver.getTasks(conf, resolverCtx);
     resolved = true;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 5f3a9cf..94bb73c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.TxnInfo;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -260,8 +261,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
-    super.initialize(conf, queryPlan, ctx);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, ctx, opContext);
 
     // Pick the formatter to use to display the results.  Either the
     // normal human readable output or a json object.
@@ -507,7 +509,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       AlterTablePartMergeFilesDesc mergeFilesDesc = work.getMergeFilesDesc();
       if (mergeFilesDesc != null) {
-        return mergeFiles(db, mergeFilesDesc);
+        return mergeFiles(db, mergeFilesDesc, driverContext);
       }
 
       AlterTableAlterPartDesc alterPartDesc = work.getAlterTableAlterPartDesc();
@@ -609,8 +611,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    * @return
    * @throws HiveException
    */
-  private int mergeFiles(Hive db, AlterTablePartMergeFilesDesc mergeFilesDesc)
-      throws HiveException {
+  private int mergeFiles(Hive db, AlterTablePartMergeFilesDesc mergeFilesDesc,
+      DriverContext driverContext) throws HiveException {
     ListBucketingCtx lbCtx = mergeFilesDesc.getLbCtx();
     boolean lbatc = lbCtx == null ? false : lbCtx.isSkewedStoredAsDir();
     int lbd = lbCtx == null ? 0 : lbCtx.calculateListBucketingLevel();
@@ -642,7 +644,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     fmd.setListBucketingDepth(lbd);
     fmd.setOutputPath(mergeFilesDesc.getOutputDir());
 
-    Operator<? extends OperatorDesc> mergeOp = OperatorFactory.get(fmd);
+    CompilationOpContext opContext = driverContext.getCtx().getOpContext();
+    Operator<? extends OperatorDesc> mergeOp = OperatorFactory.get(opContext, fmd);
 
     LinkedHashMap<String, Operator<? extends  OperatorDesc>> aliasToWork =
         new LinkedHashMap<String, Operator<? extends OperatorDesc>>();
@@ -662,7 +665,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // initialize the task and execute
-    task.initialize(db.getConf(), getQueryPlan(), driverCxt);
+    task.initialize(db.getConf(), getQueryPlan(), driverCxt, opContext);
     int ret = task.execute(driverCxt);
     return ret;
   }
@@ -4270,7 +4273,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       truncateWork.setMapperCannotSpanPartns(true);
       DriverContext driverCxt = new DriverContext();
       ColumnTruncateTask taskExec = new ColumnTruncateTask();
-      taskExec.initialize(db.getConf(), null, driverCxt);
+      taskExec.initialize(db.getConf(), null, driverCxt, null);
       taskExec.setWork(truncateWork);
       taskExec.setQueryPlan(this.getQueryPlan());
       return taskExec.execute(driverCxt);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java
index 0888c7b..b897c16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DemuxOperator.java
@@ -30,6 +30,7 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.DemuxDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -109,6 +110,15 @@ public class DemuxOperator extends Operator<DemuxDesc>
   // its children's parents lists, also see childOperatorsTag in Operator) at here.
   private int[][] newChildOperatorsTag;
 
+  /** Kryo ctor. */
+  protected DemuxOperator() {
+    super();
+  }
+
+  public DemuxOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/DummyStoreOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DummyStoreOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DummyStoreOperator.java
index 0c12570..06a3884 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DummyStoreOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DummyStoreOperator.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.DummyStoreDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -69,10 +70,15 @@ public class DummyStoreOperator extends Operator<DummyStoreDesc> implements Seri
 
   protected transient InspectableObject result;
 
-  public DummyStoreOperator() {
+  /** Kryo ctor. */
+  protected DummyStoreOperator() {
     super();
   }
 
+  public DummyStoreOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
index 1634143..4415328 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
@@ -26,6 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.mr.ExecMapper;
@@ -59,9 +60,10 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
-    super.initialize(conf, queryPlan, ctx);
-    work.initializeForFetch();
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, ctx, opContext);
+    work.initializeForFetch(opContext);
 
     try {
       // Create a file system handle

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 32bfcf5..2fa3d96 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
@@ -322,6 +323,15 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     childSpecPathDynLinkedPartitions = conf.getDirName().getName();
   }
 
+  /** Kryo ctor. */
+  protected FileSinkOperator() {
+    super();
+  }
+
+  public FileSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java
index 0e7e79d..08f2633 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FilterOperator.java
@@ -24,6 +24,7 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.IOContext;
 import org.apache.hadoop.hive.ql.io.IOContextMap;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -45,8 +46,13 @@ public class FilterOperator extends Operator<FilterDesc> implements
   private transient IOContext ioContext;
   protected transient int heartbeatInterval;
 
-  public FilterOperator() {
+  /** Kryo ctor. */
+  protected FilterOperator() {
     super();
+  }
+
+  public FilterOperator(CompilationOpContext ctx) {
+    super(ctx);
     consecutiveSearches = 0;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java
index 7a4c58a..2df7cca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ForwardOperator.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ForwardDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -61,6 +62,15 @@ public class ForwardOperator extends Operator<ForwardDesc> implements
     return "FOR";
   }
 
+  /** Kryo ctor. */
+  protected ForwardOperator() {
+    super();
+  }
+
+  public ForwardOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
index ec755a8..ed6f062 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.hadoop.hive.metastore.api.ResourceType;
 import org.apache.hadoop.hive.metastore.api.ResourceUri;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -60,8 +61,9 @@ public class FunctionTask extends Task<FunctionWork> {
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
-    super.initialize(conf, queryPlan, ctx);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, ctx, opContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
index 1693ec3..0839b42 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/GroupByOperator.java
@@ -37,6 +37,7 @@ import javolution.util.FastBitSet;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.parse.OpParseContext;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;
@@ -179,6 +180,15 @@ public class GroupByOperator extends Operator<GroupByDesc> {
     return bits;
   }
 
+  /** Kryo ctor. */
+  protected GroupByOperator() {
+    super();
+  }
+
+  public GroupByOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableDummyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableDummyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableDummyOperator.java
index 1de8c76..4749247 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableDummyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableDummyOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.HashTableDummyDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
@@ -32,6 +33,15 @@ import org.apache.hadoop.hive.serde2.SerDeUtils;
 public class HashTableDummyOperator extends Operator<HashTableDummyDesc> implements Serializable {
   private static final long serialVersionUID = 1L;
 
+  /** Kryo ctor. */
+  protected HashTableDummyOperator() {
+    super();
+  }
+
+  public HashTableDummyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java
index 76308f6..deb7c76 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/HashTableSinkOperator.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionHandler;
 import org.apache.hadoop.hive.ql.exec.persistence.HashMapWrapper;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinEagerRowContainer;
@@ -104,10 +105,17 @@ public class HashTableSinkOperator extends TerminalOperator<HashTableSinkDesc> i
   private long hashTableScale;
   private MapJoinMemoryExhaustionHandler memoryExhaustionHandler;
 
-  public HashTableSinkOperator() {
+  /** Kryo ctor. */
+  protected HashTableSinkOperator() {
+    super();
   }
 
-  public HashTableSinkOperator(MapJoinOperator mjop) {
+  public HashTableSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public HashTableSinkOperator(CompilationOpContext ctx, MapJoinOperator mjop) {
+    this(ctx);
     this.conf = new HashTableSinkDesc(mjop.getConf());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
index 3453fc9..08cc4b4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.JoinDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -55,6 +56,15 @@ public class JoinOperator extends CommonJoinOperator<JoinDesc> implements Serial
 
   private final transient LongWritable skewjoin_followup_jobs = new LongWritable(0);
 
+  /** Kryo ctor. */
+  protected JoinOperator() {
+    super();
+  }
+
+  public JoinOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewForwardOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewForwardOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewForwardOperator.java
index e866eed..4c94ad9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewForwardOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewForwardOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.LateralViewForwardDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -55,6 +56,15 @@ public class LateralViewForwardOperator extends Operator<LateralViewForwardDesc>
     return OperatorType.LATERALVIEWFORWARD;
   }
 
+  /** Kryo ctor. */
+  protected LateralViewForwardOperator() {
+    super();
+  }
+
+  public LateralViewForwardOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewJoinOperator.java
index 55bb08f..7407dc6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/LateralViewJoinOperator.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.LateralViewJoinDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -83,6 +84,15 @@ public class LateralViewJoinOperator extends Operator<LateralViewJoinDesc> {
   public static final byte SELECT_TAG = 0;
   public static final byte UDTF_TAG = 1;
 
+  /** Kryo ctor. */
+  protected LateralViewJoinOperator() {
+    super();
+  }
+
+  public LateralViewJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
index fc85bea..239d56b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/LimitOperator.java
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.LimitDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -39,6 +40,15 @@ public class LimitOperator extends Operator<LimitDesc> implements Serializable {
   protected transient int currCount;
   protected transient boolean isMap;
 
+  /** Kryo ctor. */
+  protected LimitOperator() {
+    super();
+  }
+
+  public LimitOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
index 919e72f..2f2abc1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
@@ -24,6 +24,7 @@ import java.util.Properties;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ListSinkDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -43,6 +44,15 @@ public class ListSinkOperator extends Operator<ListSinkDesc> {
   private transient FetchFormatter fetcher;
   private transient int numRows;
 
+  /** Kryo ctor. */
+  protected ListSinkOperator() {
+    super();
+  }
+
+  public ListSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index dc0b85e..91b5ca7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.HashTableLoaderFactory;
 import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext;
 import org.apache.hadoop.hive.ql.exec.persistence.BytesBytesMultiHashMap;
@@ -97,7 +98,13 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
   protected HybridHashTableContainer firstSmallTable; // The first small table;
                                                       // Only this table has spilled big table rows
 
-  public MapJoinOperator() {
+  /** Kryo ctor. */
+  protected MapJoinOperator() {
+    super();
+  }
+
+  public MapJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   public MapJoinOperator(AbstractMapJoinOperator<? extends MapJoinDesc> mjop) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
index ec0d95c..2b690f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java
@@ -37,6 +37,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.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.MapOperator.MapOpCtx;
 import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
@@ -463,6 +464,15 @@ public class MapOperator extends Operator<MapWork> implements Serializable, Clon
     return nominal;
   }
 
+  /** Kryo ctor. */
+  protected MapOperator() {
+    super();
+  }
+
+  public MapOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java
index 4f4abd3..d8444fb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MuxOperator.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.MuxDesc;
@@ -170,6 +171,15 @@ public class MuxOperator extends Operator<MuxDesc> implements Serializable{
   private transient long[] cntrs;
   private transient long[] nextCntrs;
 
+  /** Kryo ctor. */
+  protected MuxOperator() {
+    super();
+  }
+
+  public MuxOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index 85ab6b2..dbe4f80 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -30,11 +30,11 @@ import java.util.Set;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 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;
@@ -67,6 +67,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   public static final String CONTEXT_NAME_KEY = "__hive.context.name";
 
   private transient Configuration configuration;
+  protected transient CompilationOpContext cContext;
   protected List<Operator<? extends OperatorDesc>> childOperators;
   protected List<Operator<? extends OperatorDesc>> parentOperators;
   protected String operatorId;
@@ -75,8 +76,6 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   private transient boolean rootInitializeCalled = false;
   protected final transient Collection<Future<?>> asyncInitOperations = new HashSet<>();
 
-  private static AtomicInteger seqId;
-
   // It can be optimized later so that an operator operator (init/close) is performed
   // only after that operation has been performed on all the parents. This will require
   // initializing the whole tree in all the mappers (which might be required for mappers
@@ -98,38 +97,24 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
 
   protected transient State state = State.UNINIT;
 
-  static {
-    seqId = new AtomicInteger(0);
-  }
-
   private boolean useBucketizedHiveInputFormat;
 
   // dummy operator (for not increasing seqId)
-  private Operator(String name) {
-    id = name;
+  protected Operator(String name, CompilationOpContext cContext) {
+    this();
+    this.cContext = cContext;
+    this.id = name;
     initOperatorId();
+  }
+
+  protected Operator() {
     childOperators = new ArrayList<Operator<? extends OperatorDesc>>();
     parentOperators = new ArrayList<Operator<? extends OperatorDesc>>();
     abortOp = new AtomicBoolean(false);
   }
 
-  public Operator() {
-    this(String.valueOf(seqId.getAndIncrement()));
-  }
-
-  public static void resetId() {
-    seqId.set(0);
-  }
-
-  /**
-   * Create an operator with a reporter.
-   *
-   * @param reporter
-   *          Used to report progress of certain operators.
-   */
-  public Operator(Reporter reporter) {
-    this();
-    this.reporter = reporter;
+  public Operator(CompilationOpContext cContext) {
+    this(String.valueOf(cContext.nextOperatorId()), cContext);
   }
 
   public void setChildOperators(
@@ -228,7 +213,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   protected transient final boolean isLogTraceEnabled = LOG.isTraceEnabled() && PLOG.isTraceEnabled();
   protected transient String alias;
   protected transient Reporter reporter;
-  protected transient String id;
+  protected String id;
   // object inspectors for input rows
   // We will increase the size of the array on demand
   protected transient ObjectInspector[] inputObjInspectors = new ObjectInspector[1];
@@ -1129,8 +1114,8 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
     @SuppressWarnings("unchecked")
     T descClone = (T)conf.clone();
     // also clone the colExprMap by default
-    Operator<? extends OperatorDesc> ret =
-        OperatorFactory.getAndMakeChild(descClone, getSchema(), getColumnExprMap(), parentClones);
+    Operator<? extends OperatorDesc> ret = OperatorFactory.getAndMakeChild(
+            cContext, descClone, getSchema(), getColumnExprMap(), parentClones);
 
     return ret;
   }
@@ -1145,8 +1130,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   public Operator<? extends OperatorDesc> cloneOp() throws CloneNotSupportedException {
     T descClone = (T) conf.clone();
     Operator<? extends OperatorDesc> ret =
-        OperatorFactory.getAndMakeChild(
-        descClone, getSchema());
+        OperatorFactory.getAndMakeChild(cContext, descClone, getSchema());
     return ret;
   }
 
@@ -1355,7 +1339,7 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
 
   @SuppressWarnings({ "serial", "unchecked", "rawtypes" })
   private static class DummyOperator extends Operator {
-    public DummyOperator() { super("dummy"); }
+    public DummyOperator() { super("dummy", null); }
 
     @Override
     public void process(Object row, int tag) {
@@ -1384,4 +1368,13 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   public String getReduceOutputName() {
     return null;
   }
+
+  public void setCompilationOpContext(CompilationOpContext ctx) {
+    cContext = ctx;
+  }
+
+  /** @return Compilation operator context. Only available during compilation. */
+  public CompilationOpContext getCompilationOpContext() {
+    return cContext;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
index f619a56..038b96c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
@@ -19,11 +19,13 @@
 package org.apache.hadoop.hive.ql.exec;
 
 import java.util.ArrayList;
+import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorAppMasterEventOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorFileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorFilterOperator;
@@ -72,6 +74,8 @@ import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.ql.plan.UDTFDesc;
 import org.apache.hadoop.hive.ql.plan.UnionDesc;
 
+import com.google.common.base.Preconditions;
+
 /**
  * OperatorFactory.
  *
@@ -79,97 +83,68 @@ import org.apache.hadoop.hive.ql.plan.UnionDesc;
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public final class OperatorFactory {
   protected static transient final Logger LOG = LoggerFactory.getLogger(OperatorFactory.class);
-  private static final List<OpTuple> opvec;
-  private static final List<OpTuple> vectorOpvec;
+  private static final IdentityHashMap<Class<? extends OperatorDesc>,
+    Class<? extends Operator<? extends OperatorDesc>>> opvec = new IdentityHashMap<>();
+  private static final IdentityHashMap<Class<? extends OperatorDesc>,
+    Class<? extends Operator<? extends OperatorDesc>>> vectorOpvec = new IdentityHashMap<>();
 
   static {
-    opvec = new ArrayList<OpTuple>();
-    opvec.add(new OpTuple<FilterDesc>(FilterDesc.class, FilterOperator.class));
-    opvec.add(new OpTuple<SelectDesc>(SelectDesc.class, SelectOperator.class));
-    opvec.add(new OpTuple<ForwardDesc>(ForwardDesc.class, ForwardOperator.class));
-    opvec.add(new OpTuple<FileSinkDesc>(FileSinkDesc.class, FileSinkOperator.class));
-    opvec.add(new OpTuple<CollectDesc>(CollectDesc.class, CollectOperator.class));
-    opvec.add(new OpTuple<ScriptDesc>(ScriptDesc.class, ScriptOperator.class));
-    opvec.add(new OpTuple<PTFDesc>(PTFDesc.class, PTFOperator.class));
-    opvec.add(new OpTuple<ReduceSinkDesc>(ReduceSinkDesc.class, ReduceSinkOperator.class));
-    opvec.add(new OpTuple<GroupByDesc>(GroupByDesc.class, GroupByOperator.class));
-    opvec.add(new OpTuple<JoinDesc>(JoinDesc.class, JoinOperator.class));
-    opvec.add(new OpTuple<MapJoinDesc>(MapJoinDesc.class, MapJoinOperator.class));
-    opvec.add(new OpTuple<SMBJoinDesc>(SMBJoinDesc.class, SMBMapJoinOperator.class));
-    opvec.add(new OpTuple<LimitDesc>(LimitDesc.class, LimitOperator.class));
-    opvec.add(new OpTuple<TableScanDesc>(TableScanDesc.class, TableScanOperator.class));
-    opvec.add(new OpTuple<UnionDesc>(UnionDesc.class, UnionOperator.class));
-    opvec.add(new OpTuple<UDTFDesc>(UDTFDesc.class, UDTFOperator.class));
-    opvec.add(new OpTuple<LateralViewJoinDesc>(LateralViewJoinDesc.class,
-        LateralViewJoinOperator.class));
-    opvec.add(new OpTuple<LateralViewForwardDesc>(LateralViewForwardDesc.class,
-        LateralViewForwardOperator.class));
-    opvec.add(new OpTuple<HashTableDummyDesc>(HashTableDummyDesc.class,
-        HashTableDummyOperator.class));
-    opvec.add(new OpTuple<HashTableSinkDesc>(HashTableSinkDesc.class,
-        HashTableSinkOperator.class));
-    opvec.add(new OpTuple<SparkHashTableSinkDesc>(SparkHashTableSinkDesc.class,
-        SparkHashTableSinkOperator.class));
-    opvec.add(new OpTuple<DummyStoreDesc>(DummyStoreDesc.class,
-        DummyStoreOperator.class));
-    opvec.add(new OpTuple<DemuxDesc>(DemuxDesc.class,
-        DemuxOperator.class));
-    opvec.add(new OpTuple<MuxDesc>(MuxDesc.class,
-        MuxOperator.class));
-    opvec.add(new OpTuple<AppMasterEventDesc>(AppMasterEventDesc.class,
-        AppMasterEventOperator.class));
-    opvec.add(new OpTuple<DynamicPruningEventDesc>(DynamicPruningEventDesc.class,
-        AppMasterEventOperator.class));
-    opvec.add(new OpTuple<SparkPartitionPruningSinkDesc>(SparkPartitionPruningSinkDesc.class,
-        SparkPartitionPruningSinkOperator.class));
-    opvec.add(new OpTuple<RCFileMergeDesc>(RCFileMergeDesc.class,
-        RCFileMergeOperator.class));
-    opvec.add(new OpTuple<OrcFileMergeDesc>(OrcFileMergeDesc.class,
-        OrcFileMergeOperator.class));
-    opvec.add(new OpTuple<CommonMergeJoinDesc>(CommonMergeJoinDesc.class,
-        CommonMergeJoinOperator.class));
-    opvec.add(new OpTuple<ListSinkDesc>(ListSinkDesc.class,
-        ListSinkOperator.class));
+    opvec.put(FilterDesc.class, FilterOperator.class);
+    opvec.put(SelectDesc.class, SelectOperator.class);
+    opvec.put(ForwardDesc.class, ForwardOperator.class);
+    opvec.put(FileSinkDesc.class, FileSinkOperator.class);
+    opvec.put(CollectDesc.class, CollectOperator.class);
+    opvec.put(ScriptDesc.class, ScriptOperator.class);
+    opvec.put(PTFDesc.class, PTFOperator.class);
+    opvec.put(ReduceSinkDesc.class, ReduceSinkOperator.class);
+    opvec.put(GroupByDesc.class, GroupByOperator.class);
+    opvec.put(JoinDesc.class, JoinOperator.class);
+    opvec.put(MapJoinDesc.class, MapJoinOperator.class);
+    opvec.put(SMBJoinDesc.class, SMBMapJoinOperator.class);
+    opvec.put(LimitDesc.class, LimitOperator.class);
+    opvec.put(TableScanDesc.class, TableScanOperator.class);
+    opvec.put(UnionDesc.class, UnionOperator.class);
+    opvec.put(UDTFDesc.class, UDTFOperator.class);
+    opvec.put(LateralViewJoinDesc.class, LateralViewJoinOperator.class);
+    opvec.put(LateralViewForwardDesc.class, LateralViewForwardOperator.class);
+    opvec.put(HashTableDummyDesc.class, HashTableDummyOperator.class);
+    opvec.put(HashTableSinkDesc.class, HashTableSinkOperator.class);
+    opvec.put(SparkHashTableSinkDesc.class, SparkHashTableSinkOperator.class);
+    opvec.put(DummyStoreDesc.class, DummyStoreOperator.class);
+    opvec.put(DemuxDesc.class, DemuxOperator.class);
+    opvec.put(MuxDesc.class, MuxOperator.class);
+    opvec.put(AppMasterEventDesc.class, AppMasterEventOperator.class);
+    opvec.put(DynamicPruningEventDesc.class, AppMasterEventOperator.class);
+    opvec.put(SparkPartitionPruningSinkDesc.class, SparkPartitionPruningSinkOperator.class);
+    opvec.put(RCFileMergeDesc.class, RCFileMergeOperator.class);
+    opvec.put(OrcFileMergeDesc.class, OrcFileMergeOperator.class);
+    opvec.put(CommonMergeJoinDesc.class, CommonMergeJoinOperator.class);
+    opvec.put(ListSinkDesc.class, ListSinkOperator.class);
   }
 
   static {
-    vectorOpvec = new ArrayList<OpTuple>();
-    vectorOpvec.add(new OpTuple<AppMasterEventDesc>(AppMasterEventDesc.class,
-        VectorAppMasterEventOperator.class));
-    vectorOpvec.add(new OpTuple<DynamicPruningEventDesc>(DynamicPruningEventDesc.class,
-        VectorAppMasterEventOperator.class));
-    vectorOpvec.add(new OpTuple<SparkPartitionPruningSinkDesc>(
-        SparkPartitionPruningSinkDesc.class,
-        VectorSparkPartitionPruningSinkOperator.class));
-    vectorOpvec.add(new OpTuple<SelectDesc>(SelectDesc.class, VectorSelectOperator.class));
-    vectorOpvec.add(new OpTuple<GroupByDesc>(GroupByDesc.class, VectorGroupByOperator.class));
-    vectorOpvec.add(new OpTuple<MapJoinDesc>(MapJoinDesc.class, VectorMapJoinOperator.class));
-    vectorOpvec.add(new OpTuple<SMBJoinDesc>(SMBJoinDesc.class, VectorSMBMapJoinOperator.class));
-    vectorOpvec.add(new OpTuple<ReduceSinkDesc>(ReduceSinkDesc.class,
-        VectorReduceSinkOperator.class));
-    vectorOpvec.add(new OpTuple<FileSinkDesc>(FileSinkDesc.class, VectorFileSinkOperator.class));
-    vectorOpvec.add(new OpTuple<FilterDesc>(FilterDesc.class, VectorFilterOperator.class));
-    vectorOpvec.add(new OpTuple<LimitDesc>(LimitDesc.class, VectorLimitOperator.class));
-    vectorOpvec.add(new OpTuple<SparkHashTableSinkDesc>(SparkHashTableSinkDesc.class,
-        VectorSparkHashTableSinkOperator.class));
-  }
-
-  private static final class OpTuple<T extends OperatorDesc> {
-    private final Class<T> descClass;
-    private final Class<? extends Operator<?>> opClass;
-
-    public OpTuple(Class<T> descClass, Class<? extends Operator<?>> opClass) {
-      this.descClass = descClass;
-      this.opClass = opClass;
-    }
+    vectorOpvec.put(AppMasterEventDesc.class, VectorAppMasterEventOperator.class);
+    vectorOpvec.put(DynamicPruningEventDesc.class, VectorAppMasterEventOperator.class);
+    vectorOpvec.put(
+        SparkPartitionPruningSinkDesc.class, VectorSparkPartitionPruningSinkOperator.class);
+    vectorOpvec.put(SelectDesc.class, VectorSelectOperator.class);
+    vectorOpvec.put(GroupByDesc.class, VectorGroupByOperator.class);
+    vectorOpvec.put(MapJoinDesc.class, VectorMapJoinOperator.class);
+    vectorOpvec.put(SMBJoinDesc.class, VectorSMBMapJoinOperator.class);
+    vectorOpvec.put(ReduceSinkDesc.class, VectorReduceSinkOperator.class);
+    vectorOpvec.put(FileSinkDesc.class, VectorFileSinkOperator.class);
+    vectorOpvec.put(FilterDesc.class, VectorFilterOperator.class);
+    vectorOpvec.put(LimitDesc.class, VectorLimitOperator.class);
+    vectorOpvec.put(SparkHashTableSinkDesc.class, VectorSparkHashTableSinkOperator.class);
   }
 
   public static <T extends OperatorDesc> Operator<T> getVectorOperator(
-    Class<? extends Operator<?>> opClass, T conf, VectorizationContext vContext) throws HiveException {
+    Class<? extends Operator<?>> opClass, CompilationOpContext cContext, T conf,
+        VectorizationContext vContext) throws HiveException {
     try {
       Operator<T> op = (Operator<T>) opClass.getDeclaredConstructor(
-          VectorizationContext.class, OperatorDesc.class).newInstance(
-          vContext, conf);
+          CompilationOpContext.class, VectorizationContext.class, OperatorDesc.class)
+          .newInstance(cContext, vContext, conf);
       return op;
     } catch (Exception e) {
       e.printStackTrace();
@@ -177,50 +152,49 @@ public final class OperatorFactory {
     }
   }
 
-  public static <T extends OperatorDesc> Operator<T> getVectorOperator(T conf,
-      VectorizationContext vContext) throws HiveException {
+  public static <T extends OperatorDesc> Operator<T> getVectorOperator(
+      CompilationOpContext cContext, T conf, VectorizationContext vContext) throws HiveException {
     Class<T> descClass = (Class<T>) conf.getClass();
-    for (OpTuple o : vectorOpvec) {
-      if (o.descClass == descClass) {
-        return getVectorOperator(o.opClass, conf, vContext);
-      }
+    Class<?> opClass = vectorOpvec.get(descClass);
+    if (opClass != null) {
+      return getVectorOperator(vectorOpvec.get(descClass), cContext, conf, vContext);
     }
-    throw new HiveException("No vector operator for descriptor class "
-        + descClass.getName());
+    throw new HiveException("No vector operator for descriptor class " + descClass.getName());
   }
 
-  public static <T extends OperatorDesc> Operator<T> get(Class<T> opClass) {
-
-    for (OpTuple o : opvec) {
-      if (o.descClass == opClass) {
-        try {
-          Operator<T> op = (Operator<T>) o.opClass.newInstance();
-          return op;
-        } catch (Exception e) {
-          e.printStackTrace();
-          throw new RuntimeException(e);
-        }
+  public static <T extends OperatorDesc> Operator<T> get(
+      CompilationOpContext cContext, Class<T> descClass) {
+    Preconditions.checkNotNull(cContext);
+    Class<?> opClass = opvec.get(descClass);
+    if (opClass != null) {
+      try {
+        return (Operator<T>)opClass.getDeclaredConstructor(
+          CompilationOpContext.class).newInstance(cContext);
+      } catch (Exception e) {
+        e.printStackTrace();
+        throw new RuntimeException(e);
       }
     }
-    throw new RuntimeException("No operator for descriptor class "
-        + opClass.getName());
+    throw new RuntimeException("No operator for descriptor class " + descClass.getName());
   }
 
-  public static <T extends OperatorDesc> Operator<T> get(Class<T> opClass,
-      RowSchema rwsch) {
-
-    Operator<T> ret = get(opClass);
-    ret.setSchema(rwsch);
-    return ret;
+  /**
+   * Returns an operator given the conf and a list of children operators.
+   */
+  public static <T extends OperatorDesc> Operator<T> get(CompilationOpContext cContext, T conf) {
+    Operator<T> ret = get(cContext, (Class<T>) conf.getClass());
+    ret.setConf(conf);
+    return (ret);
   }
 
   /**
    * Returns an operator given the conf and a list of children operators.
    */
   public static <T extends OperatorDesc> Operator<T> get(T conf,
-    Operator<? extends OperatorDesc>... oplist) {
-    Operator<T> ret = get((Class<T>) conf.getClass());
+    Operator<? extends OperatorDesc> oplist0, Operator<? extends OperatorDesc>... oplist) {
+    Operator<T> ret = get(oplist0.getCompilationOpContext(), (Class<T>) conf.getClass());
     ret.setConf(conf);
+    makeChild(ret, oplist0);
     makeChild(ret, oplist);
     return (ret);
   }
@@ -253,27 +227,28 @@ public final class OperatorFactory {
   /**
    * Returns an operator given the conf and a list of children operators.
    */
-  public static <T extends OperatorDesc> Operator<T> get(T conf,
-      RowSchema rwsch, Operator... oplist) {
-    Operator<T> ret = get(conf, oplist);
+  public static <T extends OperatorDesc> Operator<T> get(
+      CompilationOpContext cContext, T conf, RowSchema rwsch) {
+    Operator<T> ret = get(cContext, conf);
     ret.setSchema(rwsch);
     return (ret);
   }
 
+
   /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      Operator... oplist) {
-    Operator<T> ret = get((Class<T>) conf.getClass());
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(
+      T conf, Operator oplist0, Operator... oplist) {
+    Operator<T> ret = get(oplist0.getCompilationOpContext(), (Class<T>) conf.getClass());
     ret.setConf(conf);
-    if (oplist.length == 0) {
-      return (ret);
-    }
 
     // Add the new operator as child of each of the passed in operators
+    List<Operator> children = oplist0.getChildOperators();
+    children.add(ret);
+    oplist0.setChildOperators(children);
     for (Operator op : oplist) {
-      List<Operator> children = op.getChildOperators();
+      children = op.getChildOperators();
       children.add(ret);
       op.setChildOperators(children);
     }
@@ -281,6 +256,7 @@ public final class OperatorFactory {
     // add parents for the newly created operator
     List<Operator<? extends OperatorDesc>> parent =
       new ArrayList<Operator<? extends OperatorDesc>>();
+    parent.add(oplist0);
     for (Operator op : oplist) {
       parent.add(op);
     }
@@ -293,9 +269,9 @@ public final class OperatorFactory {
   /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      List<Operator<? extends OperatorDesc>> oplist) {
-    Operator<T> ret = get((Class<T>) conf.getClass());
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(CompilationOpContext cContext,
+      T conf, List<Operator<? extends OperatorDesc>> oplist) {
+    Operator<T> ret = get(cContext, (Class<T>) conf.getClass());
     ret.setConf(conf);
     if (oplist.size() == 0) {
       return ret;
@@ -322,9 +298,49 @@ public final class OperatorFactory {
   /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      RowSchema rwsch, Operator... oplist) {
-    Operator<T> ret = getAndMakeChild(conf, oplist);
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(
+      CompilationOpContext cContext, T conf, RowSchema rwsch) {
+    Operator<T> ret = get(cContext, (Class<T>) conf.getClass());
+    ret.setConf(conf);
+    ret.setSchema(rwsch);
+    return ret;
+  }
+
+  /**
+   * Returns an operator given the conf and a list of parent operators.
+   */
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(
+      CompilationOpContext ctx, T conf, RowSchema rwsch, Operator[] oplist) {
+    Operator<T> ret = get(ctx, (Class<T>) conf.getClass());
+    ret.setConf(conf);
+    ret.setSchema(rwsch);
+    if (oplist.length == 0) return ret;
+
+    // Add the new operator as child of each of the passed in operators
+    for (Operator op : oplist) {
+      List<Operator> children = op.getChildOperators();
+      children.add(ret);
+      op.setChildOperators(children);
+    }
+
+    // add parents for the newly created operator
+    List<Operator<? extends OperatorDesc>> parent =
+      new ArrayList<Operator<? extends OperatorDesc>>();
+    for (Operator op : oplist) {
+      parent.add(op);
+    }
+
+    ret.setParentOperators(parent);
+
+    return (ret);
+  }
+
+  /**
+   * Returns an operator given the conf and a list of parent operators.
+   */
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(
+      T conf, RowSchema rwsch, Operator oplist0, Operator... oplist) {
+    Operator<T> ret = getAndMakeChild(conf, oplist0, oplist);
     ret.setSchema(rwsch);
     return ret;
   }
@@ -332,9 +348,9 @@ public final class OperatorFactory {
   /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      RowSchema rwsch, Map<String, ExprNodeDesc> colExprMap, Operator... oplist) {
-    Operator<T> ret = getAndMakeChild(conf, rwsch, oplist);
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf, RowSchema rwsch,
+      Map<String, ExprNodeDesc> colExprMap, Operator oplist0, Operator... oplist) {
+    Operator<T> ret = getAndMakeChild(conf, rwsch, oplist0, oplist);
     ret.setColumnExprMap(colExprMap);
     return (ret);
   }
@@ -342,9 +358,9 @@ public final class OperatorFactory {
   /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      RowSchema rwsch, List<Operator<? extends OperatorDesc>> oplist) {
-    Operator<T> ret = getAndMakeChild(conf, oplist);
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(CompilationOpContext cContext,
+      T conf, RowSchema rwsch, List<Operator<? extends OperatorDesc>> oplist) {
+    Operator<T> ret = getAndMakeChild(cContext, conf, oplist);
     ret.setSchema(rwsch);
     return (ret);
   }
@@ -352,9 +368,10 @@ public final class OperatorFactory {
  /**
    * Returns an operator given the conf and a list of parent operators.
    */
-  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(T conf,
-      RowSchema rwsch, Map<String, ExprNodeDesc> colExprMap, List<Operator<? extends OperatorDesc>> oplist) {
-    Operator<T> ret = getAndMakeChild(conf, rwsch, oplist);
+  public static <T extends OperatorDesc> Operator<T> getAndMakeChild(CompilationOpContext cContext,
+      T conf, RowSchema rwsch, Map<String, ExprNodeDesc> colExprMap,
+      List<Operator<? extends OperatorDesc>> oplist) {
+    Operator<T> ret = getAndMakeChild(cContext, conf, rwsch, oplist);
     ret.setColumnExprMap(colExprMap);
     return (ret);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/OrcFileMergeOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OrcFileMergeOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OrcFileMergeOperator.java
index 2c9deac..445cf3d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OrcFileMergeOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OrcFileMergeOperator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 import java.io.IOException;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.orc.Writer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -56,6 +57,15 @@ public class OrcFileMergeOperator extends
   private Reader reader;
   private FSDataInputStream fdis;
 
+  /** Kryo ctor. */
+  protected OrcFileMergeOperator() {
+    super();
+  }
+
+  public OrcFileMergeOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void process(Object row, int tag) throws HiveException {
     Object[] keyValue = (Object[]) row;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java
index 113ac21..2e9e539 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/PTFOperator.java
@@ -26,6 +26,7 @@ import java.util.Stack;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
@@ -60,6 +61,15 @@ public class PTFOperator extends Operator<PTFDesc> implements Serializable {
   transient Configuration hiveConf;
   transient PTFInvocation ptfInvocation;
 
+  /** Kryo ctor. */
+  protected PTFOperator() {
+    super();
+  }
+
+  public PTFOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   /*
    * 1. Find out if the operator is invoked at Map-Side or Reduce-side
    * 2. Get the deserialized QueryDef

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/RCFileMergeOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/RCFileMergeOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/RCFileMergeOperator.java
index c34454c..4dea1d2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/RCFileMergeOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/RCFileMergeOperator.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.RCFile;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.rcfile.merge.RCFileKeyBufferWrapper;
@@ -36,12 +37,22 @@ import java.io.IOException;
  */
 public class RCFileMergeOperator
     extends AbstractFileMergeOperator<RCFileMergeDesc> {
+
   public final static Logger LOG = LoggerFactory.getLogger("RCFileMergeMapper");
 
   RCFile.Writer outWriter;
   CompressionCodec codec = null;
   int columnNumber = 0;
 
+  /** Kryo ctor. */
+  protected RCFileMergeOperator() {
+    super();
+  }
+
+  public RCFileMergeOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void process(Object row, int tag) throws HiveException {
     Object[] keyValue = (Object[]) row;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
index 4b65952..74b4802 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
@@ -31,6 +31,7 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -151,6 +152,15 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
   protected transient long logEveryNRows = 0;
   private final transient LongWritable recordCounter = new LongWritable();
 
+  /** Kryo ctor. */
+  protected ReduceSinkOperator() {
+    super();
+  }
+
+  public ReduceSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);


[3/4] hive git commit: HIVE-12758 : Parallel compilation: Operator::resetId() is not thread-safe (Sergey Shelukhin, reviewed by Gopal V)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
index 62ae630..7cc534b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SMBMapJoinOperator.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 import org.apache.hadoop.hive.ql.io.HiveInputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -85,7 +86,13 @@ public class SMBMapJoinOperator extends AbstractMapJoinOperator<SMBJoinDesc> imp
   // performed as a smb join, based on all the tables/partitions being joined.
   private transient boolean convertedAutomaticallySMBJoin = false;
 
-  public SMBMapJoinOperator() {
+  /** Kryo ctor. */
+  protected SMBMapJoinOperator() {
+    super();
+  }
+
+  public SMBMapJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   public SMBMapJoinOperator(AbstractMapJoinOperator<? extends MapJoinDesc> mapJoinOp) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
index 63837ce..9f311a4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ScriptDesc;
@@ -266,6 +267,15 @@ public class ScriptOperator extends Operator<ScriptDesc> implements
     }
   }
 
+  /** Kryo ctor. */
+  protected ScriptOperator() {
+    super();
+  }
+
+  public ScriptOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
index b1b8459..e7c23e8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SelectOperator.java
@@ -25,6 +25,7 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -43,6 +44,15 @@ public class SelectOperator extends Operator<SelectDesc> implements Serializable
 
   private transient boolean isSelectStarNoCompute = false;
 
+  /** Kryo ctor. */
+  protected SelectOperator() {
+    super();
+  }
+
+  public SelectOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
index 45f50c3..0e6e2de 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hive.ql.exec;
 
+import java.util.LinkedList;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.InputStream;
@@ -33,6 +35,7 @@ import java.util.List;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
@@ -356,26 +359,64 @@ public class SerializationUtilities {
     // TODO: need proper clone. Meanwhile, let's at least keep this horror in one place
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.CLONE_PLAN);
+    Operator<?> op = plan.getAnyOperator();
+    CompilationOpContext ctx = (op == null) ? null : op.getCompilationOpContext();
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
     serializePlan(plan, baos, true);
     MapredWork newPlan = deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
         MapredWork.class, true);
+    // Restore the context.
+    for (Operator<?> newOp : newPlan.getAllOperators()) {
+      newOp.setCompilationOpContext(ctx);
+    }
     perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.CLONE_PLAN);
     return newPlan;
   }
 
   /**
    * Clones using the powers of XML. Do not use unless necessary.
+   * @param roots The roots.
+   * @return The clone.
+   */
+  public static List<Operator<?>> cloneOperatorTree(List<Operator<?>> roots) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
+    CompilationOpContext ctx = roots.isEmpty() ? null : roots.get(0).getCompilationOpContext();
+    serializePlan(roots, baos, true);
+    @SuppressWarnings("unchecked")
+    List<Operator<?>> result =
+        deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
+            roots.getClass(), true);
+    // Restore the context.
+    LinkedList<Operator<?>> newOps = new LinkedList<>(result);
+    while (!newOps.isEmpty()) {
+      Operator<?> newOp = newOps.poll();
+      newOp.setCompilationOpContext(ctx);
+      List<Operator<?>> children = newOp.getChildOperators();
+      if (children != null) {
+        newOps.addAll(children);
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Clones using the powers of XML. Do not use unless necessary.
    * @param plan The plan.
    * @return The clone.
    */
   public static BaseWork cloneBaseWork(BaseWork plan) {
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.CLONE_PLAN);
+    Operator<?> op = plan.getAnyRootOperator();
+    CompilationOpContext ctx = (op == null) ? null : op.getCompilationOpContext();
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
     serializePlan(plan, baos, true);
     BaseWork newPlan = deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
         plan.getClass(), true);
+    // Restore the context.
+    for (Operator<?> newOp : newPlan.getAllOperators()) {
+      newOp.setCompilationOpContext(ctx);
+    }
     perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.CLONE_PLAN);
     return newPlan;
   }
@@ -399,16 +440,6 @@ public class SerializationUtilities {
     return t;
   }
 
-  public static List<Operator<?>> cloneOperatorTree(List<Operator<?>> roots) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-    serializePlan(roots, baos, true);
-    @SuppressWarnings("unchecked")
-    List<Operator<?>> result =
-        deserializePlan(new ByteArrayInputStream(baos.toByteArray()),
-            roots.getClass(), true);
-    return result;
-  }
-
   /**
    * Serializes expression via Kryo.
    * @param expr Expression.

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java
index 7a8de2d..85344fc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/SparkHashTableSinkOperator.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinPersistableTableContainer;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainerSerDe;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
@@ -54,8 +55,15 @@ public class SparkHashTableSinkOperator
 
   private final HashTableSinkOperator htsOperator;
 
-  public SparkHashTableSinkOperator() {
-    htsOperator = new HashTableSinkOperator();
+  /** Kryo ctor. */
+  protected SparkHashTableSinkOperator() {
+    super();
+    htsOperator = null; // Kryo will set this; or so we hope.
+  }
+
+  public SparkHashTableSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+    htsOperator = new HashTableSinkOperator(ctx);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
index bb4bde9..14eacdf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsNoJobTask.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.io.StatsProvidingRecordReader;
@@ -81,8 +82,9 @@ public class StatsNoJobTask extends Task<StatsNoJobWork> implements Serializable
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
     jc = new JobConf(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
index 32806dc..5253521 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
@@ -209,6 +210,15 @@ public class TableScanOperator extends Operator<TableScanDesc> implements
 
   }
 
+  /** Kryo ctor. */
+  protected TableScanOperator() {
+    super();
+  }
+
+  public TableScanOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index 0eab63e..40c89cb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -30,6 +30,7 @@ import java.util.List;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.lib.Node;
@@ -120,7 +121,8 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
     return taskHandle;
   }
 
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext,
+      CompilationOpContext opContext) {
     this.queryPlan = queryPlan;
     isdone = false;
     started = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java
index 9e3a84f..bc311ee 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TemporaryHashSinkOperator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.HashTableSinkDesc;
 import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
@@ -27,7 +28,8 @@ import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
 import java.io.IOException;
 
 public class TemporaryHashSinkOperator extends HashTableSinkOperator {
-  public TemporaryHashSinkOperator(MapJoinDesc desc) {
+  public TemporaryHashSinkOperator(CompilationOpContext ctx, MapJoinDesc desc) {
+    super(ctx);
     conf = new HashTableSinkDesc(desc);
 
     // Sanity check the config.

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/TerminalOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TerminalOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TerminalOperator.java
index 069eab2..04d6c9f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TerminalOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TerminalOperator.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.io.Serializable;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 
 /**
@@ -29,4 +30,12 @@ public abstract class TerminalOperator<T extends OperatorDesc> extends
     Operator<T> implements Serializable {
   private static final long serialVersionUID = 1L;
 
+  /** Kryo ctor. */
+  protected TerminalOperator() {
+    super();
+  }
+
+  public TerminalOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/TezDummyStoreOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/TezDummyStoreOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/TezDummyStoreOperator.java
index e9f65be..f28f928 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/TezDummyStoreOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/TezDummyStoreOperator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 
 /**
@@ -27,6 +28,15 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
  *
  */
 public class TezDummyStoreOperator extends DummyStoreOperator {
+  /** Kryo ctor. */
+  protected TezDummyStoreOperator() {
+    super();
+  }
+
+  public TezDummyStoreOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   private boolean fetchDone = false;
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
index b3c6d91..1dae963 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.UDTFDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -58,6 +59,15 @@ public class UDTFOperator extends Operator<UDTFDesc> implements Serializable {
    */
   transient AutoProgressor autoProgressor;
 
+  /** Kryo ctor. */
+  protected UDTFOperator() {
+    super();
+  }
+
+  public UDTFOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/UnionOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/UnionOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/UnionOperator.java
index ddb23ee..3a673e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/UnionOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/UnionOperator.java
@@ -20,11 +20,10 @@ package org.apache.hadoop.hive.ql.exec;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.UnionDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
@@ -48,6 +47,15 @@ public class UnionOperator extends Operator<UnionDesc> implements Serializable {
 
   ArrayList<Object> outputRow;
 
+  /** Kryo ctor. */
+  protected UnionOperator() {
+    super();
+  }
+
+  public UnionOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   /**
    * UnionOperator will transform the input rows if the inputObjInspectors from
    * different parents are different. If one parent has exactly the same

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
index 472e8ed..b184b4e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
@@ -46,6 +46,7 @@ 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.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -141,8 +142,9 @@ public class ExecDriver extends Task<MapredWork> implements Serializable, Hadoop
    * Initialization when invoked from QL.
    */
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
 
     job = new JobConf(conf, ExecDriver.class);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
index 23497a9..c34dd1f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecMapper.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
 import org.apache.hadoop.hive.ql.exec.MapredContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -92,10 +93,11 @@ public class ExecMapper extends MapReduceBase implements Mapper {
       // create map and fetch operators
       MapWork mrwork = Utilities.getMapWork(job);
 
+      CompilationOpContext runtimeCtx = new CompilationOpContext();
       if (mrwork.getVectorMode()) {
-        mo = new VectorMapOperator();
+        mo = new VectorMapOperator(runtimeCtx);
       } else {
-        mo = new MapOperator();
+        mo = new MapOperator(runtimeCtx);
       }
       mo.setConf(mrwork);
       // initialize map operator

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
index 1070384..f5662f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HashTableLoader.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.filecache.DistributedCache;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.HashTableSinkOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapredContext;
@@ -135,7 +136,7 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
     JobConf job = new JobConf(hconf);
     MapredLocalTask localTask = new MapredLocalTask(localWork, job, false);
 
-    HashTableSinkOperator sink = new TemporaryHashSinkOperator(desc);
+    HashTableSinkOperator sink = new TemporaryHashSinkOperator(new CompilationOpContext(), desc);
     sink.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>(directWorks));
 
     for (Operator<?> operator : directWorks) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
index ee99269..1d97a44 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
@@ -44,6 +44,7 @@ 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.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -118,8 +119,9 @@ public class MapredLocalTask extends Task<MapredLocalWork> implements Serializab
   }
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
     job = new JobConf(conf, ExecDriver.class);
     execContext = new ExecMapperContext(job);
     //we don't use the HadoopJobExecHooks for local tasks

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java
index 7ada611..1634f42 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HashTableLoader.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.HashTableSinkOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapredContext;
@@ -183,7 +184,7 @@ public class HashTableLoader implements org.apache.hadoop.hive.ql.exec.HashTable
     JobConf job = new JobConf(hconf);
     MapredLocalTask localTask = new MapredLocalTask(localWork, job, false);
 
-    HashTableSinkOperator sink = new TemporaryHashSinkOperator(desc);
+    HashTableSinkOperator sink = new TemporaryHashSinkOperator(new CompilationOpContext(), desc);
     sink.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>(directWorks));
 
     for (Operator<?> operator : directWorks) {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java
index 62be3f8..d8fe35f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkMapRecordHandler.java
@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
 import org.apache.hadoop.hive.ql.exec.MapredContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -72,10 +73,11 @@ public class SparkMapRecordHandler extends SparkRecordHandler {
       // create map and fetch operators
       MapWork mrwork = Utilities.getMapWork(job);
 
+      CompilationOpContext runtimeCtx = new CompilationOpContext();
       if (mrwork.getVectorMode()) {
-        mo = new VectorMapOperator();
+        mo = new VectorMapOperator(runtimeCtx);
       } else {
-        mo = new MapOperator();
+        mo = new MapOperator(runtimeCtx);
       }
       mo.setConf(mrwork);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
index 31eee45..eb93aca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkTask.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
@@ -88,8 +89,9 @@ public class SparkTask extends Task<SparkWork> {
   private SparkCounters sparkCounters;
 
   @Override
-  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext) {
-    super.initialize(conf, queryPlan, driverContext);
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext driverContext,
+      CompilationOpContext opContext) {
+    super.initialize(conf, queryPlan, driverContext, opContext);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
index ee62ab3..a4cf0b1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java
@@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
@@ -174,10 +175,11 @@ public class MapRecordProcessor extends RecordProcessor {
 
     try {
 
+      CompilationOpContext runtimeCtx = new CompilationOpContext();
       if (mapWork.getVectorMode()) {
-        mapOp = new VectorMapOperator();
+        mapOp = new VectorMapOperator(runtimeCtx);
       } else {
-        mapOp = new MapOperator();
+        mapOp = new MapOperator(runtimeCtx);
       }
 
       mapOp.clearConnectedOperators();
@@ -188,9 +190,9 @@ public class MapRecordProcessor extends RecordProcessor {
         for (BaseWork mergeWork : mergeWorkList) {
           MapWork mergeMapWork = (MapWork) mergeWork;
           if (mergeMapWork.getVectorMode()) {
-            mergeMapOp = new VectorMapOperator();
+            mergeMapOp = new VectorMapOperator(runtimeCtx);
           } else {
-            mergeMapOp = new MapOperator();
+            mergeMapOp = new MapOperator(runtimeCtx);
           }
 
           mergeMapOpList.add(mergeMapOp);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java
index e4ca2cd..c591288 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorAppMasterEventOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.AppMasterEventOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
@@ -52,14 +53,20 @@ public class VectorAppMasterEventOperator extends AppMasterEventOperator {
 
   protected transient Object[] singleRow;
 
-  public VectorAppMasterEventOperator(VectorizationContext vContext,
-      OperatorDesc conf) {
-    super();
+  public VectorAppMasterEventOperator(
+      CompilationOpContext ctx, VectorizationContext vContext, OperatorDesc conf) {
+    super(ctx);
     this.conf = (AppMasterEventDesc) conf;
     this.vContext = vContext;
   }
 
-  public VectorAppMasterEventOperator() {
+  /** Kryo ctor. */
+  protected VectorAppMasterEventOperator() {
+    super();
+  }
+
+  public VectorAppMasterEventOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFileSinkOperator.java
index 09d4a8e..f09534c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFileSinkOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
@@ -47,15 +48,20 @@ public class VectorFileSinkOperator extends FileSinkOperator {
 
   protected transient Object[] singleRow;
 
-  public VectorFileSinkOperator(VectorizationContext vContext,
-      OperatorDesc conf) {
-    super();
+  public VectorFileSinkOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) {
+    this(ctx);
     this.conf = (FileSinkDesc) conf;
     this.vContext = vContext;
   }
 
-  public VectorFileSinkOperator() {
+  /** Kryo ctor. */
+  protected VectorFileSinkOperator() {
+    super();
+  }
 
+  public VectorFileSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
index 58f1190..74a0947 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
@@ -23,6 +23,7 @@ import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.FilterOperator;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.ConstantVectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
@@ -47,18 +48,23 @@ public class VectorFilterOperator extends FilterOperator {
   // and 0 if condition needs to be computed.
   transient private int filterMode = 0;
 
-  public VectorFilterOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
-    this();
+  public VectorFilterOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
     ExprNodeDesc oldExpression = ((FilterDesc) conf).getPredicate();
     conditionEvaluator = vContext.getVectorExpression(oldExpression, VectorExpressionDescriptor.Mode.FILTER);
     this.conf = (FilterDesc) conf;
   }
 
-  public VectorFilterOperator() {
+  /** Kryo ctor. */
+  protected VectorFilterOperator() {
     super();
   }
 
+  public VectorFilterOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
 
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
index 0524c08..b7ce309 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
@@ -33,6 +33,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.KeyWrapper;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
@@ -749,9 +750,9 @@ public class VectorGroupByOperator extends Operator<GroupByDesc> implements
 
   private static final long serialVersionUID = 1L;
 
-  public VectorGroupByOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
-    this();
+  public VectorGroupByOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
     GroupByDesc desc = (GroupByDesc) conf;
     this.conf = desc;
     List<ExprNodeDesc> keysDesc = desc.getKeys();
@@ -769,10 +770,16 @@ public class VectorGroupByOperator extends Operator<GroupByDesc> implements
     vOutContext = new VectorizationContext(getName(), desc.getOutputColumnNames());
   }
 
-  public VectorGroupByOperator() {
+  /** Kryo ctor. */
+  protected VectorGroupByOperator() {
     super();
   }
 
+  public VectorGroupByOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorLimitOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorLimitOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorLimitOperator.java
index 4cb91d4..154c647 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorLimitOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorLimitOperator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.LimitDesc;
@@ -30,11 +31,18 @@ public class VectorLimitOperator extends LimitOperator  {
 
   private static final long serialVersionUID = 1L;
 
-  public VectorLimitOperator() {
+  /** Kryo ctor. */
+  protected VectorLimitOperator() {
     super();
   }
 
-  public VectorLimitOperator(VectorizationContext vContext, OperatorDesc conf) {
+  public VectorLimitOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorLimitOperator(
+      CompilationOpContext ctx, VectorizationContext vContext, OperatorDesc conf) {
+    this(ctx);
     this.conf = (LimitDesc) conf;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinBaseOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinBaseOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinBaseOperator.java
index 4b1d9ad..6bed52f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinBaseOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinBaseOperator.java
@@ -26,6 +26,7 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.persistence.HybridHashTableContainer;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer;
@@ -64,13 +65,18 @@ public class VectorMapJoinBaseOperator extends MapJoinOperator implements Vector
 
   protected transient int tag;  // big table alias
 
-  public VectorMapJoinBaseOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinBaseOperator() {
     super();
   }
 
-  public VectorMapJoinBaseOperator (VectorizationContext vContext, OperatorDesc conf)
-    throws HiveException {
-    super();
+  public VectorMapJoinBaseOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinBaseOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx);
 
     MapJoinDesc desc = (MapJoinDesc) conf;
     this.conf = desc;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
index 8bbf020..622f777 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
@@ -27,6 +27,7 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinTableContainer;
@@ -74,15 +75,20 @@ public class VectorMapJoinOperator extends VectorMapJoinBaseOperator {
   private VectorExpressionWriter[] rowWriters;  // Writer for producing row from input batch
   protected transient Object[] singleRow;
 
-  public VectorMapJoinOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOperator() {
     super();
   }
 
+  public VectorMapJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
 
-  public VectorMapJoinOperator (VectorizationContext vContext, OperatorDesc conf)
-    throws HiveException {
+  public VectorMapJoinOperator (CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
 
-    super(vContext, conf);
+    super(ctx, vContext, conf);
 
     MapJoinDesc desc = (MapJoinDesc) conf;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOuterFilteredOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOuterFilteredOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOuterFilteredOperator.java
index b8b1f88..509a43f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOuterFilteredOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOuterFilteredOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
@@ -48,13 +49,18 @@ public class VectorMapJoinOuterFilteredOperator extends VectorMapJoinBaseOperato
 
   protected transient Object[] singleRow;
 
-  public VectorMapJoinOuterFilteredOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOuterFilteredOperator() {
     super();
   }
 
-  public VectorMapJoinOuterFilteredOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinOuterFilteredOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinOuterFilteredOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
 
     this.vContext = vContext;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
index aa0d5a5..033be38 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapOperator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.MapOperator;
 import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -27,6 +28,15 @@ public class VectorMapOperator extends MapOperator {
 
   private static final long serialVersionUID = 1L;
 
+  /** Kryo ctor. */
+  protected VectorMapOperator() {
+    super();
+  }
+
+  public VectorMapOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   public void process(Writable value) throws HiveException {
     // A mapper can span multiple files/partitions.

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java
index 41b2874..b79a3d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -44,18 +45,23 @@ public class VectorReduceSinkOperator extends ReduceSinkOperator {
 
   protected transient Object[] singleRow;
 
-  public VectorReduceSinkOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
-    this();
+  public VectorReduceSinkOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
     ReduceSinkDesc desc = (ReduceSinkDesc) conf;
     this.conf = desc;
     this.vContext = vContext;
   }
 
-  public VectorReduceSinkOperator() {
+  /** Kryo ctor. */
+  protected VectorReduceSinkOperator() {
     super();
   }
 
+  public VectorReduceSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
index 9ff9b77..9a263e6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Future;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
@@ -89,13 +90,18 @@ public class VectorSMBMapJoinOperator extends SMBMapJoinOperator implements Vect
     List<Object> evaluate(VectorHashKeyWrapper kw) throws HiveException;
 }
 
-  public VectorSMBMapJoinOperator() {
+  /** Kryo ctor. */
+  protected VectorSMBMapJoinOperator() {
     super();
   }
 
-  public VectorSMBMapJoinOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
-    this();
+  public VectorSMBMapJoinOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorSMBMapJoinOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
     SMBJoinDesc desc = (SMBJoinDesc) conf;
     this.conf = desc;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
index 73b905f..8db6eba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSelectOperator.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
@@ -53,8 +54,9 @@ public class VectorSelectOperator extends Operator<SelectDesc> implements
   // Create a new outgoing vectorization context because column name map will change.
   private VectorizationContext vOutContext;
 
-  public VectorSelectOperator(VectorizationContext vContext, OperatorDesc conf)
-      throws HiveException {
+  public VectorSelectOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    this(ctx);
     this.conf = (SelectDesc) conf;
     List<ExprNodeDesc> colList = this.conf.getColList();
     vExpressions = new VectorExpression[colList.size()];
@@ -79,7 +81,13 @@ public class VectorSelectOperator extends Operator<SelectDesc> implements
     }
   }
 
-  public VectorSelectOperator() {
+  /** Kryo ctor. */
+  protected VectorSelectOperator() {
+    super();
+  }
+
+  public VectorSelectOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java
index 8486d12..1e550e7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkHashTableSinkOperator.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.SparkHashTableSinkOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -50,11 +51,18 @@ public class VectorSparkHashTableSinkOperator extends SparkHashTableSinkOperator
 
   protected transient Object[] singleRow;
 
-  public VectorSparkHashTableSinkOperator() {
+  /** Kryo ctor. */
+  protected VectorSparkHashTableSinkOperator() {
+    super();
   }
 
-  public VectorSparkHashTableSinkOperator(VectorizationContext vContext, OperatorDesc conf) {
-    super();
+  public VectorSparkHashTableSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorSparkHashTableSinkOperator(
+      CompilationOpContext ctx, VectorizationContext vContext, OperatorDesc conf) {
+    this(ctx);
     this.vContext = vContext;
     this.conf = (SparkHashTableSinkDesc) conf;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java
index eb0b408..2f02250 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSparkPartitionPruningSinkOperator.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.concurrent.Future;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
@@ -46,14 +47,20 @@ public class VectorSparkPartitionPruningSinkOperator extends SparkPartitionPruni
 
   protected transient Object[] singleRow;
 
-  public VectorSparkPartitionPruningSinkOperator(VectorizationContext context,
-      OperatorDesc conf) {
-    super();
+  public VectorSparkPartitionPruningSinkOperator(CompilationOpContext ctx,
+      VectorizationContext context, OperatorDesc conf) {
+    this(ctx);
     this.conf = (SparkPartitionPruningSinkDesc) conf;
     this.vContext = context;
   }
 
-  public VectorSparkPartitionPruningSinkOperator() {
+  /** Kryo ctor. */
+  protected VectorSparkPartitionPruningSinkOperator() {
+    super();
+  }
+
+  public VectorSparkPartitionPruningSinkOperator(CompilationOpContext ctx) {
+    super(ctx);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
index 2502ae2..e26e31b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
@@ -27,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.HashTableLoaderFactory;
 import org.apache.hadoop.hive.ql.exec.HashTableLoader;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
@@ -176,13 +177,18 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
   // The small table hash table for the native vectorized map join operator.
   protected transient VectorMapJoinHashTable vectorMapJoinHashTable;
 
-  public VectorMapJoinCommonOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinCommonOperator() {
     super();
   }
 
-  public VectorMapJoinCommonOperator(VectorizationContext vContext, OperatorDesc conf)
-          throws HiveException {
-    super();
+  public VectorMapJoinCommonOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinCommonOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx);
 
     MapJoinDesc desc = (MapJoinDesc) conf;
     this.conf = desc;

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
index c1c137b..5cbace4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinGenerateResultOperator.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.commons.lang.ArrayUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.persistence.HybridHashTableContainer;
 import org.apache.hadoop.hive.ql.exec.persistence.HybridHashTableContainer.HashPartition;
 import org.apache.hadoop.hive.ql.exec.persistence.MapJoinBytesTableContainer;
@@ -86,13 +87,18 @@ public abstract class VectorMapJoinGenerateResultOperator extends VectorMapJoinC
   // Debug display.
   protected transient long batchCounter;
 
-  public VectorMapJoinGenerateResultOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinGenerateResultOperator() {
     super();
   }
 
-  public VectorMapJoinGenerateResultOperator(VectorizationContext vContext, OperatorDesc conf)
-              throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinGenerateResultOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   protected void commonSetup(VectorizedRowBatch batch) throws HiveException {

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
index 6b33a39..dfb5bf8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyGenerateResultOperator.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -84,13 +85,18 @@ public abstract class VectorMapJoinInnerBigOnlyGenerateResultOperator
   // Pre-allocated member for storing index into the hashMultiSetResults for each spilled row.
   protected transient int[] spillHashMapResultIndices;
 
-  public VectorMapJoinInnerBigOnlyGenerateResultOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerBigOnlyGenerateResultOperator() {
     super();
   }
 
-  public VectorMapJoinInnerBigOnlyGenerateResultOperator(VectorizationContext vContext, OperatorDesc conf)
-              throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerBigOnlyGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerBigOnlyGenerateResultOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
index 9e77d22..0bba141 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyLongOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -72,12 +73,18 @@ public class VectorMapJoinInnerBigOnlyLongOperator extends VectorMapJoinInnerBig
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerBigOnlyLongOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerBigOnlyLongOperator() {
     super();
   }
 
-  public VectorMapJoinInnerBigOnlyLongOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerBigOnlyLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerBigOnlyLongOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
index e4f6c5d..621804b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyMultiKeyOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -77,12 +78,18 @@ public class VectorMapJoinInnerBigOnlyMultiKeyOperator extends VectorMapJoinInne
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerBigOnlyMultiKeyOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerBigOnlyMultiKeyOperator() {
     super();
   }
 
-  public VectorMapJoinInnerBigOnlyMultiKeyOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerBigOnlyMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerBigOnlyMultiKeyOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
index 2711b10..10e75ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerBigOnlyStringOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -68,12 +69,18 @@ public class VectorMapJoinInnerBigOnlyStringOperator extends VectorMapJoinInnerB
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerBigOnlyStringOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerBigOnlyStringOperator() {
     super();
   }
 
-  public VectorMapJoinInnerBigOnlyStringOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerBigOnlyStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerBigOnlyStringOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerGenerateResultOperator.java
index 36d0611..319a2b0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerGenerateResultOperator.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
@@ -89,13 +90,18 @@ public abstract class VectorMapJoinInnerGenerateResultOperator
   // Pre-allocated member for storing index into the hashMapResults for each spilled row.
   protected transient int[] spillHashMapResultIndices;
 
-  public VectorMapJoinInnerGenerateResultOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerGenerateResultOperator() {
     super();
   }
 
-  public VectorMapJoinInnerGenerateResultOperator(VectorizationContext vContext, OperatorDesc conf)
-              throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerGenerateResultOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
index 0197225..804d69c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerLongOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -71,12 +72,18 @@ public class VectorMapJoinInnerLongOperator extends VectorMapJoinInnerGenerateRe
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerLongOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerLongOperator() {
     super();
   }
 
-  public VectorMapJoinInnerLongOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerLongOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
index 837d97b..fcfa0bd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerMultiKeyOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -75,12 +76,18 @@ public class VectorMapJoinInnerMultiKeyOperator extends VectorMapJoinInnerGenera
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerMultiKeyOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerMultiKeyOperator() {
     super();
   }
 
-  public VectorMapJoinInnerMultiKeyOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerMultiKeyOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
index b2711c3..0f9baae 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinInnerStringOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -67,12 +68,18 @@ public class VectorMapJoinInnerStringOperator extends VectorMapJoinInnerGenerate
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinInnerStringOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinInnerStringOperator() {
     super();
   }
 
-  public VectorMapJoinInnerStringOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinInnerStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinInnerStringOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiGenerateResultOperator.java
index d1d6c42..c71ebba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiGenerateResultOperator.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -70,13 +71,18 @@ public abstract class VectorMapJoinLeftSemiGenerateResultOperator
   // Pre-allocated member for storing index into the hashSetResults for each spilled row.
   protected transient int[] spillHashMapResultIndices;
 
-  public VectorMapJoinLeftSemiGenerateResultOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinLeftSemiGenerateResultOperator() {
     super();
   }
 
-  public VectorMapJoinLeftSemiGenerateResultOperator(VectorizationContext vContext, OperatorDesc conf)
-              throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinLeftSemiGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinLeftSemiGenerateResultOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
index 4b8ab58..1149a9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiLongOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -72,12 +73,18 @@ public class VectorMapJoinLeftSemiLongOperator extends VectorMapJoinLeftSemiGene
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinLeftSemiLongOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinLeftSemiLongOperator() {
     super();
   }
 
-  public VectorMapJoinLeftSemiLongOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinLeftSemiLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinLeftSemiLongOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
index bdf7901..e0baebc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiMultiKeyOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -76,12 +77,18 @@ public class VectorMapJoinLeftSemiMultiKeyOperator extends VectorMapJoinLeftSemi
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinLeftSemiMultiKeyOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinLeftSemiMultiKeyOperator() {
     super();
   }
 
-  public VectorMapJoinLeftSemiMultiKeyOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinLeftSemiMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinLeftSemiMultiKeyOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
index a8d3459..49e1177 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinLeftSemiStringOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -68,12 +69,18 @@ public class VectorMapJoinLeftSemiStringOperator extends VectorMapJoinLeftSemiGe
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinLeftSemiStringOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinLeftSemiStringOperator() {
     super();
   }
 
-  public VectorMapJoinLeftSemiStringOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinLeftSemiStringOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinLeftSemiStringOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
index 5a88784..0e2d65a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterGenerateResultOperator.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.vector.mapjoin;
 import java.io.IOException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
@@ -111,13 +112,18 @@ public abstract class VectorMapJoinOuterGenerateResultOperator
   protected transient int[] noMatchs;
   protected transient int[] merged;
 
-  public VectorMapJoinOuterGenerateResultOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOuterGenerateResultOperator() {
     super();
   }
 
-  public VectorMapJoinOuterGenerateResultOperator(VectorizationContext vContext, OperatorDesc conf)
-              throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinOuterGenerateResultOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinOuterGenerateResultOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
index 5b687fd..58bd0ab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterLongOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
@@ -72,12 +73,18 @@ public class VectorMapJoinOuterLongOperator extends VectorMapJoinOuterGenerateRe
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinOuterLongOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOuterLongOperator() {
     super();
   }
 
-  public VectorMapJoinOuterLongOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinOuterLongOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinOuterLongOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/88fceaca/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
index e212a2a..7f9afd2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinOuterMultiKeyOperator.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
@@ -75,12 +76,18 @@ public class VectorMapJoinOuterMultiKeyOperator extends VectorMapJoinOuterGenera
   // Pass-thru constructors.
   //
 
-  public VectorMapJoinOuterMultiKeyOperator() {
+  /** Kryo ctor. */
+  protected VectorMapJoinOuterMultiKeyOperator() {
     super();
   }
 
-  public VectorMapJoinOuterMultiKeyOperator(VectorizationContext vContext, OperatorDesc conf) throws HiveException {
-    super(vContext, conf);
+  public VectorMapJoinOuterMultiKeyOperator(CompilationOpContext ctx) {
+    super(ctx);
+  }
+
+  public VectorMapJoinOuterMultiKeyOperator(CompilationOpContext ctx,
+      VectorizationContext vContext, OperatorDesc conf) throws HiveException {
+    super(ctx, vContext, conf);
   }
 
   //---------------------------------------------------------------------------