http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStatsReaderHook.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStatsReaderHook.java 
b/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStatsReaderHook.java
new file mode 100644
index 0000000..6b5fa1f
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/stats/OperatorStatsReaderHook.java
@@ -0,0 +1,96 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
+import org.apache.hadoop.hive.ql.exec.tez.TezTask;
+import org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext;
+import org.apache.hadoop.hive.ql.hooks.HookContext;
+import org.apache.hadoop.hive.ql.hooks.HookContext.HookType;
+import org.apache.hadoop.hive.ql.hooks.PrivateHookContext;
+import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OperatorStatsReaderHook implements ExecuteWithHookContext {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(OperatorStatsReaderHook.class);
+  private boolean collectOnSuccess;
+
+  @Override
+  public void run(HookContext hookContext) throws Exception {
+
+    if (hookContext.getHookType() == HookType.PRE_EXEC_HOOK) {
+      return;
+    }
+    if (hookContext.getHookType() == HookType.POST_EXEC_HOOK && 
!isCollectOnSuccess()) {
+      return;
+    }
+
+    HiveConf conf = hookContext.getConf();
+    QueryPlan plan = hookContext.getQueryPlan();
+    List<TezTask> rootTasks = Utilities.getTezTasks(plan.getRootTasks());
+    for (TezTask tezTask : rootTasks) {
+      List<BaseWork> baseWorks = tezTask.getWork().getAllWork();
+      for (BaseWork baseWork : baseWorks) {
+        String vertexName = baseWork.getName();
+        LOG.debug("Reading runtime statistics for tez vertex task: {}", 
vertexName);
+        TezCounters counters = tezTask.getTezCounters();
+        if (counters != null) {
+          String groupName = HiveConf.getVar(conf, 
HiveConf.ConfVars.HIVECOUNTERGROUP);
+          for (Operator<? extends OperatorDesc> op : 
baseWork.getAllOperators()) {
+            String operatorId = op.getOperatorId();
+            OperatorStats operatorStats = null;
+            String counterName = 
Operator.Counter.RECORDS_OUT_OPERATOR.toString() + "_" + operatorId;
+            TezCounter tezCounter = 
counters.getGroup(groupName).findCounter(counterName, false);
+            if (tezCounter != null) {
+              if (operatorStats == null) {
+                operatorStats = new OperatorStats(operatorId);
+              }
+              operatorStats.setOutputRecords(tezCounter.getValue());
+            }
+
+            if (operatorStats != null) {
+              ((PrivateHookContext) 
hookContext).getContext().getPlanMapper().link(op, operatorStats);
+            } else {
+              LOG.debug("Unable to get statistics for vertex: {} opId: {} 
groupName: {}", vertexName, operatorId,
+                  groupName);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  public boolean isCollectOnSuccess() {
+    return collectOnSuccess;
+  }
+
+  public void setCollectOnSuccess(boolean collectOnSuccess) {
+    this.collectOnSuccess = collectOnSuccess;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAssertTrueOOM.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAssertTrueOOM.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAssertTrueOOM.java
new file mode 100644
index 0000000..c5c7383
--- /dev/null
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAssertTrueOOM.java
@@ -0,0 +1,74 @@
+/*
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.exec.mapjoin.MapJoinMemoryExhaustionError;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorConverters;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.BooleanWritable;
+
+@UDFType(deterministic = false)
+public class GenericUDFAssertTrueOOM extends GenericUDF {
+  private ObjectInspectorConverters.Converter conditionConverter = null;
+
+  public GenericUDFAssertTrueOOM() {
+  }
+
+  @Override
+  public ObjectInspector initialize(ObjectInspector[] arguments) throws 
UDFArgumentException {
+    HiveConf conf = SessionState.getSessionConf();
+    if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_IN_TEST)) {
+      throw new RuntimeException("this UDF is only available in testmode");
+    }
+    if (arguments.length != 1) {
+      throw new UDFArgumentLengthException("ASSERT_TRUE_OOM() expects one 
argument.");
+    }
+    if (arguments[0].getCategory() != Category.PRIMITIVE) {
+      throw new UDFArgumentTypeException(0, "Argument to ASSERT_TRUE_OOM() 
should be primitive.");
+    }
+    conditionConverter =
+        ObjectInspectorConverters.getConverter(arguments[0], 
PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
+
+    return PrimitiveObjectInspectorFactory.writableVoidObjectInspector;
+  }
+
+  @Override
+  public Object evaluate(DeferredObject[] arguments) throws HiveException {
+    BooleanWritable condition = (BooleanWritable) 
conditionConverter.convert(arguments[0].get());
+    if (condition == null || !condition.get()) {
+      throw new MapJoinMemoryExhaustionError("assert_true_oom: assertation 
failed; Simulated OOM");
+    }
+    return null;
+  }
+
+  @Override
+  public String getDisplayString(String[] children) {
+    return getStandardDisplayString("assert_true_oom", children);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
index 6c469bf..e8516fb 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFEnforceNotNullConstraint.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/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 fa3f89f..e65288b 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
@@ -41,8 +41,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorFactory;
-import 
org.apache.hadoop.hive.ql.exec.util.collectoroperator.RowVectorCollectorTestOperator;
-import org.apache.hadoop.hive.ql.exec.util.rowobjects.RowTestObjects;
 import 
org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFCountStar;
 import 
org.apache.hadoop.hive.ql.exec.vector.util.FakeCaptureVectorToRowOutputOperator;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromConcat;
@@ -61,7 +59,6 @@ import org.apache.hadoop.hive.ql.plan.VectorGroupByDesc;
 import org.apache.hadoop.hive.ql.plan.VectorGroupByDesc.ProcessingMode;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount;
-import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount.GenericUDAFCountEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMin;
@@ -75,7 +72,6 @@ import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import 
org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -2555,7 +2551,7 @@ public class TestVectorGroupByOperator {
         (VectorGroupByOperator) Vectorizer.vectorizeGroupByOperator(groupByOp, 
ctx, vectorDesc);
 
     FakeCaptureVectorToRowOutputOperator out = 
FakeCaptureVectorToRowOutputOperator.addCaptureOutputChild(cCtx, vgo);
-    vgo.initialize(null, null);
+    vgo.initialize(hconf, null);
 
     for (VectorizedRowBatch unit: data) {
       vgo.process(unit,  0);

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOpSigFactory.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOpSigFactory.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOpSigFactory.java
new file mode 100644
index 0000000..5d6a274
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOpSigFactory.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.optimizer.signature;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+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.AbstractOperatorDesc;
+import org.apache.hadoop.hive.ql.plan.api.OperatorType;
+import org.junit.Rule;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.Spy;
+import org.mockito.junit.MockitoJUnit;
+import org.mockito.junit.MockitoRule;
+
+
+public class TestOpSigFactory {
+  CompilationOpContext cCtx = new CompilationOpContext();
+
+  @Rule
+  public MockitoRule a = MockitoJUnit.rule();
+
+  @Spy
+  OpTreeSignatureFactory f = OpTreeSignatureFactory.newCache();
+
+  public static class SampleDesc extends AbstractOperatorDesc {
+    private static final long serialVersionUID = 1L;
+    private int desc_invocations;
+
+    @Signature
+    public int asd() {
+      desc_invocations++;
+      return 8;
+    }
+
+    public int getDesc_invocations() {
+      return desc_invocations;
+    }
+  }
+
+  static class SampleOperator extends Operator<SampleDesc> {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void process(Object row, int tag) throws HiveException {
+    }
+
+    @Override
+    public String getName() {
+      return "A1";
+    }
+
+    @Override
+    public OperatorType getType() {
+      return OperatorType.FILTER;
+    }
+  }
+
+  @Test
+  public void checkExplicit() {
+    SampleOperator so = new SampleOperator();
+    SampleDesc sd = new SampleDesc();
+    so.setConf(sd);
+
+    f.getSignature(so);
+    f.getSignature(so);
+
+    verify(f, times(2)).getSignature(Mockito.any());
+    assertEquals(1, sd.getDesc_invocations());
+  }
+
+  @Test
+  public void checkImplicit() {
+    SampleOperator so = new SampleOperator();
+    SampleDesc sd = new SampleDesc();
+    so.setConf(sd);
+
+    SampleOperator so2 = new SampleOperator();
+    SampleDesc sd2 = new SampleDesc();
+    so2.setConf(sd2);
+
+    so.getParentOperators().add(so2);
+    so2.getChildOperators().add(so);
+
+    f.getSignature(so);
+    // computes the sig of every object
+    verify(f, times(2)).getSignature(Mockito.any());
+    assertEquals(1, sd.getDesc_invocations());
+    assertEquals(1, sd2.getDesc_invocations());
+
+    f.getSignature(so);
+    f.getSignature(so2);
+
+    verify(f, times(4)).getSignature(Mockito.any());
+    assertEquals(1, sd.getDesc_invocations());
+    assertEquals(1, sd2.getDesc_invocations());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOperatorSignature.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOperatorSignature.java
 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOperatorSignature.java
new file mode 100644
index 0000000..8c899e7
--- /dev/null
+++ 
b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/signature/TestOperatorSignature.java
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.optimizer.signature;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.FilterDesc;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestOperatorSignature {
+
+  GenericUDF udf = new GenericUDFConcat();
+
+  CompilationOpContext cCtx = new CompilationOpContext();
+
+  @Test
+  public void testFilterOpEquals() {
+    Operator<? extends OperatorDesc> op7 = getFilterOp(7);
+    Operator<? extends OperatorDesc> op8 = getFilterOp(8);
+    Operator<? extends OperatorDesc> op7b = getFilterOp(7);
+
+    checkEquals(op7, op7b);
+    checkNotEquals(op7, op8);
+  }
+
+  @Test
+  public void testTree1() {
+    Operator<?> tr37 = getFilTsOp(3, 7);
+    Operator<?> tr37a = getFilTsOp(3, 7);
+    Operator<?> tr17 = getFilTsOp(1, 7);
+    Operator<?> tr31 = getFilTsOp(3, 1);
+
+    checkEquals(tr37, tr37a);
+
+    checkTreeNotEquals(tr37, tr17);
+    checkTreeEquals(tr37, tr37a);
+    checkTreeNotEquals(tr37, tr31);
+  }
+
+  private Operator<?> getFilTsOp(int i, int j) {
+    Operator<TableScanDesc> ts = getTsOp(i);
+    Operator<? extends OperatorDesc> fil = getFilterOp(j);
+
+    ts.getChildOperators().add(fil);
+    fil.getParentOperators().add(ts);
+
+    return fil;
+  }
+
+  @Test
+  public void testTableScand() {
+    Operator<TableScanDesc> t1 = getTsOp(3);
+    Operator<TableScanDesc> t1a = getTsOp(3);
+    Operator<TableScanDesc> t2 = getTsOp(4);
+
+    assertTrue(t1.logicalEquals(t1a));
+    checkEquals(t1, t1a);
+    checkNotEquals(t1, t2);
+  }
+
+  private void checkEquals(Operator<?> o1, Operator<?> o2) {
+    assertTrue(o1.logicalEquals(o2));
+    OpSignature s1 = OpSignature.of(o1);
+    OpSignature s2 = OpSignature.of(o2);
+
+    s1.proveEquals(s2);
+    assertTrue("sigCmp", s1.signatureCompare(s2));
+    assertEquals(s1.hashCode(), s2.hashCode());
+    assertEquals(s1, s2);
+  }
+
+
+  private void checkNotEquals(Operator<? extends OperatorDesc> o1, Operator<? 
extends OperatorDesc> o2) {
+    assertFalse(o1.logicalEquals(o2));
+    OpSignature s1 = OpSignature.of(o1);
+    OpSignature s2 = OpSignature.of(o2);
+
+    assertFalse(s1.signatureCompare(s2));
+    // this might be a little bit too much...but in most cases this should be 
true
+    assertNotEquals(s1.hashCode(), s2.hashCode());
+    assertNotEquals(s1, s2);
+  }
+
+  private void checkTreeEquals(Operator<?> o1, Operator<?> o2) {
+    OpTreeSignature ts1 = OpTreeSignature.of(o1);
+    OpTreeSignature ts2 = OpTreeSignature.of(o2);
+
+    assertEquals(ts1.hashCode(), ts2.hashCode());
+    assertEquals(ts1, ts2);
+  }
+
+  private void checkTreeNotEquals(Operator<? extends OperatorDesc> o1, 
Operator<? extends OperatorDesc> o2) {
+
+    OpTreeSignature ts1 = OpTreeSignature.of(o1);
+    OpTreeSignature ts2 = OpTreeSignature.of(o2);
+
+    assertNotEquals(ts1.hashCode(), ts2.hashCode());
+    assertNotEquals(ts1, ts2);
+  }
+
+
+  private Operator<? extends OperatorDesc> getFilterOp(int constVal) {
+    ExprNodeDesc pred = new ExprNodeConstantDesc(constVal);
+    FilterDesc fd = new FilterDesc(pred, true);
+    Operator<? extends OperatorDesc> op = OperatorFactory.get(cCtx, fd);
+    return op;
+  }
+
+  private Operator<TableScanDesc> getTsOp(int i) {
+    Table tblMetadata = new Table("db", "table");
+    // FIXME: I think this shouldn't be sensitive to the alias; but currently 
its included in logicalEquals...check that
+    TableScanDesc desc = new TableScanDesc("alias"/*+ cCtx.nextOperatorId()*/, 
tblMetadata);
+      List<ExprNodeDesc> as =
+        Lists.newArrayList(new 
ExprNodeConstantDesc(TypeInfoFactory.intTypeInfo, Integer.valueOf(i)),
+          new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, "c1", "aa", 
false));
+    ExprNodeGenericFuncDesc f1 = new 
ExprNodeGenericFuncDesc(TypeInfoFactory.intTypeInfo, udf, as);
+    desc.setFilterExpr(f1);
+    Operator<TableScanDesc> ts = OperatorFactory.get(cCtx, desc);
+    return ts;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java 
b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
new file mode 100644
index 0000000..9fe95e4
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestCounterMapping.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.plan.mapping;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
+import org.apache.hadoop.hive.ql.plan.mapper.SimpleRuntimeStatsSource;
+import org.apache.hadoop.hive.ql.reexec.ReExecDriver;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.LinkGroup;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.stats.OperatorStats;
+import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
+import org.apache.hive.testutils.HiveTestEnvSetup;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+public class TestCounterMapping {
+
+  @ClassRule
+  public static HiveTestEnvSetup env_setup = new HiveTestEnvSetup();
+
+  @Rule
+  public TestRule methodRule = env_setup.getMethodRule();
+
+  static Comparator<Operator<?>> OPERATOR_ID_COMPARATOR = new 
Comparator<Operator<?>>() {
+
+    @Override
+    public int compare(Operator<?> o1, Operator<?> o2) {
+      return Objects.compare(o1.getOperatorId(), o2.getOperatorId(), 
Comparator.naturalOrder());
+    }
+  };
+
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    IDriver driver = createDriver();
+    dropTables(driver);
+    String cmds[] = {
+        // @formatter:off
+        "create table s (x int)",
+        "insert into s values (1),(2),(3),(4),(5),(6),(7),(8),(9),(10)",
+        "create table tu(id_uv int,id_uw int,u int)",
+        "create table tv(id_uv int,v int)",
+        "create table tw(id_uw int,w int)",
+
+        "from s\n" +
+        "insert overwrite table tu\n" +
+        "        select x,x,x\n" +
+        "        where x<=6 or x=10\n" +
+        "insert overwrite table tv\n" +
+        "        select x,x\n" +
+        "        where x<=3 or x=10\n" +
+        "insert overwrite table tw\n" +
+        "        select x,x\n" +
+        "",
+        // @formatter:on
+    };
+    for (String cmd : cmds) {
+      int ret = driver.run(cmd).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    IDriver driver = createDriver();
+    dropTables(driver);
+  }
+
+  public static void dropTables(IDriver driver) throws Exception {
+    String tables[] = { "s", "tu", "tv", "tw" };
+    for (String t : tables) {
+      int ret = driver.run("drop table if exists " + t).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  private PlanMapper getMapperForQuery(IDriver driver, String query) {
+    int ret;
+    ret = driver.run(query).getResponseCode();
+    assertEquals("Checking command success", 0, ret);
+    PlanMapper pm0 = driver.getContext().getPlanMapper();
+    return pm0;
+  }
+
+  @Test
+  public void testUsageOfRuntimeInfo() throws ParseException {
+    IDriver driver = createDriver();
+    String query = "select sum(u) from tu where u>1";
+    PlanMapper pm1 = getMapperForQuery(driver, query);
+
+    List<FilterOperator> filters1 = pm1.getAll(FilterOperator.class);
+    filters1.sort(OPERATOR_ID_COMPARATOR.reversed());
+    FilterOperator filter1 = filters1.get(0);
+
+    driver = createDriver();
+    ((ReExecDriver) driver).setRuntimeStatsSource(new 
SimpleRuntimeStatsSource(pm1));
+
+    PlanMapper pm2 = getMapperForQuery(driver, query);
+
+    List<FilterOperator> filters2 = pm2.getAll(FilterOperator.class);
+    filters2.sort(OPERATOR_ID_COMPARATOR.reversed());
+    FilterOperator filter2 = filters2.get(0);
+
+    assertEquals("original check", 7, filter1.getStatistics().getNumRows());
+    assertEquals("optimized check", 6, filter2.getStatistics().getNumRows());
+
+  }
+
+  @Test
+  @Ignore("needs HiveFilter mapping")
+  public void testMappingJoinLookup() throws ParseException {
+    IDriver driver = createDriver();
+
+    PlanMapper pm0 = getMapperForQuery(driver, "select sum(tu.id_uv),sum(u) 
from tu join tv on (tu.id_uv = tv.id_uv) where u>1 and v>1");
+
+    Iterator<LinkGroup> itG = pm0.iterateGroups();
+    int checkedOperators = 0;
+    while (itG.hasNext()) {
+      LinkGroup g = itG.next();
+      List<HiveFilter> hfs = g.getAll(HiveFilter.class);
+      List<OperatorStats> oss = g.getAll(OperatorStats.class);
+      List<FilterOperator> fos = g.getAll(FilterOperator.class);
+
+      if (fos.size() > 0 && oss.size() > 0) {
+        if (hfs.size() == 0) {
+          fail("HiveFilter is not connected?");
+        }
+        OperatorStats os = oss.get(0);
+        if (!(os.getOutputRecords() == 3 || os.getOutputRecords() == 6)) {
+          fail("nonexpected number of records produced");
+        }
+        checkedOperators++;
+      }
+    }
+    assertEquals(2, checkedOperators);
+  }
+
+  private static IDriver createDriver() {
+    //    HiveConf conf = new HiveConf(Driver.class);
+    HiveConf conf = env_setup.getTestCtx().hiveConf;
+    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, true);
+    
conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, 
true);
+    conf.set("hive.auto.convert.join", "false");
+    conf.set("hive.optimize.ppd", "false");
+
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        
"org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    //    conf.setVar(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, 
CheckInputReadEntityDirect.class.getName());
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, 
false);
+    HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, 
OperatorStatsReaderHook.class.getName());
+    SessionState.start(conf);
+
+    IDriver driver = DriverFactory.newDriver(conf);
+    return driver;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java 
b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java
new file mode 100644
index 0000000..cfb0ca3
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestOperatorCmp.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.plan.mapping;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Iterator;
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.parse.ParseException;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.LinkGroup;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
+import org.apache.hive.testutils.HiveTestEnvSetup;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+public class TestOperatorCmp {
+
+  @ClassRule
+  public static HiveTestEnvSetup env_setup = new HiveTestEnvSetup();
+
+  @Rule
+  public TestRule methodRule = env_setup.getMethodRule();
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    IDriver driver = createDriver();
+    dropTables(driver);
+    String cmds[] = {
+        // @formatter:off
+        "create table tu(id_uv int,id_uw int,u int)",
+        "create table tv(id_uv int,v int)",
+        "create table tw(id_uw int,w int)",
+
+        "insert into tu values 
(10,10,10),(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6)",
+        "insert into tv values (10,10),(1,1),(2,2),(3,3)",
+        "insert into tw values 
(10,10),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9)",
+        // @formatter:on
+    };
+    for (String cmd : cmds) {
+      int ret = driver.run(cmd).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    IDriver driver = createDriver();
+    dropTables(driver);
+  }
+
+  public static void dropTables(IDriver driver) throws Exception {
+    String tables[] = { "tu", "tv", "tw" };
+    for (String t : tables) {
+      int ret = driver.run("drop table if exists " + t).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  private PlanMapper getMapperForQuery(IDriver driver, String query) {
+    int ret = driver.run(query).getResponseCode();
+    assertEquals("Checking command success", 0, ret);
+    PlanMapper pm0 = driver.getContext().getPlanMapper();
+    return pm0;
+  }
+
+  @Test
+  public void testUnrelatedFiltersAreNotMatched0() throws ParseException {
+    IDriver driver = createDriver();
+    String query = "select u from tu where id_uv = 1 union all select v from 
tv where id_uv = 1";
+
+    PlanMapper pm = getMapperForQuery(driver, query);
+    Iterator<LinkGroup> itG = pm.iterateGroups();
+    List<FilterOperator> fos = pm.getAll(FilterOperator.class);
+    assertEquals(2, fos.size());
+
+    assertFalse("logicalEquals", compareOperators(fos.get(0), fos.get(1)));
+
+  }
+
+  @Test
+  public void testUnrelatedFiltersAreNotMatched1() throws ParseException {
+    IDriver driver = createDriver();
+    PlanMapper pm0 = getMapperForQuery(driver, "select u from tu where id_uv = 
1 group by u");
+    PlanMapper pm1 = getMapperForQuery(driver, "select v from tv where id_uv = 
1 group by v");
+    List<FilterOperator> fos0 = pm0.getAll(FilterOperator.class);
+    List<FilterOperator> fos1 = pm1.getAll(FilterOperator.class);
+    assertEquals(1, fos0.size());
+    assertEquals(1, fos1.size());
+
+    assertFalse("logicalEquals", compareOperators(fos0.get(0), fos1.get(0)));
+  }
+
+  @Test
+  public void testDifferentFiltersAreNotMatched1() throws ParseException {
+    IDriver driver = createDriver();
+    PlanMapper pm0 = getMapperForQuery(driver, "select u from tu where id_uv = 
1 group by u");
+    PlanMapper pm1 = getMapperForQuery(driver, "select u from tu where id_uv = 
2 group by u");
+
+    assertHelper(AssertHelperOp.SAME, pm0, pm1, TableScanOperator.class);
+    assertHelper(AssertHelperOp.NOT_SAME, pm0, pm1, FilterOperator.class);
+
+  }
+
+  @Test
+  public void testSameFiltersMatched1() throws ParseException, Exception {
+    IDriver driver = createDriver();
+    PlanMapper pm0 = getMapperForQuery(driver, "select u from tu where id_uv = 
1 group by u");
+    PlanMapper pm1 = getMapperForQuery(driver, "select u from tu where id_uv = 
1 group by u");
+
+    assertHelper(AssertHelperOp.SAME, pm0, pm1, FilterOperator.class);
+    assertHelper(AssertHelperOp.SAME, pm0, pm1, TableScanOperator.class);
+  }
+
+
+  enum AssertHelperOp {
+    SAME, NOT_SAME
+  };
+
+  private <T extends Operator<?>> void assertHelper(AssertHelperOp 
same,PlanMapper pm0, PlanMapper pm1, Class<T> clazz) {
+    List<T> fos0 = pm0.getAll(clazz);
+    List<T> fos1 = pm1.getAll(clazz);
+    assertEquals(1, fos0.size());
+    assertEquals(1, fos1.size());
+
+    if (same == AssertHelperOp.SAME) {
+      assertTrue(clazz + " " + same, compareOperators(fos0.get(0), 
fos1.get(0)));
+    } else {
+      assertFalse(clazz + " " + same, compareOperators(fos0.get(0), 
fos1.get(0)));
+    }
+  }
+
+  private boolean compareOperators(Operator<?> opL, Operator<?> opR) {
+    return opL.logicalEqualsTree(opR);
+  }
+
+  private static IDriver createDriver() {
+    HiveConf conf = env_setup.getTestCtx().hiveConf;
+
+    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, true);
+    
conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ALWAYS_COLLECT_OPERATOR_STATS, 
true);
+    conf.setVar(ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, "reoptimize");
+    conf.set("zzz", "1");
+    conf.set("reexec.overlay.zzz", "2000");
+    //
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        
"org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, 
false);
+    HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, 
OperatorStatsReaderHook.class.getName());
+    SessionState.start(conf);
+
+    IDriver driver = DriverFactory.newDriver(conf);
+    return driver;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
----------------------------------------------------------------------
diff --git 
a/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java 
b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
new file mode 100644
index 0000000..6d7bb07
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/mapping/TestReOptimization.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hive.ql.plan.mapping;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.DriverFactory;
+import org.apache.hadoop.hive.ql.IDriver;
+import org.apache.hadoop.hive.ql.exec.FilterOperator;
+import org.apache.hadoop.hive.ql.plan.Statistics;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper;
+import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.LinkGroup;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.stats.OperatorStats;
+import org.apache.hadoop.hive.ql.stats.OperatorStatsReaderHook;
+import org.apache.hive.testutils.HiveTestEnvSetup;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+public class TestReOptimization {
+
+  @ClassRule
+  public static HiveTestEnvSetup env_setup = new HiveTestEnvSetup();
+
+  @Rule
+  public TestRule methodRule = env_setup.getMethodRule();
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    IDriver driver = createDriver("");
+    dropTables(driver);
+    String cmds[] = {
+        // @formatter:off
+        "create table tu(id_uv int,id_uw int,u int)",
+        "create table tv(id_uv int,v int)",
+        "create table tw(id_uw int,w int)",
+
+        "insert into tu values 
(10,10,10),(1,1,1),(2,2,2),(3,3,3),(4,4,4),(5,5,5),(6,6,6)",
+        "insert into tv values (10,10),(1,1),(2,2),(3,3)",
+        "insert into tw values 
(10,10),(1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(8,8),(9,9)",
+        // @formatter:on
+    };
+    for (String cmd : cmds) {
+      int ret = driver.run(cmd).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    IDriver driver = createDriver("");
+    dropTables(driver);
+  }
+
+  public static void dropTables(IDriver driver) throws Exception {
+    String tables[] = { "tu", "tv", "tw" };
+    for (String t : tables) {
+      int ret = driver.run("drop table if exists " + t).getResponseCode();
+      assertEquals("Checking command success", 0, ret);
+    }
+  }
+
+  private PlanMapper getMapperForQuery(IDriver driver, String query) throws 
CommandProcessorResponse {
+    CommandProcessorResponse res = driver.run(query);
+    if (res.getResponseCode() != 0) {
+      throw res;
+    }
+    PlanMapper pm0 = driver.getContext().getPlanMapper();
+    return pm0;
+  }
+
+  @Test
+  public void testStatsAreSetInReopt() throws Exception {
+    IDriver driver = createDriver("overlay,reoptimize");
+    String query = "select assert_true_oom(${hiveconf:zzz} > sum(u*v)) from tu 
join tv on (tu.id_uv=tv.id_uv) where u<10 and v>1";
+
+    PlanMapper pm = getMapperForQuery(driver, query);
+    Iterator<LinkGroup> itG = pm.iterateGroups();
+    int checkedOperators = 0;
+    while (itG.hasNext()) {
+      LinkGroup g = itG.next();
+      List<FilterOperator> fos = g.getAll(FilterOperator.class);
+      List<OperatorStats> oss = g.getAll(OperatorStats.class);
+      // FIXME: oss seems to contain duplicates
+
+      if (fos.size() > 0 && oss.size() > 0) {
+        fos.sort(TestCounterMapping.OPERATOR_ID_COMPARATOR.reversed());
+
+        FilterOperator fo = fos.get(0);
+        OperatorStats os = oss.get(0);
+
+        Statistics stats = fo.getStatistics();
+        assertEquals(os.getOutputRecords(), stats.getNumRows());
+
+        if (!(os.getOutputRecords() == 3 || os.getOutputRecords() == 6)) {
+          fail("nonexpected number of records produced");
+        }
+        checkedOperators++;
+      }
+    }
+    assertEquals(2, checkedOperators);
+  }
+
+  @Test
+  public void testReExecutedIfMapJoinError() throws Exception {
+
+    IDriver driver = createDriver("overlay,reoptimize");
+    String query =
+        "select assert_true_oom(${hiveconf:zzz}>sum(1)) from tu join tv on 
(tu.id_uv=tv.id_uv) where u<10 and v>1";
+    PlanMapper pm = getMapperForQuery(driver, query);
+
+  }
+
+  @Test(expected = CommandProcessorResponse.class)
+  public void testNotReExecutedIfAssertionError() throws Exception {
+    IDriver driver = createDriver("reoptimize");
+    String query =
+        "select assert_true(${hiveconf:zzz}>sum(1)) from tu join tv on 
(tu.id_uv=tv.id_uv) where u<10 and v>1";
+
+    PlanMapper pm = getMapperForQuery(driver, query);
+    assertEquals(1, driver.getContext().getExecutionIndex());
+  }
+
+  @Test
+  public void testExplainSupport() throws Exception {
+
+    IDriver driver = createDriver("overlay,reoptimize");
+    String query = "explain reoptimization select 1 from tu join tv on 
(tu.id_uv=tv.id_uv) where u<10 and v>1";
+    PlanMapper pm = getMapperForQuery(driver, query);
+    List<String> res = new ArrayList<>();
+    List<String> res1 = new ArrayList<>();
+    while (driver.getResults(res1)) {
+      res.addAll(res1);
+    }
+
+    assertEquals("2TS", 2, res.stream().filter(line -> 
line.contains("TS_")).count());
+    assertEquals("2TS(runtime)", 2,
+        res.stream().filter(line -> line.contains("TS") && 
line.contains("runtime")).count());
+
+  }
+
+  private static IDriver createDriver(String strategies) {
+    HiveConf conf = env_setup.getTestCtx().hiveConf;
+
+    conf.setBoolVar(ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, true);
+    conf.setVar(ConfVars.HIVE_QUERY_REEXECUTION_STRATEGIES, strategies);
+    conf.setBoolVar(ConfVars.HIVE_EXPLAIN_USER, true);
+    conf.set("zzz", "1");
+    conf.set("reexec.overlay.zzz", "2000");
+    //
+    conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+        
"org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory");
+    HiveConf.setBoolVar(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, 
false);
+    HiveConf.setVar(conf, HiveConf.ConfVars.POSTEXECHOOKS, 
OperatorStatsReaderHook.class.getName());
+    SessionState.start(conf);
+
+    IDriver driver = DriverFactory.newDriver(conf);
+    return driver;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hive/testutils/HiveTestEnvSetup.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hive/testutils/HiveTestEnvSetup.java 
b/ql/src/test/org/apache/hive/testutils/HiveTestEnvSetup.java
index cc97317..f872da0 100644
--- a/ql/src/test/org/apache/hive/testutils/HiveTestEnvSetup.java
+++ b/ql/src/test/org/apache/hive/testutils/HiveTestEnvSetup.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/org/apache/hive/testutils/TestHiveTestEnvSetup.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hive/testutils/TestHiveTestEnvSetup.java 
b/ql/src/test/org/apache/hive/testutils/TestHiveTestEnvSetup.java
index 9161366..c34affd 100644
--- a/ql/src/test/org/apache/hive/testutils/TestHiveTestEnvSetup.java
+++ b/ql/src/test/org/apache/hive/testutils/TestHiveTestEnvSetup.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/queries/clientnegative/bad_exec_hooks.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/bad_exec_hooks.q 
b/ql/src/test/queries/clientnegative/bad_exec_hooks.q
index 709d8d9..ee6f73f 100644
--- a/ql/src/test/queries/clientnegative/bad_exec_hooks.q
+++ b/ql/src/test/queries/clientnegative/bad_exec_hooks.q
@@ -1,3 +1,4 @@
+set hive.query.reexecution.enabled=false;
 set hive.exec.pre.hooks="org.this.is.a.bad.class";
 
 EXPLAIN

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/queries/clientpositive/hook_order.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/hook_order.q 
b/ql/src/test/queries/clientpositive/hook_order.q
index b6753b3..7602731 100644
--- a/ql/src/test/queries/clientpositive/hook_order.q
+++ b/ql/src/test/queries/clientpositive/hook_order.q
@@ -1,3 +1,5 @@
+set hive.query.reexecution.enabled=false;
+
 SET 
hive.exec.pre.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond;
 SET 
hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirst,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecond;
 SET 
hive.semantic.analyzer.hook=org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunFirstSemanticAnalysisHook,org.apache.hadoop.hive.ql.hooks.VerifyHooksRunInOrder$RunSecondSemanticAnalysisHook;

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/queries/clientpositive/retry_failure.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/retry_failure.q 
b/ql/src/test/queries/clientpositive/retry_failure.q
new file mode 100644
index 0000000..0b9922b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/retry_failure.q
@@ -0,0 +1,11 @@
+create table tx(a int,f string);
+insert into tx values (1,'non_existent_file');
+
+set zzz=1;
+set reexec.overlay.zzz=2;
+
+set hive.query.reexecution.enabled=true;
+set hive.query.reexecution.strategies=overlay;
+
+select assert_true(${hiveconf:zzz} > a) from tx group by a;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/queries/clientpositive/retry_failure_oom.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/retry_failure_oom.q 
b/ql/src/test/queries/clientpositive/retry_failure_oom.q
new file mode 100644
index 0000000..21c9f8c
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/retry_failure_oom.q
@@ -0,0 +1,12 @@
+
+create table tx(a int,f string);
+insert into tx values (1,'non_existent_file');
+
+set zzz=1;
+set reexec.overlay.zzz=2;
+
+set hive.query.reexecution.enabled=true;
+set hive.query.reexecution.strategies=overlay,reoptimize;
+
+select assert_true_oom(${hiveconf:zzz} > a) from tx group by a;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q 
b/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
new file mode 100644
index 0000000..c594255
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/retry_failure_stat_changes.q
@@ -0,0 +1,29 @@
+
+create table tx(a int,u int);
+insert into tx values (1,1),(2,2),(3,3),(4,4),(5,5),(6,6),(7,7),(10,10);
+
+create table px(a int,p int);
+insert into px values (2,2),(3,3),(5,5),(7,7),(11,11);
+
+
+set hive.explain.user=true;
+set hive.query.reexecution.enabled=true;
+set hive.query.reexecution.strategies=overlay,reoptimize;
+
+explain REOPTIMIZATION 
+select sum(u*p) from tx join px on (u=p) where u<10 and p>2;
+
+set hive.auto.convert.join=false;
+explain analyze
+select sum(u*p) from tx join px on (u=p) where u<10 and p>2;
+set hive.auto.convert.join=true;
+explain analyze
+select sum(u*p) from tx join px on (u=p) where u<10 and p>2;
+
+set zzz=1;
+set reexec.overlay.zzz=2000;
+
+explain
+select assert_true_oom(${hiveconf:zzz} > sum(u*p)) from tx join px on 
(tx.a=px.a) where u<10 and p>2;
+select assert_true_oom(${hiveconf:zzz} > sum(u*p)) from tx join px on 
(tx.a=px.a) where u<10 and p>2;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out 
b/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
index 8699160..18f4c69 100644
--- a/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
+++ b/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
@@ -23,6 +23,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 57
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 57
+   RECORDS_OUT_OPERATOR_GBY_2: 57
+   RECORDS_OUT_OPERATOR_GBY_4: 57
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 57
+   RECORDS_OUT_OPERATOR_SEL_1: 84
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 57
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -41,6 +53,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 121
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 189
+   RECORDS_OUT_OPERATOR_FS_4: 189
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 189
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -67,6 +91,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 121
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 189
+   RECORDS_OUT_OPERATOR_FS_4: 189
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 189
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -85,6 +121,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 184
    RECORDS_OUT_1_default.src2: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 292
+   RECORDS_OUT_OPERATOR_FS_4: 292
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 292
+   RECORDS_OUT_OPERATOR_SEL_5: 184
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -121,6 +169,22 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_2_default.src3: 105
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 105
+   RECORDS_OUT_OPERATOR_FIL_9: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_FS_8: 105
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_SEL_6: 105
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-2 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -143,6 +207,22 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_2_default.src3: 208
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 208
+   RECORDS_OUT_OPERATOR_FIL_9: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_FS_8: 208
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 184
+   RECORDS_OUT_OPERATOR_SEL_6: 208
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-2 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -175,6 +255,21 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 64
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 84
+   RECORDS_OUT_OPERATOR_FIL_11: 105
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_FS_9: 189
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 226
+   RECORDS_OUT_OPERATOR_SEL_7: 189
+   RECORDS_OUT_OPERATOR_TS_0: 500
+   RECORDS_OUT_OPERATOR_TS_3: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    GROUPED_INPUT_SPLITS_Map_4: 1
@@ -203,6 +298,21 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 127
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 84
+   RECORDS_OUT_OPERATOR_FIL_11: 208
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_FS_9: 292
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 392
+   RECORDS_OUT_OPERATOR_SEL_7: 292
+   RECORDS_OUT_OPERATOR_TS_0: 500
+   RECORDS_OUT_OPERATOR_TS_3: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    GROUPED_INPUT_SPLITS_Map_4: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out 
b/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
index 9f6426c..28f376f 100644
--- a/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
+++ b/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
@@ -23,6 +23,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 57
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 57
+   RECORDS_OUT_OPERATOR_GBY_2: 57
+   RECORDS_OUT_OPERATOR_GBY_4: 57
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 57
+   RECORDS_OUT_OPERATOR_SEL_1: 84
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 57
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -41,6 +53,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 121
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 189
+   RECORDS_OUT_OPERATOR_FS_4: 189
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 189
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -67,6 +91,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 121
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 189
+   RECORDS_OUT_OPERATOR_FS_4: 189
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 189
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -85,6 +121,18 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_0: 184
    RECORDS_OUT_1_default.src2: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_5: 292
+   RECORDS_OUT_OPERATOR_FS_4: 292
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 292
+   RECORDS_OUT_OPERATOR_SEL_5: 184
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -121,6 +169,22 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_2_default.src3: 105
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 105
+   RECORDS_OUT_OPERATOR_FIL_9: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_FS_8: 105
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 121
+   RECORDS_OUT_OPERATOR_SEL_6: 105
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-2 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -143,6 +207,22 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 84
    RECORDS_OUT_2_default.src3: 208
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 208
+   RECORDS_OUT_OPERATOR_FIL_9: 84
+   RECORDS_OUT_OPERATOR_FS_4: 84
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_FS_8: 208
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 184
+   RECORDS_OUT_OPERATOR_SEL_6: 208
+   RECORDS_OUT_OPERATOR_TS_0: 500
 Stage-2 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    INPUT_DIRECTORIES_Map_1: 1
@@ -175,6 +255,21 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 64
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 84
+   RECORDS_OUT_OPERATOR_FIL_11: 105
+   RECORDS_OUT_OPERATOR_FS_6: 121
+   RECORDS_OUT_OPERATOR_FS_9: 189
+   RECORDS_OUT_OPERATOR_GBY_2: 121
+   RECORDS_OUT_OPERATOR_GBY_4: 121
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 121
+   RECORDS_OUT_OPERATOR_SEL_1: 189
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 226
+   RECORDS_OUT_OPERATOR_SEL_7: 189
+   RECORDS_OUT_OPERATOR_TS_0: 500
+   RECORDS_OUT_OPERATOR_TS_3: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    GROUPED_INPUT_SPLITS_Map_4: 1
@@ -203,6 +298,21 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 127
+   RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
+   RECORDS_OUT_OPERATOR_FIL_10: 84
+   RECORDS_OUT_OPERATOR_FIL_11: 208
+   RECORDS_OUT_OPERATOR_FS_6: 184
+   RECORDS_OUT_OPERATOR_FS_9: 292
+   RECORDS_OUT_OPERATOR_GBY_2: 184
+   RECORDS_OUT_OPERATOR_GBY_4: 184
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 184
+   RECORDS_OUT_OPERATOR_SEL_1: 292
+   RECORDS_OUT_OPERATOR_SEL_2: 84
+   RECORDS_OUT_OPERATOR_SEL_5: 392
+   RECORDS_OUT_OPERATOR_SEL_7: 292
+   RECORDS_OUT_OPERATOR_TS_0: 500
+   RECORDS_OUT_OPERATOR_TS_3: 500
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 1
    GROUPED_INPUT_SPLITS_Map_4: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out 
b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out
index cd3023f..af237ea 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap_counters.q.out
@@ -261,6 +261,13 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2100
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_9: 1
+   RECORDS_OUT_OPERATOR_GBY_8: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_7: 2100
+   RECORDS_OUT_OPERATOR_SEL_6: 2100
+   RECORDS_OUT_OPERATOR_TS_0: 2100
 Stage-1 LLAP IO COUNTERS:
    ALLOCATED_BYTES: 262144
    ALLOCATED_USED_BYTES: 26
@@ -289,6 +296,9 @@ Stage-1 FILE SYSTEM COUNTERS:
 Stage-1 HIVE COUNTERS:
    CREATED_FILES: 1
    RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 0
    INPUT_DIRECTORIES_Map_1: 1
@@ -311,6 +321,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 8
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 8
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 8
+   RECORDS_OUT_OPERATOR_SEL_9: 8
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    ALLOCATED_BYTES: 1048576
    ALLOCATED_USED_BYTES: 2732
@@ -343,6 +361,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 22
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 22
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 22
+   RECORDS_OUT_OPERATOR_SEL_9: 22
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -373,6 +399,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 16
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 16
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 16
+   RECORDS_OUT_OPERATOR_SEL_9: 16
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -403,6 +437,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 18
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 18
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 18
+   RECORDS_OUT_OPERATOR_SEL_9: 18
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -433,6 +475,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 1
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 1
+   RECORDS_OUT_OPERATOR_SEL_9: 1
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -463,6 +513,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 32
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 32
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 32
+   RECORDS_OUT_OPERATOR_SEL_9: 32
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -493,6 +551,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 32
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 32
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 32
+   RECORDS_OUT_OPERATOR_SEL_9: 32
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -523,6 +589,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 1697
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 1697
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 1697
+   RECORDS_OUT_OPERATOR_SEL_9: 1697
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -553,6 +627,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 12
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 12
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 12
+   RECORDS_OUT_OPERATOR_SEL_9: 12
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -583,6 +665,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 1713
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 1713
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 1713
+   RECORDS_OUT_OPERATOR_SEL_9: 1713
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -613,6 +703,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 6
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 6
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 6
+   RECORDS_OUT_OPERATOR_SEL_9: 6
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -643,6 +741,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 50
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_7: 50
+   RECORDS_OUT_OPERATOR_FS_9: 1
+   RECORDS_OUT_OPERATOR_GBY_8: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 50
+   RECORDS_OUT_OPERATOR_SEL_2: 50
+   RECORDS_OUT_OPERATOR_TS_0: 1100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -673,6 +779,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 318
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 318
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 318
+   RECORDS_OUT_OPERATOR_SEL_9: 318
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0
@@ -700,6 +814,9 @@ Stage-1 FILE SYSTEM COUNTERS:
 Stage-1 HIVE COUNTERS:
    CREATED_FILES: 1
    RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 0
    INPUT_DIRECTORIES_Map_1: 1
@@ -719,6 +836,9 @@ Stage-1 FILE SYSTEM COUNTERS:
 Stage-1 HIVE COUNTERS:
    CREATED_FILES: 1
    RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 0
    INPUT_DIRECTORIES_Map_1: 1
@@ -741,6 +861,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 0
+   RECORDS_OUT_OPERATOR_SEL_9: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 830
    CACHE_MISS_BYTES: 0
@@ -768,6 +896,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_7: 0
+   RECORDS_OUT_OPERATOR_FS_9: 1
+   RECORDS_OUT_OPERATOR_GBY_8: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_3: 0
+   RECORDS_OUT_OPERATOR_SEL_2: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 830
    CACHE_MISS_BYTES: 0
@@ -792,6 +928,9 @@ Stage-1 FILE SYSTEM COUNTERS:
 Stage-1 HIVE COUNTERS:
    CREATED_FILES: 1
    RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 0
    INPUT_DIRECTORIES_Map_1: 1
@@ -814,6 +953,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    ALLOCATED_BYTES: 1310720
    ALLOCATED_USED_BYTES: 13796
@@ -846,6 +993,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 6
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 6
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 6
+   RECORDS_OUT_OPERATOR_SEL_9: 6
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -876,6 +1031,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 6
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 6
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 6
+   RECORDS_OUT_OPERATOR_SEL_9: 6
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -903,6 +1066,9 @@ Stage-1 FILE SYSTEM COUNTERS:
 Stage-1 HIVE COUNTERS:
    CREATED_FILES: 1
    RECORDS_OUT_0: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
 Stage-1 INPUT COUNTERS:
    GROUPED_INPUT_SPLITS_Map_1: 0
    INPUT_DIRECTORIES_Map_1: 1
@@ -925,6 +1091,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2100
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2100
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2100
+   RECORDS_OUT_OPERATOR_SEL_9: 2100
+   RECORDS_OUT_OPERATOR_TS_0: 2100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -955,6 +1129,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 0
+   RECORDS_OUT_OPERATOR_SEL_9: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1735
    CACHE_MISS_BYTES: 0
@@ -982,6 +1164,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1012,6 +1202,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1042,6 +1240,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1072,6 +1278,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 81
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 81
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 81
+   RECORDS_OUT_OPERATOR_SEL_9: 81
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1102,6 +1316,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 74
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 74
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 74
+   RECORDS_OUT_OPERATOR_SEL_9: 74
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1132,6 +1354,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 12
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 12
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 12
+   RECORDS_OUT_OPERATOR_SEL_9: 12
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1162,6 +1392,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 13
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 13
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 13
+   RECORDS_OUT_OPERATOR_SEL_9: 13
+   RECORDS_OUT_OPERATOR_TS_0: 2000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1192,6 +1430,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 1
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 1
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 1
+   RECORDS_OUT_OPERATOR_SEL_9: 1
+   RECORDS_OUT_OPERATOR_TS_0: 100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1222,6 +1468,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 7
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 7
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 7
+   RECORDS_OUT_OPERATOR_SEL_9: 7
+   RECORDS_OUT_OPERATOR_TS_0: 1100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1252,6 +1506,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 0
+   RECORDS_OUT_OPERATOR_SEL_9: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1735
    CACHE_MISS_BYTES: 0
@@ -1279,6 +1541,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 0
+   RECORDS_OUT_OPERATOR_SEL_9: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1735
    CACHE_MISS_BYTES: 0
@@ -1306,6 +1576,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 0
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 0
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 0
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 0
+   RECORDS_OUT_OPERATOR_SEL_9: 0
+   RECORDS_OUT_OPERATOR_TS_0: 0
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1735
    CACHE_MISS_BYTES: 0
@@ -1333,6 +1611,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1363,6 +1649,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 6
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 6
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 6
+   RECORDS_OUT_OPERATOR_SEL_9: 6
+   RECORDS_OUT_OPERATOR_TS_0: 1100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 5715
    CACHE_MISS_BYTES: 0
@@ -1393,6 +1687,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 1000
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 1000
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 6770
    CACHE_MISS_BYTES: 0
@@ -1423,6 +1725,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2
+   RECORDS_OUT_OPERATOR_SEL_9: 2
+   RECORDS_OUT_OPERATOR_TS_0: 100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 6770
    CACHE_MISS_BYTES: 0

http://git-wip-us.apache.org/repos/asf/hive/blob/31e36f01/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out 
b/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out
index e7a1a5b..01fb4af 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap_counters1.q.out
@@ -261,6 +261,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2094
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2094
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2094
+   RECORDS_OUT_OPERATOR_SEL_9: 2094
+   RECORDS_OUT_OPERATOR_TS_0: 2100
 Stage-1 LLAP IO COUNTERS:
    ALLOCATED_BYTES: 1310720
    ALLOCATED_USED_BYTES: 2758
@@ -293,6 +301,14 @@ Stage-1 HIVE COUNTERS:
    RECORDS_IN_Map_1: 2100
    RECORDS_OUT_0: 1
    RECORDS_OUT_INTERMEDIATE_Map_1: 2094
+   RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
+   RECORDS_OUT_OPERATOR_FIL_8: 2094
+   RECORDS_OUT_OPERATOR_FS_12: 1
+   RECORDS_OUT_OPERATOR_GBY_11: 1
+   RECORDS_OUT_OPERATOR_MAP_0: 0
+   RECORDS_OUT_OPERATOR_RS_10: 2094
+   RECORDS_OUT_OPERATOR_SEL_9: 2094
+   RECORDS_OUT_OPERATOR_TS_0: 2100
 Stage-1 LLAP IO COUNTERS:
    CACHE_HIT_BYTES: 1079
    CACHE_MISS_BYTES: 0

Reply via email to