You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2014/05/22 03:14:38 UTC

[01/24] git commit: Enhance plan format to number operator ids and fragment ids.

Repository: incubator-drill
Updated Branches:
  refs/heads/diagnostics2 [created] c34a190fc


Enhance plan format to number operator ids and fragment ids.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/1bc276dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/1bc276dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/1bc276dd

Branch: refs/heads/diagnostics2
Commit: 1bc276ddf5987054098d5a9497dbe71f661c4f88
Parents: 70fab8c
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri May 16 08:47:19 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:11:21 2014 -0700

----------------------------------------------------------------------
 .../planner/physical/BroadcastExchangePrel.java |   2 +-
 .../exec/planner/physical/ExchangePrel.java     |  18 ++
 .../physical/HashToMergeExchangePrel.java       |   2 +-
 .../physical/HashToRandomExchangePrel.java      |   2 +-
 .../physical/OrderedPartitionExchangePrel.java  |   2 +-
 .../exec/planner/physical/PrelVisitor.java      |   1 +
 .../physical/SelectionVectorPrelVisitor.java    |   5 +
 .../physical/SingleMergeExchangePrel.java       |   2 +-
 .../planner/physical/UnionExchangePrel.java     |   2 +-
 .../physical/explain/NumberingRelWriter.java    | 193 ++++++++++++++++
 .../planner/physical/explain/PrelSequencer.java | 222 +++++++++++++++++++
 .../planner/sql/handlers/DefaultSqlHandler.java |   7 +
 12 files changed, 452 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
index a0704b7..e0f3ee1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
@@ -36,7 +36,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class BroadcastExchangePrel extends SinglePrel{
+public class BroadcastExchangePrel extends ExchangePrel{
 
   public BroadcastExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
     super(cluster, traitSet, input);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
new file mode 100644
index 0000000..360fa80
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.planner.physical;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public abstract class ExchangePrel extends SinglePrel{
+
+  public ExchangePrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitExchange(this, value);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
index 45bc390..262fd8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
@@ -39,7 +39,7 @@ import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataTypeField;
 
 
-public class HashToMergeExchangePrel extends SinglePrel {
+public class HashToMergeExchangePrel extends ExchangePrel {
 
   private final List<DistributionField> distFields;
   private int numEndPoints = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
index d582684..ec9ed79 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -41,7 +41,7 @@ import org.eigenbase.reltype.RelDataTypeField;
 import org.eigenbase.rex.RexNode;
 
 
-public class HashToRandomExchangePrel extends SinglePrel {
+public class HashToRandomExchangePrel extends ExchangePrel {
 
   private final List<DistributionField> fields;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
index f551a27..36a67cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
@@ -33,7 +33,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class OrderedPartitionExchangePrel extends SinglePrel {
+public class OrderedPartitionExchangePrel extends ExchangePrel {
 
   public OrderedPartitionExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
     super(cluster, traitSet, input);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
index ab22b97..862b133 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
@@ -21,6 +21,7 @@ package org.apache.drill.exec.planner.physical;
 public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
 
+  public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
   public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
index aa7e91c..224ab76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
@@ -34,6 +34,11 @@ public class SelectionVectorPrelVisitor implements PrelVisitor<Prel, Void, Runti
   }
 
   @Override
+  public Prel visitExchange(ExchangePrel prel, Void value) throws RuntimeException {
+    return visitPrel(prel, value);
+  }
+
+  @Override
   public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
     SelectionVectorMode[] encodings = prel.getSupportedEncodings();
     List<RelNode> children = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
index 9b93058..05d6e89 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
@@ -38,7 +38,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class SingleMergeExchangePrel extends SinglePrel {
+public class SingleMergeExchangePrel extends ExchangePrel {
 
   private final RelCollation collation ;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
index c2cf685..5d6b85d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
@@ -35,7 +35,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class UnionExchangePrel extends SinglePrel {
+public class UnionExchangePrel extends ExchangePrel {
 
   public UnionExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
     super(cluster, traitSet, input);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
new file mode 100644
index 0000000..6522ad9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
@@ -0,0 +1,193 @@
+/**
+ * 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.drill.exec.planner.physical.explain;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.linq4j.Ord;
+import net.hydromatic.optiq.runtime.Spacer;
+
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.explain.PrelSequencer.OpId;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
+import org.eigenbase.rel.metadata.RelMetadataQuery;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.SqlExplainLevel;
+import org.eigenbase.util.Pair;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Copied mostly from RelWriterImpl but customized to create user useful ids.
+ */
+class NumberingRelWriter implements RelWriter {
+  //~ Instance fields --------------------------------------------------------
+
+  protected final PrintWriter pw;
+  private final SqlExplainLevel detailLevel;
+  protected final Spacer spacer = new Spacer();
+  private final List<Pair<String, Object>> values = new ArrayList<Pair<String, Object>>();
+
+  private final Map<Prel, OpId> ids;
+  //~ Constructors -----------------------------------------------------------
+
+  public NumberingRelWriter(Map<Prel, OpId> ids, PrintWriter pw, SqlExplainLevel detailLevel) {
+    this.pw = pw;
+    this.ids = ids;
+    this.detailLevel = detailLevel;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  protected void explain_(
+      RelNode rel,
+      List<Pair<String, Object>> values) {
+    List<RelNode> inputs = rel.getInputs();
+
+    if (!RelMetadataQuery.isVisibleInExplain(
+        rel,
+        detailLevel)) {
+      // render children in place of this, at same level
+      explainInputs(inputs);
+      return;
+    }
+
+    StringBuilder s = new StringBuilder();
+    OpId id = ids.get(rel);
+    s.append(String.format("%02d-%02d",id.fragmentId, id.opId));
+    s.append("  ");
+    if(id.opId == 0){
+      for(int i =0; i < spacer.get(); i++){ s.append('-');}
+    }else{
+
+      spacer.spaces(s);
+    }
+
+    s.append("  ");
+
+    s.append(rel.getRelTypeName().replace("Prel", ""));
+    if (detailLevel != SqlExplainLevel.NO_ATTRIBUTES) {
+      int j = 0;
+      for (Pair<String, Object> value : values) {
+        if (value.right instanceof RelNode) {
+          continue;
+        }
+        if (j++ == 0) {
+          s.append("(");
+        } else {
+          s.append(", ");
+        }
+        s.append(value.left)
+            .append("=[")
+            .append(value.right)
+            .append("]");
+      }
+      if (j > 0) {
+        s.append(")");
+      }
+    }
+    if (detailLevel == SqlExplainLevel.ALL_ATTRIBUTES) {
+      s.append(": rowcount = ")
+          .append(RelMetadataQuery.getRowCount(rel))
+          .append(", cumulative cost = ")
+          .append(RelMetadataQuery.getCumulativeCost(rel));
+       s.append(", id = ").append(rel.getId());
+    }
+    pw.println(s);
+    spacer.add(2);
+    explainInputs(inputs);
+    spacer.subtract(2);
+  }
+
+  private void explainInputs(List<RelNode> inputs) {
+    for (RelNode input : inputs) {
+      input.explain(this);
+    }
+  }
+
+  public final void explain(RelNode rel, List<Pair<String, Object>> valueList) {
+    explain_(rel, valueList);
+  }
+
+  public SqlExplainLevel getDetailLevel() {
+    return detailLevel;
+  }
+
+  public RelWriter input(String term, RelNode input) {
+    values.add(Pair.of(term, (Object) input));
+    return this;
+  }
+
+  public RelWriter item(String term, Object value) {
+    values.add(Pair.of(term, value));
+    return this;
+  }
+
+  public RelWriter itemIf(String term, Object value, boolean condition) {
+    if (condition) {
+      item(term, value);
+    }
+    return this;
+  }
+
+  public RelWriter done(RelNode node) {
+    int i = 0;
+    if (values.size() > 0 && values.get(0).left.equals("subset")) {
+      ++i;
+    }
+    for (RelNode input : node.getInputs()) {
+      assert values.get(i).right == input;
+      ++i;
+    }
+    for (RexNode expr : node.getChildExps()) {
+      assert values.get(i).right == expr;
+      ++i;
+    }
+    final List<Pair<String, Object>> valuesCopy =
+        ImmutableList.copyOf(values);
+    values.clear();
+    explain_(node, valuesCopy);
+    pw.flush();
+    return this;
+  }
+
+  public boolean nest() {
+    return false;
+  }
+
+  /**
+   * Converts the collected terms and values to a string. Does not write to
+   * the parent writer.
+   */
+  public String simple() {
+    final StringBuilder buf = new StringBuilder("(");
+    for (Ord<Pair<String, Object>> ord : Ord.zip(values)) {
+      if (ord.i > 0) {
+        buf.append(", ");
+      }
+      buf.append(ord.e.left).append("=[").append(ord.e.right).append("]");
+    }
+    buf.append(")");
+    return buf.toString();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
new file mode 100644
index 0000000..169deca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
@@ -0,0 +1,222 @@
+/**
+ * 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.drill.exec.planner.physical.explain;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.PrelVisitor;
+import org.eigenbase.rel.RelWriter;
+import org.eigenbase.sql.SqlExplainLevel;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, RuntimeException>{
+
+  private List<Frag> frags = Lists.newLinkedList();
+
+
+  public static String printWithIds(final Prel rel, SqlExplainLevel explainlevel){
+      if (rel == null) {
+        return null;
+      }
+      PrelSequencer s = new PrelSequencer();
+      final StringWriter sw = new StringWriter();
+      final RelWriter planWriter = new NumberingRelWriter(s.go(rel), new PrintWriter(sw), explainlevel);
+      rel.explain(planWriter);
+      return sw.toString();
+
+  }
+
+
+
+  static class Frag implements Iterable<Frag>{
+    Prel root;
+    int majorFragmentId;
+    final List<Frag> children = Lists.newArrayList();
+    public Frag(Prel root) {
+      super();
+      this.root = root;
+    }
+
+    @Override
+    public Iterator<Frag> iterator() {
+      return children.iterator();
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((children == null) ? 0 : children.hashCode());
+      result = prime * result + majorFragmentId;
+      result = prime * result + ((root == null) ? 0 : root.hashCode());
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      Frag other = (Frag) obj;
+      if (children == null) {
+        if (other.children != null)
+          return false;
+      } else if (!children.equals(other.children))
+        return false;
+      if (majorFragmentId != other.majorFragmentId)
+        return false;
+      if (root == null) {
+        if (other.root != null)
+          return false;
+      } else if (!root.equals(other.root))
+        return false;
+      return true;
+    }
+
+    @Override
+    public String toString() {
+      final int maxLen = 10;
+      return "Frag [root=" + root + ", majorFragmentId=" + majorFragmentId + ", children="
+          + (children != null ? children.subList(0, Math.min(children.size(), maxLen)) : null) + "]";
+    }
+
+
+  }
+
+  static class OpId{
+    int fragmentId;
+    int opId;
+    public OpId(int fragmentId, int opId) {
+      super();
+      this.fragmentId = fragmentId;
+      this.opId = opId;
+    }
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + fragmentId;
+      result = prime * result + opId;
+      return result;
+    }
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      OpId other = (OpId) obj;
+      if (fragmentId != other.fragmentId)
+        return false;
+      if (opId != other.opId)
+        return false;
+      return true;
+    }
+    @Override
+    public String toString() {
+      return fragmentId + ":*:" + opId;
+    }
+
+
+  }
+
+  public Map<Prel, OpId> go(Prel root){
+
+    // get fragments.
+    Frag rootFrag = new Frag(root);
+    frags.add(rootFrag);
+    root.accept(this, rootFrag);
+
+    // do depth first traversal of fragments to assign major fragment ids.
+    Queue<Frag> q = Lists.newLinkedList();
+
+    q.add(rootFrag);
+    int majorFragmentId = 0;
+    while(!q.isEmpty()){
+      Frag frag = q.remove();
+
+      frag.majorFragmentId = majorFragmentId++;
+
+      for(Frag child : frag){
+        q.add(child);
+      }
+    }
+
+    // for each fragment, do a dfs of operators to assign operator ids.
+    Map<Prel, OpId> ids = Maps.newHashMap();
+    for(Frag f : frags){
+      int id = 0;
+      Queue<Prel> ops = Lists.newLinkedList();
+      ops.add(f.root);
+      while(!ops.isEmpty()){
+        Prel p = ops.remove();
+        if(p instanceof ExchangePrel && p != f.root) continue;
+        ids.put(p, new OpId(f.majorFragmentId, id++) );
+
+        List<Prel> children = Lists.reverse(Lists.newArrayList(p.iterator()));
+        for(Prel child : children){
+          ops.add(child);
+        }
+      }
+    }
+
+
+    return ids;
+
+  }
+
+
+  @Override
+  public Void visitExchange(ExchangePrel prel, Frag value) throws RuntimeException {
+    Frag newFrag = new Frag(prel);
+    frags.add(newFrag);
+    value.children.add(newFrag);
+    for(Prel child : prel){
+      child.accept(this, newFrag);
+    }
+
+    return null;
+  }
+
+  @Override
+  public Void visitPrel(Prel prel, Frag value) throws RuntimeException {
+    for(Prel children : prel){
+      children.accept(this, value);
+    }
+    return null;
+  }
+
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1bc276dd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 3fa9861..1cb3cfb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -41,6 +41,7 @@ import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.SelectionVectorPrelVisitor;
+import org.apache.drill.exec.planner.physical.explain.PrelSequencer;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
@@ -70,6 +71,12 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     }
   }
 
+  protected void log(String name, Prel node) {
+    if (logger.isDebugEnabled()) {
+      logger.debug(name + " : \n" + PrelSequencer.printWithIds(node, SqlExplainLevel.ALL_ATTRIBUTES));
+    }
+  }
+
   protected void log(String name, PhysicalPlan plan) throws JsonProcessingException {
     if (logger.isDebugEnabled()) {
       String planText = plan.unparse(context.getConfig().getMapper().writer());


[14/24] diag wip

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 26c74a1..2ec7ba9 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -208,6 +208,124 @@ public final class UserBitShared {
   }
 
   /**
+   * Protobuf enum {@code exec.shared.FragmentState}
+   */
+  public enum FragmentState
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SENDING = 0;</code>
+     */
+    SENDING(0, 0),
+    /**
+     * <code>AWAITING_ALLOCATION = 1;</code>
+     */
+    AWAITING_ALLOCATION(1, 1),
+    /**
+     * <code>RUNNING = 2;</code>
+     */
+    RUNNING(2, 2),
+    /**
+     * <code>FINISHED = 3;</code>
+     */
+    FINISHED(3, 3),
+    /**
+     * <code>CANCELLED = 4;</code>
+     */
+    CANCELLED(4, 4),
+    /**
+     * <code>FAILED = 5;</code>
+     */
+    FAILED(5, 5),
+    ;
+
+    /**
+     * <code>SENDING = 0;</code>
+     */
+    public static final int SENDING_VALUE = 0;
+    /**
+     * <code>AWAITING_ALLOCATION = 1;</code>
+     */
+    public static final int AWAITING_ALLOCATION_VALUE = 1;
+    /**
+     * <code>RUNNING = 2;</code>
+     */
+    public static final int RUNNING_VALUE = 2;
+    /**
+     * <code>FINISHED = 3;</code>
+     */
+    public static final int FINISHED_VALUE = 3;
+    /**
+     * <code>CANCELLED = 4;</code>
+     */
+    public static final int CANCELLED_VALUE = 4;
+    /**
+     * <code>FAILED = 5;</code>
+     */
+    public static final int FAILED_VALUE = 5;
+
+
+    public final int getNumber() { return value; }
+
+    public static FragmentState valueOf(int value) {
+      switch (value) {
+        case 0: return SENDING;
+        case 1: return AWAITING_ALLOCATION;
+        case 2: return RUNNING;
+        case 3: return FINISHED;
+        case 4: return CANCELLED;
+        case 5: return FAILED;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<FragmentState>() {
+            public FragmentState findValueByNumber(int number) {
+              return FragmentState.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(2);
+    }
+
+    private static final FragmentState[] VALUES = values();
+
+    public static FragmentState valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private FragmentState(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:exec.shared.FragmentState)
+  }
+
+  /**
    * Protobuf enum {@code exec.shared.CoreOperatorType}
    */
   public enum CoreOperatorType
@@ -525,7 +643,7 @@ public final class UserBitShared {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(2);
+      return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(3);
     }
 
     private static final CoreOperatorType[] VALUES = values();
@@ -6884,27 +7002,42 @@ public final class UserBitShared {
     com.google.protobuf.ByteString
         getQueryBytes();
 
-    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
+    // optional string plan = 6;
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    boolean hasPlan();
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    java.lang.String getPlan();
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    com.google.protobuf.ByteString
+        getPlanBytes();
+
+    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> 
         getFragmentProfileList();
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index);
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     int getFragmentProfileCount();
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
         getFragmentProfileOrBuilderList();
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
         int index);
@@ -7000,9 +7133,14 @@ public final class UserBitShared {
               break;
             }
             case 50: {
-              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+              bitField0_ |= 0x00000020;
+              plan_ = input.readBytes();
+              break;
+            }
+            case 58: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
                 fragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile>();
-                mutable_bitField0_ |= 0x00000020;
+                mutable_bitField0_ |= 0x00000040;
               }
               fragmentProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.PARSER, extensionRegistry));
               break;
@@ -7015,7 +7153,7 @@ public final class UserBitShared {
         throw new com.google.protobuf.InvalidProtocolBufferException(
             e.getMessage()).setUnfinishedMessage(this);
       } finally {
-        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
           fragmentProfile_ = java.util.Collections.unmodifiableList(fragmentProfile_);
         }
         this.unknownFields = unknownFields.build();
@@ -7163,36 +7301,79 @@ public final class UserBitShared {
       }
     }
 
-    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
-    public static final int FRAGMENT_PROFILE_FIELD_NUMBER = 6;
+    // optional string plan = 6;
+    public static final int PLAN_FIELD_NUMBER = 6;
+    private java.lang.Object plan_;
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    public boolean hasPlan() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    public java.lang.String getPlan() {
+      java.lang.Object ref = plan_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          plan_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string plan = 6;</code>
+     */
+    public com.google.protobuf.ByteString
+        getPlanBytes() {
+      java.lang.Object ref = plan_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        plan_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;
+    public static final int FRAGMENT_PROFILE_FIELD_NUMBER = 7;
     private java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> fragmentProfile_;
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> getFragmentProfileList() {
       return fragmentProfile_;
     }
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
         getFragmentProfileOrBuilderList() {
       return fragmentProfile_;
     }
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     public int getFragmentProfileCount() {
       return fragmentProfile_.size();
     }
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index) {
       return fragmentProfile_.get(index);
     }
     /**
-     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
         int index) {
@@ -7205,6 +7386,7 @@ public final class UserBitShared {
       start_ = 0L;
       end_ = 0L;
       query_ = "";
+      plan_ = "";
       fragmentProfile_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
@@ -7234,8 +7416,11 @@ public final class UserBitShared {
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeBytes(5, getQueryBytes());
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBytes(6, getPlanBytes());
+      }
       for (int i = 0; i < fragmentProfile_.size(); i++) {
-        output.writeMessage(6, fragmentProfile_.get(i));
+        output.writeMessage(7, fragmentProfile_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -7266,9 +7451,13 @@ public final class UserBitShared {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(5, getQueryBytes());
       }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, getPlanBytes());
+      }
       for (int i = 0; i < fragmentProfile_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(6, fragmentProfile_.get(i));
+          .computeMessageSize(7, fragmentProfile_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -7402,9 +7591,11 @@ public final class UserBitShared {
         bitField0_ = (bitField0_ & ~0x00000008);
         query_ = "";
         bitField0_ = (bitField0_ & ~0x00000010);
+        plan_ = "";
+        bitField0_ = (bitField0_ & ~0x00000020);
         if (fragmentProfileBuilder_ == null) {
           fragmentProfile_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
+          bitField0_ = (bitField0_ & ~0x00000040);
         } else {
           fragmentProfileBuilder_.clear();
         }
@@ -7460,10 +7651,14 @@ public final class UserBitShared {
           to_bitField0_ |= 0x00000010;
         }
         result.query_ = query_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.plan_ = plan_;
         if (fragmentProfileBuilder_ == null) {
-          if (((bitField0_ & 0x00000020) == 0x00000020)) {
+          if (((bitField0_ & 0x00000040) == 0x00000040)) {
             fragmentProfile_ = java.util.Collections.unmodifiableList(fragmentProfile_);
-            bitField0_ = (bitField0_ & ~0x00000020);
+            bitField0_ = (bitField0_ & ~0x00000040);
           }
           result.fragmentProfile_ = fragmentProfile_;
         } else {
@@ -7502,11 +7697,16 @@ public final class UserBitShared {
           query_ = other.query_;
           onChanged();
         }
+        if (other.hasPlan()) {
+          bitField0_ |= 0x00000020;
+          plan_ = other.plan_;
+          onChanged();
+        }
         if (fragmentProfileBuilder_ == null) {
           if (!other.fragmentProfile_.isEmpty()) {
             if (fragmentProfile_.isEmpty()) {
               fragmentProfile_ = other.fragmentProfile_;
-              bitField0_ = (bitField0_ & ~0x00000020);
+              bitField0_ = (bitField0_ & ~0x00000040);
             } else {
               ensureFragmentProfileIsMutable();
               fragmentProfile_.addAll(other.fragmentProfile_);
@@ -7519,7 +7719,7 @@ public final class UserBitShared {
               fragmentProfileBuilder_.dispose();
               fragmentProfileBuilder_ = null;
               fragmentProfile_ = other.fragmentProfile_;
-              bitField0_ = (bitField0_ & ~0x00000020);
+              bitField0_ = (bitField0_ & ~0x00000040);
               fragmentProfileBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getFragmentProfileFieldBuilder() : null;
@@ -7848,13 +8048,87 @@ public final class UserBitShared {
         return this;
       }
 
-      // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
+      // optional string plan = 6;
+      private java.lang.Object plan_ = "";
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public boolean hasPlan() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public java.lang.String getPlan() {
+        java.lang.Object ref = plan_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          plan_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public com.google.protobuf.ByteString
+          getPlanBytes() {
+        java.lang.Object ref = plan_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          plan_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public Builder setPlan(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        plan_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public Builder clearPlan() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        plan_ = getDefaultInstance().getPlan();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string plan = 6;</code>
+       */
+      public Builder setPlanBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        plan_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;
       private java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> fragmentProfile_ =
         java.util.Collections.emptyList();
       private void ensureFragmentProfileIsMutable() {
-        if (!((bitField0_ & 0x00000020) == 0x00000020)) {
+        if (!((bitField0_ & 0x00000040) == 0x00000040)) {
           fragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile>(fragmentProfile_);
-          bitField0_ |= 0x00000020;
+          bitField0_ |= 0x00000040;
          }
       }
 
@@ -7862,7 +8136,7 @@ public final class UserBitShared {
           org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> fragmentProfileBuilder_;
 
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> getFragmentProfileList() {
         if (fragmentProfileBuilder_ == null) {
@@ -7872,7 +8146,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public int getFragmentProfileCount() {
         if (fragmentProfileBuilder_ == null) {
@@ -7882,7 +8156,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index) {
         if (fragmentProfileBuilder_ == null) {
@@ -7892,7 +8166,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder setFragmentProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
@@ -7909,7 +8183,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder setFragmentProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
@@ -7923,7 +8197,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder addFragmentProfile(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
         if (fragmentProfileBuilder_ == null) {
@@ -7939,7 +8213,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder addFragmentProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
@@ -7956,7 +8230,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder addFragmentProfile(
           org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
@@ -7970,7 +8244,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder addFragmentProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
@@ -7984,7 +8258,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder addAllFragmentProfile(
           java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> values) {
@@ -7998,12 +8272,12 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder clearFragmentProfile() {
         if (fragmentProfileBuilder_ == null) {
           fragmentProfile_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000020);
+          bitField0_ = (bitField0_ & ~0x00000040);
           onChanged();
         } else {
           fragmentProfileBuilder_.clear();
@@ -8011,7 +8285,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public Builder removeFragmentProfile(int index) {
         if (fragmentProfileBuilder_ == null) {
@@ -8024,14 +8298,14 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder getFragmentProfileBuilder(
           int index) {
         return getFragmentProfileFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
           int index) {
@@ -8041,7 +8315,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
            getFragmentProfileOrBuilderList() {
@@ -8052,14 +8326,14 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder addFragmentProfileBuilder() {
         return getFragmentProfileFieldBuilder().addBuilder(
             org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder addFragmentProfileBuilder(
           int index) {
@@ -8067,7 +8341,7 @@ public final class UserBitShared {
             index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 7;</code>
        */
       public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder> 
            getFragmentProfileBuilderList() {
@@ -8080,7 +8354,7 @@ public final class UserBitShared {
           fragmentProfileBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
               org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder>(
                   fragmentProfile_,
-                  ((bitField0_ & 0x00000020) == 0x00000020),
+                  ((bitField0_ & 0x00000040) == 0x00000040),
                   getParentForChildren(),
                   isClean());
           fragmentProfile_ = null;
@@ -8860,60 +9134,118 @@ public final class UserBitShared {
   public interface MinorFragmentProfileOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional int32 minor_fragment_id = 1;
+    // optional .exec.shared.FragmentState state = 1;
+    /**
+     * <code>optional .exec.shared.FragmentState state = 1;</code>
+     */
+    boolean hasState();
+    /**
+     * <code>optional .exec.shared.FragmentState state = 1;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.FragmentState getState();
+
+    // optional .exec.shared.DrillPBError error = 2;
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    boolean hasError();
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.DrillPBError getError();
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder();
+
+    // optional int32 minor_fragment_id = 3;
     /**
-     * <code>optional int32 minor_fragment_id = 1;</code>
+     * <code>optional int32 minor_fragment_id = 3;</code>
      */
     boolean hasMinorFragmentId();
     /**
-     * <code>optional int32 minor_fragment_id = 1;</code>
+     * <code>optional int32 minor_fragment_id = 3;</code>
      */
     int getMinorFragmentId();
 
-    // repeated .exec.shared.OperatorProfile operator_profile = 2;
+    // repeated .exec.shared.OperatorProfile operator_profile = 4;
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile> 
         getOperatorProfileList();
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.OperatorProfile getOperatorProfile(int index);
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     int getOperatorProfileCount();
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder> 
         getOperatorProfileOrBuilderList();
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder getOperatorProfileOrBuilder(
         int index);
 
-    // optional int64 start_time = 3;
+    // optional int64 start_time = 5;
     /**
-     * <code>optional int64 start_time = 3;</code>
+     * <code>optional int64 start_time = 5;</code>
      */
     boolean hasStartTime();
     /**
-     * <code>optional int64 start_time = 3;</code>
+     * <code>optional int64 start_time = 5;</code>
      */
     long getStartTime();
 
-    // optional int64 end_time = 4;
+    // optional int64 end_time = 6;
     /**
-     * <code>optional int64 end_time = 4;</code>
+     * <code>optional int64 end_time = 6;</code>
      */
     boolean hasEndTime();
     /**
-     * <code>optional int64 end_time = 4;</code>
+     * <code>optional int64 end_time = 6;</code>
      */
     long getEndTime();
+
+    // optional int64 memory_used = 7;
+    /**
+     * <code>optional int64 memory_used = 7;</code>
+     */
+    boolean hasMemoryUsed();
+    /**
+     * <code>optional int64 memory_used = 7;</code>
+     */
+    long getMemoryUsed();
+
+    // optional int64 max_memory_used = 8;
+    /**
+     * <code>optional int64 max_memory_used = 8;</code>
+     */
+    boolean hasMaxMemoryUsed();
+    /**
+     * <code>optional int64 max_memory_used = 8;</code>
+     */
+    long getMaxMemoryUsed();
+
+    // optional .exec.DrillbitEndpoint endpoint = 9;
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    boolean hasEndpoint();
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint();
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder();
   }
   /**
    * Protobuf type {@code exec.shared.MinorFragmentProfile}
@@ -8967,39 +9299,86 @@ public final class UserBitShared {
               break;
             }
             case 8: {
-              bitField0_ |= 0x00000001;
-              minorFragmentId_ = input.readInt32();
+              int rawValue = input.readEnum();
+              org.apache.drill.exec.proto.UserBitShared.FragmentState value = org.apache.drill.exec.proto.UserBitShared.FragmentState.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(1, rawValue);
+              } else {
+                bitField0_ |= 0x00000001;
+                state_ = value;
+              }
               break;
             }
             case 18: {
-              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+              org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = error_.toBuilder();
+              }
+              error_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.DrillPBError.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(error_);
+                error_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              minorFragmentId_ = input.readInt32();
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
                 operatorProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.OperatorProfile>();
-                mutable_bitField0_ |= 0x00000002;
+                mutable_bitField0_ |= 0x00000008;
               }
               operatorProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.OperatorProfile.PARSER, extensionRegistry));
               break;
             }
-            case 24: {
-              bitField0_ |= 0x00000002;
+            case 40: {
+              bitField0_ |= 0x00000008;
               startTime_ = input.readInt64();
               break;
             }
-            case 32: {
-              bitField0_ |= 0x00000004;
+            case 48: {
+              bitField0_ |= 0x00000010;
               endTime_ = input.readInt64();
               break;
             }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
-          operatorProfile_ = java.util.Collections.unmodifiableList(operatorProfile_);
-        }
+            case 56: {
+              bitField0_ |= 0x00000020;
+              memoryUsed_ = input.readInt64();
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000040;
+              maxMemoryUsed_ = input.readInt64();
+              break;
+            }
+            case 74: {
+              org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
+                subBuilder = endpoint_.toBuilder();
+              }
+              endpoint_ = input.readMessage(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(endpoint_);
+                endpoint_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000080;
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          operatorProfile_ = java.util.Collections.unmodifiableList(operatorProfile_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -9032,95 +9411,192 @@ public final class UserBitShared {
     }
 
     private int bitField0_;
-    // optional int32 minor_fragment_id = 1;
-    public static final int MINOR_FRAGMENT_ID_FIELD_NUMBER = 1;
+    // optional .exec.shared.FragmentState state = 1;
+    public static final int STATE_FIELD_NUMBER = 1;
+    private org.apache.drill.exec.proto.UserBitShared.FragmentState state_;
+    /**
+     * <code>optional .exec.shared.FragmentState state = 1;</code>
+     */
+    public boolean hasState() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .exec.shared.FragmentState state = 1;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.FragmentState getState() {
+      return state_;
+    }
+
+    // optional .exec.shared.DrillPBError error = 2;
+    public static final int ERROR_FIELD_NUMBER = 2;
+    private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_;
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    public boolean hasError() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
+      return error_;
+    }
+    /**
+     * <code>optional .exec.shared.DrillPBError error = 2;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
+      return error_;
+    }
+
+    // optional int32 minor_fragment_id = 3;
+    public static final int MINOR_FRAGMENT_ID_FIELD_NUMBER = 3;
     private int minorFragmentId_;
     /**
-     * <code>optional int32 minor_fragment_id = 1;</code>
+     * <code>optional int32 minor_fragment_id = 3;</code>
      */
     public boolean hasMinorFragmentId() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
-     * <code>optional int32 minor_fragment_id = 1;</code>
+     * <code>optional int32 minor_fragment_id = 3;</code>
      */
     public int getMinorFragmentId() {
       return minorFragmentId_;
     }
 
-    // repeated .exec.shared.OperatorProfile operator_profile = 2;
-    public static final int OPERATOR_PROFILE_FIELD_NUMBER = 2;
+    // repeated .exec.shared.OperatorProfile operator_profile = 4;
+    public static final int OPERATOR_PROFILE_FIELD_NUMBER = 4;
     private java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile> operatorProfile_;
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     public java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile> getOperatorProfileList() {
       return operatorProfile_;
     }
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder> 
         getOperatorProfileOrBuilderList() {
       return operatorProfile_;
     }
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     public int getOperatorProfileCount() {
       return operatorProfile_.size();
     }
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.OperatorProfile getOperatorProfile(int index) {
       return operatorProfile_.get(index);
     }
     /**
-     * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+     * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
      */
     public org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder getOperatorProfileOrBuilder(
         int index) {
       return operatorProfile_.get(index);
     }
 
-    // optional int64 start_time = 3;
-    public static final int START_TIME_FIELD_NUMBER = 3;
+    // optional int64 start_time = 5;
+    public static final int START_TIME_FIELD_NUMBER = 5;
     private long startTime_;
     /**
-     * <code>optional int64 start_time = 3;</code>
+     * <code>optional int64 start_time = 5;</code>
      */
     public boolean hasStartTime() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
-     * <code>optional int64 start_time = 3;</code>
+     * <code>optional int64 start_time = 5;</code>
      */
     public long getStartTime() {
       return startTime_;
     }
 
-    // optional int64 end_time = 4;
-    public static final int END_TIME_FIELD_NUMBER = 4;
+    // optional int64 end_time = 6;
+    public static final int END_TIME_FIELD_NUMBER = 6;
     private long endTime_;
     /**
-     * <code>optional int64 end_time = 4;</code>
+     * <code>optional int64 end_time = 6;</code>
      */
     public boolean hasEndTime() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>optional int64 end_time = 4;</code>
+     * <code>optional int64 end_time = 6;</code>
      */
     public long getEndTime() {
       return endTime_;
     }
 
+    // optional int64 memory_used = 7;
+    public static final int MEMORY_USED_FIELD_NUMBER = 7;
+    private long memoryUsed_;
+    /**
+     * <code>optional int64 memory_used = 7;</code>
+     */
+    public boolean hasMemoryUsed() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>optional int64 memory_used = 7;</code>
+     */
+    public long getMemoryUsed() {
+      return memoryUsed_;
+    }
+
+    // optional int64 max_memory_used = 8;
+    public static final int MAX_MEMORY_USED_FIELD_NUMBER = 8;
+    private long maxMemoryUsed_;
+    /**
+     * <code>optional int64 max_memory_used = 8;</code>
+     */
+    public boolean hasMaxMemoryUsed() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>optional int64 max_memory_used = 8;</code>
+     */
+    public long getMaxMemoryUsed() {
+      return maxMemoryUsed_;
+    }
+
+    // optional .exec.DrillbitEndpoint endpoint = 9;
+    public static final int ENDPOINT_FIELD_NUMBER = 9;
+    private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_;
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    public boolean hasEndpoint() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+      return endpoint_;
+    }
+    /**
+     * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+     */
+    public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+      return endpoint_;
+    }
+
     private void initFields() {
+      state_ = org.apache.drill.exec.proto.UserBitShared.FragmentState.SENDING;
+      error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
       minorFragmentId_ = 0;
       operatorProfile_ = java.util.Collections.emptyList();
       startTime_ = 0L;
       endTime_ = 0L;
+      memoryUsed_ = 0L;
+      maxMemoryUsed_ = 0L;
+      endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -9135,16 +9611,31 @@ public final class UserBitShared {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeInt32(1, minorFragmentId_);
-      }
-      for (int i = 0; i < operatorProfile_.size(); i++) {
-        output.writeMessage(2, operatorProfile_.get(i));
+        output.writeEnum(1, state_.getNumber());
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeInt64(3, startTime_);
+        output.writeMessage(2, error_);
       }
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(4, endTime_);
+        output.writeInt32(3, minorFragmentId_);
+      }
+      for (int i = 0; i < operatorProfile_.size(); i++) {
+        output.writeMessage(4, operatorProfile_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt64(5, startTime_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeInt64(6, endTime_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeInt64(7, memoryUsed_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeInt64(8, maxMemoryUsed_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(9, endpoint_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -9157,19 +9648,39 @@ public final class UserBitShared {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(1, minorFragmentId_);
+          .computeEnumSize(1, state_.getNumber());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, error_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(3, minorFragmentId_);
       }
       for (int i = 0; i < operatorProfile_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, operatorProfile_.get(i));
+          .computeMessageSize(4, operatorProfile_.get(i));
       }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(3, startTime_);
+          .computeInt64Size(5, startTime_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(6, endTime_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(7, memoryUsed_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(8, maxMemoryUsed_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(4, endTime_);
+          .computeMessageSize(9, endpoint_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -9279,7 +9790,9 @@ public final class UserBitShared {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getErrorFieldBuilder();
           getOperatorProfileFieldBuilder();
+          getEndpointFieldBuilder();
         }
       }
       private static Builder create() {
@@ -9288,18 +9801,36 @@ public final class UserBitShared {
 
       public Builder clear() {
         super.clear();
-        minorFragmentId_ = 0;
+        state_ = org.apache.drill.exec.proto.UserBitShared.FragmentState.SENDING;
         bitField0_ = (bitField0_ & ~0x00000001);
+        if (errorBuilder_ == null) {
+          error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+        } else {
+          errorBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        minorFragmentId_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000004);
         if (operatorProfileBuilder_ == null) {
           operatorProfile_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000008);
         } else {
           operatorProfileBuilder_.clear();
         }
         startTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
+        bitField0_ = (bitField0_ & ~0x00000010);
         endTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000008);
+        bitField0_ = (bitField0_ & ~0x00000020);
+        memoryUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000040);
+        maxMemoryUsed_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000080);
+        if (endpointBuilder_ == null) {
+          endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+        } else {
+          endpointBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -9331,24 +9862,52 @@ public final class UserBitShared {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
+        result.state_ = state_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (errorBuilder_ == null) {
+          result.error_ = error_;
+        } else {
+          result.error_ = errorBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
         result.minorFragmentId_ = minorFragmentId_;
         if (operatorProfileBuilder_ == null) {
-          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+          if (((bitField0_ & 0x00000008) == 0x00000008)) {
             operatorProfile_ = java.util.Collections.unmodifiableList(operatorProfile_);
-            bitField0_ = (bitField0_ & ~0x00000002);
+            bitField0_ = (bitField0_ & ~0x00000008);
           }
           result.operatorProfile_ = operatorProfile_;
         } else {
           result.operatorProfile_ = operatorProfileBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000002;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
         }
         result.startTime_ = startTime_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000004;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
         }
         result.endTime_ = endTime_;
+        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
+          to_bitField0_ |= 0x00000020;
+        }
+        result.memoryUsed_ = memoryUsed_;
+        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
+          to_bitField0_ |= 0x00000040;
+        }
+        result.maxMemoryUsed_ = maxMemoryUsed_;
+        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+          to_bitField0_ |= 0x00000080;
+        }
+        if (endpointBuilder_ == null) {
+          result.endpoint_ = endpoint_;
+        } else {
+          result.endpoint_ = endpointBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -9365,6 +9924,12 @@ public final class UserBitShared {
 
       public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile other) {
         if (other == org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance()) return this;
+        if (other.hasState()) {
+          setState(other.getState());
+        }
+        if (other.hasError()) {
+          mergeError(other.getError());
+        }
         if (other.hasMinorFragmentId()) {
           setMinorFragmentId(other.getMinorFragmentId());
         }
@@ -9372,7 +9937,7 @@ public final class UserBitShared {
           if (!other.operatorProfile_.isEmpty()) {
             if (operatorProfile_.isEmpty()) {
               operatorProfile_ = other.operatorProfile_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000008);
             } else {
               ensureOperatorProfileIsMutable();
               operatorProfile_.addAll(other.operatorProfile_);
@@ -9385,7 +9950,7 @@ public final class UserBitShared {
               operatorProfileBuilder_.dispose();
               operatorProfileBuilder_ = null;
               operatorProfile_ = other.operatorProfile_;
-              bitField0_ = (bitField0_ & ~0x00000002);
+              bitField0_ = (bitField0_ & ~0x00000008);
               operatorProfileBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getOperatorProfileFieldBuilder() : null;
@@ -9400,6 +9965,15 @@ public final class UserBitShared {
         if (other.hasEndTime()) {
           setEndTime(other.getEndTime());
         }
+        if (other.hasMemoryUsed()) {
+          setMemoryUsed(other.getMemoryUsed());
+        }
+        if (other.hasMaxMemoryUsed()) {
+          setMaxMemoryUsed(other.getMaxMemoryUsed());
+        }
+        if (other.hasEndpoint()) {
+          mergeEndpoint(other.getEndpoint());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -9427,46 +10001,199 @@ public final class UserBitShared {
       }
       private int bitField0_;
 
-      // optional int32 minor_fragment_id = 1;
+      // optional .exec.shared.FragmentState state = 1;
+      private org.apache.drill.exec.proto.UserBitShared.FragmentState state_ = org.apache.drill.exec.proto.UserBitShared.FragmentState.SENDING;
+      /**
+       * <code>optional .exec.shared.FragmentState state = 1;</code>
+       */
+      public boolean hasState() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .exec.shared.FragmentState state = 1;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.FragmentState getState() {
+        return state_;
+      }
+      /**
+       * <code>optional .exec.shared.FragmentState state = 1;</code>
+       */
+      public Builder setState(org.apache.drill.exec.proto.UserBitShared.FragmentState value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000001;
+        state_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.FragmentState state = 1;</code>
+       */
+      public Builder clearState() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        state_ = org.apache.drill.exec.proto.UserBitShared.FragmentState.SENDING;
+        onChanged();
+        return this;
+      }
+
+      // optional .exec.shared.DrillPBError error = 2;
+      private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public boolean hasError() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
+        if (errorBuilder_ == null) {
+          return error_;
+        } else {
+          return errorBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public Builder setError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+        if (errorBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          error_ = value;
+          onChanged();
+        } else {
+          errorBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public Builder setError(
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
+        if (errorBuilder_ == null) {
+          error_ = builderForValue.build();
+          onChanged();
+        } else {
+          errorBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public Builder mergeError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
+        if (errorBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
+              error_ != org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance()) {
+            error_ =
+              org.apache.drill.exec.proto.UserBitShared.DrillPBError.newBuilder(error_).mergeFrom(value).buildPartial();
+          } else {
+            error_ = value;
+          }
+          onChanged();
+        } else {
+          errorBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000002;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public Builder clearError() {
+        if (errorBuilder_ == null) {
+          error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
+          onChanged();
+        } else {
+          errorBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder() {
+        bitField0_ |= 0x00000002;
+        onChanged();
+        return getErrorFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
+        if (errorBuilder_ != null) {
+          return errorBuilder_.getMessageOrBuilder();
+        } else {
+          return error_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.DrillPBError error = 2;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
+          getErrorFieldBuilder() {
+        if (errorBuilder_ == null) {
+          errorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
+                  error_,
+                  getParentForChildren(),
+                  isClean());
+          error_ = null;
+        }
+        return errorBuilder_;
+      }
+
+      // optional int32 minor_fragment_id = 3;
       private int minorFragmentId_ ;
       /**
-       * <code>optional int32 minor_fragment_id = 1;</code>
+       * <code>optional int32 minor_fragment_id = 3;</code>
        */
       public boolean hasMinorFragmentId() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
+        return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
-       * <code>optional int32 minor_fragment_id = 1;</code>
+       * <code>optional int32 minor_fragment_id = 3;</code>
        */
       public int getMinorFragmentId() {
         return minorFragmentId_;
       }
       /**
-       * <code>optional int32 minor_fragment_id = 1;</code>
+       * <code>optional int32 minor_fragment_id = 3;</code>
        */
       public Builder setMinorFragmentId(int value) {
-        bitField0_ |= 0x00000001;
+        bitField0_ |= 0x00000004;
         minorFragmentId_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 minor_fragment_id = 1;</code>
+       * <code>optional int32 minor_fragment_id = 3;</code>
        */
       public Builder clearMinorFragmentId() {
-        bitField0_ = (bitField0_ & ~0x00000001);
+        bitField0_ = (bitField0_ & ~0x00000004);
         minorFragmentId_ = 0;
         onChanged();
         return this;
       }
 
-      // repeated .exec.shared.OperatorProfile operator_profile = 2;
+      // repeated .exec.shared.OperatorProfile operator_profile = 4;
       private java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile> operatorProfile_ =
         java.util.Collections.emptyList();
       private void ensureOperatorProfileIsMutable() {
-        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+        if (!((bitField0_ & 0x00000008) == 0x00000008)) {
           operatorProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.OperatorProfile>(operatorProfile_);
-          bitField0_ |= 0x00000002;
+          bitField0_ |= 0x00000008;
          }
       }
 
@@ -9474,7 +10201,7 @@ public final class UserBitShared {
           org.apache.drill.exec.proto.UserBitShared.OperatorProfile, org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder, org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder> operatorProfileBuilder_;
 
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile> getOperatorProfileList() {
         if (operatorProfileBuilder_ == null) {
@@ -9484,7 +10211,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public int getOperatorProfileCount() {
         if (operatorProfileBuilder_ == null) {
@@ -9494,7 +10221,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.OperatorProfile getOperatorProfile(int index) {
         if (operatorProfileBuilder_ == null) {
@@ -9504,7 +10231,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder setOperatorProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.OperatorProfile value) {
@@ -9521,7 +10248,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder setOperatorProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder builderForValue) {
@@ -9535,7 +10262,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder addOperatorProfile(org.apache.drill.exec.proto.UserBitShared.OperatorProfile value) {
         if (operatorProfileBuilder_ == null) {
@@ -9551,7 +10278,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder addOperatorProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.OperatorProfile value) {
@@ -9568,7 +10295,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder addOperatorProfile(
           org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder builderForValue) {
@@ -9582,7 +10309,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder addOperatorProfile(
           int index, org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder builderForValue) {
@@ -9596,7 +10323,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder addAllOperatorProfile(
           java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.OperatorProfile> values) {
@@ -9610,12 +10337,12 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder clearOperatorProfile() {
         if (operatorProfileBuilder_ == null) {
           operatorProfile_ = java.util.Collections.emptyList();
-          bitField0_ = (bitField0_ & ~0x00000002);
+          bitField0_ = (bitField0_ & ~0x00000008);
           onChanged();
         } else {
           operatorProfileBuilder_.clear();
@@ -9623,7 +10350,7 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public Builder removeOperatorProfile(int index) {
         if (operatorProfileBuilder_ == null) {
@@ -9636,14 +10363,14 @@ public final class UserBitShared {
         return this;
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder getOperatorProfileBuilder(
           int index) {
         return getOperatorProfileFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder getOperatorProfileOrBuilder(
           int index) {
@@ -9653,7 +10380,7 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder> 
            getOperatorProfileOrBuilderList() {
@@ -9664,14 +10391,14 @@ public final class UserBitShared {
         }
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder addOperatorProfileBuilder() {
         return getOperatorProfileFieldBuilder().addBuilder(
             org.apache.drill.exec.proto.UserBitShared.OperatorProfile.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder addOperatorProfileBuilder(
           int index) {
@@ -9679,7 +10406,7 @@ public final class UserBitShared {
             index, org.apache.drill.exec.proto.UserBitShared.OperatorProfile.getDefaultInstance());
       }
       /**
-       * <code>repeated .exec.shared.OperatorProfile operator_profile = 2;</code>
+       * <code>repeated .exec.shared.OperatorProfile operator_profile = 4;</code>
        */
       public java.util.List<org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder> 
            getOperatorProfileBuilderList() {
@@ -9692,7 +10419,7 @@ public final class UserBitShared {
           operatorProfileBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
               org.apache.drill.exec.proto.UserBitShared.OperatorProfile, org.apache.drill.exec.proto.UserBitShared.OperatorProfile.Builder, org.apache.drill.exec.proto.UserBitShared.OperatorProfileOrBuilder>(
                   operatorProfile_,
-                  ((bitField0_ & 0x00000002) == 0x00000002),
+                  ((bitField0_ & 0x00000008) == 0x00000008),
                   getParentForChildren(),
                   isClean());
           operatorProfile_ = null;
@@ -9700,70 +10427,253 @@ public final class UserBitShared {
         return operatorProfileBuilder_;
       }
 
-      // optional int64 start_time = 3;
+      // optional int64 start_time = 5;
       private long startTime_ ;
       /**
-       * <code>optional int64 start_time = 3;</code>
+       * <code>optional int64 start_time = 5;</code>
        */
       public boolean hasStartTime() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
+        return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
-       * <code>optional int64 start_time = 3;</code>
+       * <code>optional int64 start_time = 5;</code>
        */
       public long getStartTime() {
         return startTime_;
       }
       /**
-       * <code>optional int64 start_time = 3;</code>
+       * <code>optional int64 start_time = 5;</code>
        */
       public Builder setStartTime(long value) {
-        bitField0_ |= 0x00000004;
+        bitField0_ |= 0x00000010;
         startTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int64 start_time = 3;</code>
+       * <code>optional int64 start_time = 5;</code>
+       */
+      public Builder clearStartTime() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        startTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 end_time = 6;
+      private long endTime_ ;
+      /**
+       * <code>optional int64 end_time = 6;</code>
+       */
+      public boolean hasEndTime() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional int64 end_time = 6;</code>
+       */
+      public long getEndTime() {
+        return endTime_;
+      }
+      /**
+       * <code>optional int64 end_time = 6;</code>
+       */
+      public Builder setEndTime(long value) {
+        bitField0_ |= 0x00000020;
+        endTime_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 end_time = 6;</code>
+       */
+      public Builder clearEndTime() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        endTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 memory_used = 7;
+      private long memoryUsed_ ;
+      /**
+       * <code>optional int64 memory_used = 7;</code>
+       */
+      public boolean hasMemoryUsed() {
+        return ((bitField0_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional int64 memory_used = 7;</code>
+       */
+      public long getMemoryUsed() {
+        return memoryUsed_;
+      }
+      /**
+       * <code>optional int64 memory_used = 7;</code>
+       */
+      public Builder setMemoryUsed(long value) {
+        bitField0_ |= 0x00000040;
+        memoryUsed_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 memory_used = 7;</code>
+       */
+      public Builder clearMemoryUsed() {
+        bitField0_ = (bitField0_ & ~0x00000040);
+        memoryUsed_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 max_memory_used = 8;
+      private long maxMemoryUsed_ ;
+      /**
+       * <code>optional int64 max_memory_used = 8;</code>
+       */
+      public boolean hasMaxMemoryUsed() {
+        return ((bitField0_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional int64 max_memory_used = 8;</code>
+       */
+      public long getMaxMemoryUsed() {
+        return maxMemoryUsed_;
+      }
+      /**
+       * <code>optional int64 max_memory_used = 8;</code>
+       */
+      public Builder setMaxMemoryUsed(long value) {
+        bitField0_ |= 0x00000080;
+        maxMemoryUsed_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 max_memory_used = 8;</code>
+       */
+      public Builder clearMaxMemoryUsed() {
+        bitField0_ = (bitField0_ & ~0x00000080);
+        maxMemoryUsed_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional .exec.DrillbitEndpoint endpoint = 9;
+      private org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> endpointBuilder_;
+      /**
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+       */
+      public boolean hasEndpoint() {
+        return ((bitField0_ & 0x00000100) == 0x00000100);
+      }
+      /**
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+       */
+      public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint getEndpoint() {
+        if (endpointBuilder_ == null) {
+          return endpoint_;
+        } else {
+          return endpointBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+       */
+      public Builder setEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+        if (endpointBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          endpoint_ = value;
+          onChanged();
+        } else {
+          endpointBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
+       */
+      public Builder setEndpoint(
+          org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder builderForValue) {
+        if (endpointBuilder_ == null) {
+          endpoint_ = builderForValue.build();
+          onChanged();
+        } else {
+          endpointBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000100;
+        return this;
+      }
+      /**
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
        */
-      public Builder clearStartTime() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        startTime_ = 0L;
-        onChanged();
+      public Builder mergeEndpoint(org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint value) {
+        if (endpointBuilder_ == null) {
+          if (((bitField0_ & 0x00000100) == 0x00000100) &&
+              endpoint_ != org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance()) {
+            endpoint_ =
+              org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.newBuilder(endpoint_).mergeFrom(value).buildPartial();
+          } else {
+            endpoint_ = value;
+          }
+          onChanged();
+        } else {
+          endpointBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000100;
         return this;
       }
-
-      // optional int64 end_time = 4;
-      private long endTime_ ;
       /**
-       * <code>optional int64 end_time = 4;</code>
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
        */
-      public boolean hasEndTime() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
+      public Builder clearEndpoint() {
+        if (endpointBuilder_ == null) {
+          endpoint_ = org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.getDefaultInstance();
+          onChanged();
+        } else {
+          endpointBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000100);
+        return this;
       }
       /**
-       * <code>optional int64 end_time = 4;</code>
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
        */
-      public long getEndTime() {
-        return endTime_;
+      public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder getEndpointBuilder() {
+        bitField0_ |= 0x00000100;
+        onChanged();
+        return getEndpointFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional int64 end_time = 4;</code>
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
        */
-      public Builder setEndTime(long value) {
-        bitField0_ |= 0x00000008;
-        endTime_ = value;
-        onChanged();
-        return this;
+      public org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder getEndpointOrBuilder() {
+        if (endpointBuilder_ != null) {
+          return endpointBuilder_.getMessageOrBuilder();
+        } else {
+          return endpoint_;
+        }
       }
       /**
-       * <code>optional int64 end_time = 4;</code>
+       * <code>optional .exec.DrillbitEndpoint endpoint = 9;</code>
        */
-      public Builder clearEndTime() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        endTime_ = 0L;
-        onChanged();
-        return this;
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder> 
+          getEndpointFieldBuilder() {
+        if (endpointBuilder_ == null) {
+          endpointBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint.Builder, org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpointOrBuilder>(
+                  endpoint_,
+                  getParentForChildren(),
+                  isClean());
+          endpoint_ = null;
+        }
+        return endpointBuilder_;
       }
 
       // @@protoc_insertion_point(builder_scope:exec.shared.MinorFragmentProfile)
@@ -9805,20 +10715,6 @@ public final class UserBitShared {
     org.apache.drill.exec.proto.UserBitShared.StreamProfileOrBuilder getInputProfileOrBuilder(
         int index);
 
-    // optional .exec.shared.StreamProfile output_profile = 2;
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    boolean hasOutputProfile();
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.StreamProfile getOutputProfile();
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.StreamProfileOrBuilder getOutputProfileOrBuilder();
-
     // optional int32 operator_id = 3;
     /**
      * <code>optional int32 operator_id = 3;</code>
@@ -9953,48 +10849,35 @@ public final class UserBitShared {
               inputProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.StreamProfile.PARSER, extensionRegistry));
               break;
             }
-            case 18: {
-              org.apache.drill.exec.proto.UserBitShared.StreamProfile.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000001) == 0x00000001)) {
-                subBuilder = outputProfile_.toBuilder();
-              }
-              outputProfile_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.StreamProfile.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(outputProfile_);
-                outputProfile_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000001;
-              break;
-            }
             case 24: {
-              bitField0_ |= 0x00000002;
+              bitField0_ |= 0x00000001;
               operatorId_ = input.readInt32();
               break;
             }
             case 32: {
-              bitField0_ |= 0x00000004;
+              bitField0_ |= 0x00000002;
               operatorType_ = input.readInt32();
               break;
             }
             case 40: {
-              bitField0_ |= 0x00000008;
+              bitField0_ |= 0x00000004;
               setupNanos_ = input.readInt64();
               break;
             }
             case 48: {
-              bitField0_ |= 0x00000010;
+              bitField0_ |= 0x00000008;
               processNanos_ = input.readInt64();
               break;
             }
             case 56: {
-              bitField0_ |= 0x00000020;
+              bitField0_ |= 0x00000010;
               localMemoryAllocated_ = input.readInt64();
               break;
             }
             case 66: {
-              if (!((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
                 metric_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MetricValue>();
-                mutable_bitField0_ |= 0x00000080;
+                mutable_bitField0_ |= 0x00000040;
               }
               metric_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.MetricValue.PARSER, extensionRegistry));
               break;
@@ -10010,7 +10893,7 @@ public final class UserBitShared {
         if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
           inputProfile_ = java.util.Collections.unmodifiableList(inputProfile_);
         }
-        if (((mutable_bitField0_ & 0x00000080) == 0x00000080)) {
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
           metric_ = java.util.Collections.unmodifiableList(metric_);
         }
         this.unknownFields = unknownFields.build();
@@ -10081,28 +10964,6 @@ public final class UserBitShared {
       return inputProfile_.get(index);
     }
 
-    // optional .exec.shared.StreamProfile output_profile = 2;
-    public static final int OUTPUT_PROFILE_FIELD_NUMBER = 2;
-    private org.apache.drill.exec.proto.UserBitShared.StreamProfile outputProfile_;
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    public boolean hasOutputProfile() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.StreamProfile getOutputProfile() {
-      return outputProfile_;
-    }
-    /**
-     * <code>optional .exec.shared.StreamProfile output_profile = 2;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.StreamProfileOrBuilder getOutputProfileOrBuilder() {
-      return outputProfile_;
-    }
-
     // optional int32 operator_id = 3;
     public static final int OPERATOR_ID_FIELD_NUMBER = 3;
     private int operatorId_;
@@ -10110,7 +10971,7 @@ public final class UserBitShared {
      * <code>optional int32 operator_id = 3;</code>
      */
     public boolean hasOperatorId() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
+      return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
      * <code>optional int32 operator_id = 3;</code>
@@ -10126,7 +10987,7 @@ public final class UserBitShared {
      * <code>optional int32 operator_type = 4;</code>
      */
     public boolean hasOperatorType() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
+      return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
      * <code>optional int32 operator_type = 4;</code>
@@ -10142,7 +11003,7 @@ public final class UserBitShared {
      * <code>optional int64 setup_nanos = 5;</code>
      */
     public boolean hasSetupNanos() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
+      return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
      * <code>optional int64 setup_nanos = 5;</code>
@@ -10158,7 +11019,7 @@ public final class UserBitShared {
      * <code>optional int64 process_nanos = 6;</code>
      */
     public boolean hasProcessNanos() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
+      return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
      * <code>optional int64 process_nanos = 6;</code>
@@ -10174,7 +11035,7 @@ public final class UserBitShared {
      * <code>optional int64 local_memory_allocated = 7;</code>
      */
     public boolean hasLocalMemoryAllocated() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+      return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
      * <code>optional int64 local_memory_allocated = 7;</code>
@@ -10221,7 +11082,6 @@ public final class UserBitShared {
 
     private void initFields() {
       inputProfile_ = java.util.Collections.emptyList();
-      outputProfile_ = org.apache.drill.exec.proto.UserBitShared.StreamProfile.getDefaultInstance();
       operatorId_ = 0;
       operatorType_ = 0;
       setupNanos_ = 0L;
@@ -10245,21 +11105,18 @@ public final class UserBitShared {
         output.writeMessage(1, inputProfile_.get(i));
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeMessage(2, outputProfile_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt32(3, operatorId_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt32(4, operatorType_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt64(5, setupNanos_);
       }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeInt64(6, processNanos_);
       }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
         output.writeInt64(7, localMemoryAllocated_);
       }
       for (int i = 0; i < metric_.size(); i++) {
@@ -10280,25 +11137,21 @@ public final class UserBitShared {
       }
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(2, outputProfile_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(3, operatorId_);
       }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
           .computeInt

<TRUNCATED>

[07/24] status changes

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
index 69ba832..826ebf5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
@@ -17,22 +17,21 @@
  */
 package org.apache.drill.exec.physical.impl;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.junit.Ignore;
 import org.junit.Test;
 
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
 
 public class TestBroadcastExchange extends PopUnitTestBase {
   @Test
@@ -51,7 +50,7 @@ public class TestBroadcastExchange extends PopUnitTestBase {
               FileUtils.getResourceAsFile("/sender/broadcast_exchange.json"), Charsets.UTF_8)
               .replace("#{LEFT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.left.json").toURI().toString())
               .replace("#{RIGHT_FILE}", FileUtils.getResourceAsFile("/join/merge_single_batch.right.json").toURI().toString());
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, physicalPlan);
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
       int count = 0;
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0) count += b.getHeader().getRowCount();
@@ -75,7 +74,7 @@ public class TestBroadcastExchange extends PopUnitTestBase {
 
       String physicalPlan = Files.toString(
           FileUtils.getResourceAsFile("/sender/broadcast_exchange_long_run.json"), Charsets.UTF_8);
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, physicalPlan);
+      List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, physicalPlan);
       int count = 0;
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0) count += b.getHeader().getRowCount();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java
deleted file mode 100644
index 24cae70..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java
+++ /dev/null
@@ -1,225 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl;
-
-import java.lang.reflect.Field;
-import java.nio.ByteBuffer;
-import java.util.Random;
-
-import sun.misc.Unsafe;
-import com.google.caliper.Benchmark;
-import com.google.caliper.Param;
-
-@SuppressWarnings("restriction")
-public class TestExecutionAbstractions extends Benchmark {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExecutionAbstractions.class);
-
-  /**
-   * General goal is compare the performance of abstract versus concrete
-   * implementations of selection vector dereferencing.
-   */
-
-  private static enum Implementation {
-    CONCRETE, ABSTRACT
-  };
-
-  private static enum SelectionVectorMode {
-    NONE, SV2, SV4
-  };
-
-  @Param
-  private Implementation impl;
-  @Param
-  private SelectionVectorMode mode;
-
-  private int scale = 1024*1024*8;
-
-  private final Unsafe unsafe = retrieveUnsafe();
-  private final ByteBuffer a;
-  private final ByteBuffer b;
-  private final ByteBuffer b2;
-  private final ByteBuffer c;
-  private final ByteBuffer sv2;
-  private final ByteBuffer sv4;
-  private final int max;
-  
-
-  public TestExecutionAbstractions() throws Exception {
-    sv2 = ByteBuffer.allocateDirect(scale * 2);
-    sv4 = ByteBuffer.allocateDirect(scale * 4);
-    a = ByteBuffer.allocateDirect(scale * 8);
-    b = ByteBuffer.allocateDirect(scale * 8);
-    b2 = ByteBuffer.allocateDirect(scale * 8);
-    c = ByteBuffer.allocateDirect(scale * 8);
-    int svPos = 0;
-    int i = 0;
-    try {
-
-      Random r = new Random();
-      for (; i < scale; i++) {
-        a.putLong(i * 8, r.nextLong());
-        b.putLong(i * 8, r.nextLong());
-
-        if (r.nextBoolean()) {
-          sv2.putChar(svPos * 2, (char) i);
-          sv4.putInt(svPos * 4, i);
-          svPos++;
-        }
-      }
-      System.out.println("Created test data.");
-      max = mode == SelectionVectorMode.NONE ? 1024 : svPos;
-
-    } catch (Exception ex) {
-      System.out.println("i: " + i + ", svPos" + svPos);
-      throw ex;
-    }
-  }
-
-   private Unsafe retrieveUnsafe(){
-     sun.misc.Unsafe localUnsafe = null;
-  
-   try {
-   Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
-   field.setAccessible(true);
-   localUnsafe = (sun.misc.Unsafe) field.get(null);
-   } catch (Exception e) {
-   throw new AssertionError(e);
-   }
-  
-   return localUnsafe;
-   }
-
-  public void timeAdd(int reps) {
-    for (int r = 0; r < reps; r++) {
-      switch (impl) {
-
-      case CONCRETE:
-        switch (mode) {
-
-        case NONE:
-          for (int i = 0; i < max; i++) {
-            
-            c.putLong(i * 8, a.getLong(i * 8) + b.getLong(i * 8));
-          }
-
-          break;
-        case SV2:
-          for (int i = 0; i < max; i++) {
-            int index = sv2.getChar(i*2) * 8;
-            c.putLong(i * 8, a.getLong(index) + b.getLong(index));
-          }
-          break;
-        case SV4:
-          for (int i = 0; i < max; i++) {
-            int index = sv4.getInt(i*4) * 8;
-            c.putLong(i * 8, a.getLong(index) + b.getLong(index));
-          }
-          break;
-        }
-        break;
-      case ABSTRACT:
-        LongGetter aGetter = null;
-        LongGetter bGetter = null;
-
-        switch (mode) {
-
-        case NONE:
-          aGetter = new StraightGetter(a);
-          bGetter = new StraightGetter(b);
-          break;
-        case SV2:
-          aGetter = new Sv2Getter(sv2, a);
-          bGetter = new Sv2Getter(sv2, b);
-          break;
-        case SV4:
-          aGetter = new Sv4Getter(sv4, a);
-          bGetter = new Sv4Getter(sv4, b);
-          break;
-
-        }
-
-        for (int i = 0; i < max; i++) {
-          c.putLong(i * 8, aGetter.getLong(i) + bGetter.getLong(i));
-        }
-        break;
-      }
-    }
-
-  }
-
-  private static interface LongGetter {
-    long getLong(int index);
-  }
-
-  private static class StraightGetter implements LongGetter {
-
-    final ByteBuffer b;
-
-    public StraightGetter(ByteBuffer b) {
-      super();
-      this.b = b;
-    }
-
-    @Override
-    public long getLong(int index) {
-      return b.getLong(index * 8);
-    }
-  }
-
-  private static class Sv2Getter implements LongGetter {
-    final ByteBuffer b;
-    final ByteBuffer sv;
-
-    public Sv2Getter(ByteBuffer sv, ByteBuffer b) {
-      super();
-      this.b = b;
-      this.sv = sv;
-    }
-
-    @Override
-    public long getLong(int index) {
-      int pos = sv.getChar(index * 2);
-      return b.getLong(pos * 8);
-    }
-  }
-
-  private static class Sv4Getter implements LongGetter {
-    final ByteBuffer b;
-    final ByteBuffer sv;
-
-    public Sv4Getter(ByteBuffer sv, ByteBuffer b) {
-      super();
-      this.b = b;
-      this.sv = sv;
-    }
-
-    @Override
-    public long getLong(int index) {
-      int pos = sv.getInt(index * 4);
-      return b.getLong(pos * 8);
-    }
-  }
-  
-  private long allocate(long bytes){
-    return unsafe.allocateMemory(bytes);
-    
-  }
-  
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
new file mode 100644
index 0000000..216cb3b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
@@ -0,0 +1,21 @@
+package org.apache.drill.exec.server;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.glassfish.hk2.api.Factory;
+
+public class DrillClientFactory implements Factory<DrillClient>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClientFactory.class);
+
+
+
+  @Override
+  public void dispose(DrillClient arg0) {
+  }
+
+  @Override
+  public DrillClient provide() {
+    return new DrillClient();
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
new file mode 100644
index 0000000..cdb04ec
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
@@ -0,0 +1,22 @@
+package org.apache.drill.exec.server;
+
+import javax.inject.Inject;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+
+import org.apache.drill.exec.client.DrillClient;
+
+@Path("hello")
+public class HelloResource {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HelloResource.class);
+
+  @Inject DrillClient client;
+
+  @GET
+  @Produces("text/plain")
+  public String getHello() {
+    return "hello world" + client;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
new file mode 100644
index 0000000..6e873ed
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
@@ -0,0 +1,48 @@
+package org.apache.drill.exec.server;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.glassfish.hk2.utilities.binding.AbstractBinder;
+import org.glassfish.jersey.jackson.JacksonFeature;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.server.ServerProperties;
+import org.glassfish.jersey.servlet.ServletContainer;
+import org.junit.Test;
+
+public class TestJersey {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJersey.class);
+
+  @Test
+  public void x() throws Exception {
+    Server server = new Server(8112);
+    ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS);
+    context.setContextPath("/");
+    server.setHandler(context);
+    ServletHolder h = new ServletHolder(new ServletContainer(new MyApplication()));
+    h.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.drill.exec.server");
+    h.setInitOrder(1);
+    context.addServlet(h, "/*");
+    server.start();
+    server.join();
+
+  }
+
+  public class MyApplication extends ResourceConfig {
+
+    public MyApplication() {
+        registerClasses(HelloResource.class);
+        register(JacksonFeature.class);
+        property(ServerProperties.METAINF_SERVICES_LOOKUP_DISABLE, true);
+        register(new AbstractBinder() {
+            @Override
+            protected void configure() {
+              this.bind(new DrillClient()).to(DrillClient.class);
+            }
+        });
+    }
+}
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
new file mode 100644
index 0000000..42fc39e
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
@@ -0,0 +1,9 @@
+package org.apache.drill.exec.server.rest;
+
+import javax.ws.rs.Path;
+
+@Path("/")
+public class RootResource {
+  public int hi = 5;
+  public String blue = "yo";
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/logback.xml b/exec/java-exec/src/test/resources/logback.xml
index b8e0ca2..b13bc58 100644
--- a/exec/java-exec/src/test/resources/logback.xml
+++ b/exec/java-exec/src/test/resources/logback.xml
@@ -42,7 +42,7 @@
 <!--   </logger> -->
 
 	<root>
-		<level value="error" />
+		<level value="debug" />
 		<appender-ref ref="STDOUT" />
 	</root>
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
index be56b96..594e3ff 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
@@ -31,7 +31,7 @@ import net.hydromatic.avatica.AvaticaStatement;
 
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
@@ -48,8 +48,8 @@ public class DrillResultSet extends AvaticaResultSet {
   private volatile QueryId queryId;
   private final DrillClient client;
   final RecordBatchLoader currentBatch;
-  final DrillCursor cursor; 
-  
+  final DrillCursor cursor;
+
   public DrillResultSet(AvaticaStatement statement, AvaticaPrepareResult prepareResult,
       ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
     super(statement, prepareResult, resultSetMetaData, timeZone);
@@ -60,7 +60,7 @@ public class DrillResultSet extends AvaticaResultSet {
     this.client = client;
     cursor = new DrillCursor(this);
   }
-  
+
   @Override
   protected void cancel() {
     cleanup();
@@ -73,7 +73,7 @@ public class DrillResultSet extends AvaticaResultSet {
     }
     listener.close();
   }
-  
+
   @Override protected DrillResultSet execute() throws SQLException{
     // Call driver's callback. It is permitted to throw a RuntimeException.
     DrillConnectionImpl connection = (DrillConnectionImpl) statement.getConnection();
@@ -89,10 +89,10 @@ public class DrillResultSet extends AvaticaResultSet {
       cursor.next();
     } catch (InterruptedException e) {
     }
-    
+
     return this;
   }
-  
+
   class Listener implements UserResultsListener {
     private static final int MAX = 100;
     private volatile RpcException ex;
@@ -102,9 +102,9 @@ public class DrillResultSet extends AvaticaResultSet {
     private volatile boolean closed = false;
     private CountDownLatch latch = new CountDownLatch(1);
     private AtomicBoolean receivedMessage = new AtomicBoolean(false);
-    
-    
-    
+
+
+
     final LinkedBlockingDeque<QueryResultBatch> queue = Queues.newLinkedBlockingDeque();
 
     private boolean releaseIfFirst(){
@@ -112,10 +112,10 @@ public class DrillResultSet extends AvaticaResultSet {
         latch.countDown();
         return true;
       }
-      
+
       return false;
     }
-    
+
     @Override
     public void submissionFailed(RpcException ex) {
       releaseIfFirst();
@@ -151,7 +151,7 @@ public class DrillResultSet extends AvaticaResultSet {
       if (result.getHeader().getErrorCount() > 0) {
         submissionFailed(new RpcException(String.format("%s", result.getHeader().getErrorList())));
       }
-      
+
       releaseIfFirst();
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java b/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
index 9480ba4..a2e97d1 100644
--- a/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
+++ b/protocol/src/main/java/org/apache/drill/common/types/TypeProtos.java
@@ -47,21 +47,13 @@ public final class TypeProtos {
      */
     MAP(1, 1),
     /**
-     * <code>REPEATMAP = 2;</code>
-     *
-     * <pre>
-     *  a repeated map column (means that multiple children sit below this)
-     * </pre>
-     */
-    REPEATMAP(2, 2),
-    /**
      * <code>TINYINT = 3;</code>
      *
      * <pre>
      *  single byte signed integer
      * </pre>
      */
-    TINYINT(3, 3),
+    TINYINT(2, 3),
     /**
      * <code>SMALLINT = 4;</code>
      *
@@ -69,7 +61,7 @@ public final class TypeProtos {
      *  two byte signed integer
      * </pre>
      */
-    SMALLINT(4, 4),
+    SMALLINT(3, 4),
     /**
      * <code>INT = 5;</code>
      *
@@ -77,7 +69,7 @@ public final class TypeProtos {
      *  four byte signed integer
      * </pre>
      */
-    INT(5, 5),
+    INT(4, 5),
     /**
      * <code>BIGINT = 6;</code>
      *
@@ -85,7 +77,7 @@ public final class TypeProtos {
      *  eight byte signed integer
      * </pre>
      */
-    BIGINT(6, 6),
+    BIGINT(5, 6),
     /**
      * <code>DECIMAL9 = 7;</code>
      *
@@ -93,7 +85,7 @@ public final class TypeProtos {
      *  a decimal supporting precision between 1 and 9
      * </pre>
      */
-    DECIMAL9(7, 7),
+    DECIMAL9(6, 7),
     /**
      * <code>DECIMAL18 = 8;</code>
      *
@@ -101,7 +93,7 @@ public final class TypeProtos {
      *  a decimal supporting precision between 10 and 18
      * </pre>
      */
-    DECIMAL18(8, 8),
+    DECIMAL18(7, 8),
     /**
      * <code>DECIMAL28SPARSE = 9;</code>
      *
@@ -109,7 +101,7 @@ public final class TypeProtos {
      *  a decimal supporting precision between 19 and 28
      * </pre>
      */
-    DECIMAL28SPARSE(9, 9),
+    DECIMAL28SPARSE(8, 9),
     /**
      * <code>DECIMAL38SPARSE = 10;</code>
      *
@@ -117,7 +109,7 @@ public final class TypeProtos {
      *  a decimal supporting precision between 29 and 38
      * </pre>
      */
-    DECIMAL38SPARSE(10, 10),
+    DECIMAL38SPARSE(9, 10),
     /**
      * <code>MONEY = 11;</code>
      *
@@ -125,7 +117,7 @@ public final class TypeProtos {
      *  signed decimal with two digit precision
      * </pre>
      */
-    MONEY(11, 11),
+    MONEY(10, 11),
     /**
      * <code>DATE = 12;</code>
      *
@@ -133,7 +125,7 @@ public final class TypeProtos {
      *  days since 4713bc
      * </pre>
      */
-    DATE(12, 12),
+    DATE(11, 12),
     /**
      * <code>TIME = 13;</code>
      *
@@ -141,7 +133,7 @@ public final class TypeProtos {
      *  time in micros before or after 2000/1/1
      * </pre>
      */
-    TIME(13, 13),
+    TIME(12, 13),
     /**
      * <code>TIMETZ = 14;</code>
      *
@@ -149,7 +141,7 @@ public final class TypeProtos {
      *  time in micros before or after 2000/1/1 with timezone
      * </pre>
      */
-    TIMETZ(14, 14),
+    TIMETZ(13, 14),
     /**
      * <code>TIMESTAMPTZ = 15;</code>
      *
@@ -157,7 +149,7 @@ public final class TypeProtos {
      *  unix epoch time in millis
      * </pre>
      */
-    TIMESTAMPTZ(15, 15),
+    TIMESTAMPTZ(14, 15),
     /**
      * <code>TIMESTAMP = 16;</code>
      *
@@ -165,7 +157,7 @@ public final class TypeProtos {
      *  TBD
      * </pre>
      */
-    TIMESTAMP(16, 16),
+    TIMESTAMP(15, 16),
     /**
      * <code>INTERVAL = 17;</code>
      *
@@ -173,7 +165,7 @@ public final class TypeProtos {
      *  TBD
      * </pre>
      */
-    INTERVAL(17, 17),
+    INTERVAL(16, 17),
     /**
      * <code>FLOAT4 = 18;</code>
      *
@@ -181,7 +173,7 @@ public final class TypeProtos {
      *  4 byte ieee 754 
      * </pre>
      */
-    FLOAT4(18, 18),
+    FLOAT4(17, 18),
     /**
      * <code>FLOAT8 = 19;</code>
      *
@@ -189,7 +181,7 @@ public final class TypeProtos {
      *  8 byte ieee 754
      * </pre>
      */
-    FLOAT8(19, 19),
+    FLOAT8(18, 19),
     /**
      * <code>BIT = 20;</code>
      *
@@ -197,7 +189,7 @@ public final class TypeProtos {
      *  single bit value (boolean)
      * </pre>
      */
-    BIT(20, 20),
+    BIT(19, 20),
     /**
      * <code>FIXEDCHAR = 21;</code>
      *
@@ -205,11 +197,11 @@ public final class TypeProtos {
      *  utf8 fixed length string, padded with spaces
      * </pre>
      */
-    FIXEDCHAR(21, 21),
+    FIXEDCHAR(20, 21),
     /**
      * <code>FIXED16CHAR = 22;</code>
      */
-    FIXED16CHAR(22, 22),
+    FIXED16CHAR(21, 22),
     /**
      * <code>FIXEDBINARY = 23;</code>
      *
@@ -217,7 +209,7 @@ public final class TypeProtos {
      *  fixed length binary, padded with 0 bytes
      * </pre>
      */
-    FIXEDBINARY(23, 23),
+    FIXEDBINARY(22, 23),
     /**
      * <code>VARCHAR = 24;</code>
      *
@@ -225,7 +217,7 @@ public final class TypeProtos {
      *  utf8 variable length string
      * </pre>
      */
-    VARCHAR(24, 24),
+    VARCHAR(23, 24),
     /**
      * <code>VAR16CHAR = 25;</code>
      *
@@ -233,7 +225,7 @@ public final class TypeProtos {
      * utf16 variable length string
      * </pre>
      */
-    VAR16CHAR(25, 25),
+    VAR16CHAR(24, 25),
     /**
      * <code>VARBINARY = 26;</code>
      *
@@ -241,7 +233,7 @@ public final class TypeProtos {
      *  variable length binary
      * </pre>
      */
-    VARBINARY(26, 26),
+    VARBINARY(25, 26),
     /**
      * <code>UINT1 = 29;</code>
      *
@@ -249,7 +241,7 @@ public final class TypeProtos {
      *  unsigned 1 byte integer
      * </pre>
      */
-    UINT1(27, 29),
+    UINT1(26, 29),
     /**
      * <code>UINT2 = 30;</code>
      *
@@ -257,7 +249,7 @@ public final class TypeProtos {
      *  unsigned 2 byte integer
      * </pre>
      */
-    UINT2(28, 30),
+    UINT2(27, 30),
     /**
      * <code>UINT4 = 31;</code>
      *
@@ -265,7 +257,7 @@ public final class TypeProtos {
      *  unsigned 4 byte integer
      * </pre>
      */
-    UINT4(29, 31),
+    UINT4(28, 31),
     /**
      * <code>UINT8 = 32;</code>
      *
@@ -273,7 +265,7 @@ public final class TypeProtos {
      *  unsigned 8 byte integer
      * </pre>
      */
-    UINT8(30, 32),
+    UINT8(29, 32),
     /**
      * <code>DECIMAL28DENSE = 33;</code>
      *
@@ -281,7 +273,7 @@ public final class TypeProtos {
      * dense decimal representation, supporting precision between 19 and 28
      * </pre>
      */
-    DECIMAL28DENSE(31, 33),
+    DECIMAL28DENSE(30, 33),
     /**
      * <code>DECIMAL38DENSE = 34;</code>
      *
@@ -289,7 +281,7 @@ public final class TypeProtos {
      * dense decimal representation, supporting precision between 28 and 38
      * </pre>
      */
-    DECIMAL38DENSE(32, 34),
+    DECIMAL38DENSE(31, 34),
     /**
      * <code>NULL = 37;</code>
      *
@@ -300,7 +292,7 @@ public final class TypeProtos {
      *    MSGPACK4 = 36;   //  msgpack encoded complex type. (up to 2^32 in length)
      * </pre>
      */
-    NULL(33, 37),
+    NULL(32, 37),
     /**
      * <code>INTERVALYEAR = 38;</code>
      *
@@ -308,7 +300,7 @@ public final class TypeProtos {
      * Interval type specifying YEAR to MONTH
      * </pre>
      */
-    INTERVALYEAR(34, 38),
+    INTERVALYEAR(33, 38),
     /**
      * <code>INTERVALDAY = 39;</code>
      *
@@ -316,11 +308,11 @@ public final class TypeProtos {
      * Interval type specifying DAY to SECONDS
      * </pre>
      */
-    INTERVALDAY(35, 39),
+    INTERVALDAY(34, 39),
     /**
      * <code>LIST = 40;</code>
      */
-    LIST(36, 40),
+    LIST(35, 40),
     ;
 
     /**
@@ -340,14 +332,6 @@ public final class TypeProtos {
      */
     public static final int MAP_VALUE = 1;
     /**
-     * <code>REPEATMAP = 2;</code>
-     *
-     * <pre>
-     *  a repeated map column (means that multiple children sit below this)
-     * </pre>
-     */
-    public static final int REPEATMAP_VALUE = 2;
-    /**
      * <code>TINYINT = 3;</code>
      *
      * <pre>
@@ -622,7 +606,6 @@ public final class TypeProtos {
       switch (value) {
         case 0: return LATE;
         case 1: return MAP;
-        case 2: return REPEATMAP;
         case 3: return TINYINT;
         case 4: return SMALLINT;
         case 5: return INT;
@@ -1788,22 +1771,22 @@ public final class TypeProtos {
       "inor_type\030\001 \001(\0162\021.common.MinorType\022\036\n\004mo" +
       "de\030\002 \001(\0162\020.common.DataMode\022\r\n\005width\030\003 \001(" +
       "\005\022\021\n\tprecision\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010t" +
-      "imeZone\030\006 \001(\005*\205\004\n\tMinorType\022\010\n\004LATE\020\000\022\007\n" +
-      "\003MAP\020\001\022\r\n\tREPEATMAP\020\002\022\013\n\007TINYINT\020\003\022\014\n\010SM" +
-      "ALLINT\020\004\022\007\n\003INT\020\005\022\n\n\006BIGINT\020\006\022\014\n\010DECIMAL" +
-      "9\020\007\022\r\n\tDECIMAL18\020\010\022\023\n\017DECIMAL28SPARSE\020\t\022" +
-      "\023\n\017DECIMAL38SPARSE\020\n\022\t\n\005MONEY\020\013\022\010\n\004DATE\020" +
-      "\014\022\010\n\004TIME\020\r\022\n\n\006TIMETZ\020\016\022\017\n\013TIMESTAMPTZ\020\017",
-      "\022\r\n\tTIMESTAMP\020\020\022\014\n\010INTERVAL\020\021\022\n\n\006FLOAT4\020" +
-      "\022\022\n\n\006FLOAT8\020\023\022\007\n\003BIT\020\024\022\r\n\tFIXEDCHAR\020\025\022\017\n" +
-      "\013FIXED16CHAR\020\026\022\017\n\013FIXEDBINARY\020\027\022\013\n\007VARCH" +
-      "AR\020\030\022\r\n\tVAR16CHAR\020\031\022\r\n\tVARBINARY\020\032\022\t\n\005UI" +
-      "NT1\020\035\022\t\n\005UINT2\020\036\022\t\n\005UINT4\020\037\022\t\n\005UINT8\020 \022\022" +
-      "\n\016DECIMAL28DENSE\020!\022\022\n\016DECIMAL38DENSE\020\"\022\010" +
-      "\n\004NULL\020%\022\020\n\014INTERVALYEAR\020&\022\017\n\013INTERVALDA" +
-      "Y\020\'\022\010\n\004LIST\020(*4\n\010DataMode\022\014\n\010OPTIONAL\020\000\022" +
-      "\014\n\010REQUIRED\020\001\022\014\n\010REPEATED\020\002B-\n\035org.apach" +
-      "e.drill.common.typesB\nTypeProtosH\001"
+      "imeZone\030\006 \001(\005*\366\003\n\tMinorType\022\010\n\004LATE\020\000\022\007\n" +
+      "\003MAP\020\001\022\013\n\007TINYINT\020\003\022\014\n\010SMALLINT\020\004\022\007\n\003INT" +
+      "\020\005\022\n\n\006BIGINT\020\006\022\014\n\010DECIMAL9\020\007\022\r\n\tDECIMAL1" +
+      "8\020\010\022\023\n\017DECIMAL28SPARSE\020\t\022\023\n\017DECIMAL38SPA" +
+      "RSE\020\n\022\t\n\005MONEY\020\013\022\010\n\004DATE\020\014\022\010\n\004TIME\020\r\022\n\n\006" +
+      "TIMETZ\020\016\022\017\n\013TIMESTAMPTZ\020\017\022\r\n\tTIMESTAMP\020\020",
+      "\022\014\n\010INTERVAL\020\021\022\n\n\006FLOAT4\020\022\022\n\n\006FLOAT8\020\023\022\007" +
+      "\n\003BIT\020\024\022\r\n\tFIXEDCHAR\020\025\022\017\n\013FIXED16CHAR\020\026\022" +
+      "\017\n\013FIXEDBINARY\020\027\022\013\n\007VARCHAR\020\030\022\r\n\tVAR16CH" +
+      "AR\020\031\022\r\n\tVARBINARY\020\032\022\t\n\005UINT1\020\035\022\t\n\005UINT2\020" +
+      "\036\022\t\n\005UINT4\020\037\022\t\n\005UINT8\020 \022\022\n\016DECIMAL28DENS" +
+      "E\020!\022\022\n\016DECIMAL38DENSE\020\"\022\010\n\004NULL\020%\022\020\n\014INT" +
+      "ERVALYEAR\020&\022\017\n\013INTERVALDAY\020\'\022\010\n\004LIST\020(*4" +
+      "\n\010DataMode\022\014\n\010OPTIONAL\020\000\022\014\n\010REQUIRED\020\001\022\014" +
+      "\n\010REPEATED\020\002B-\n\035org.apache.drill.common." +
+      "typesB\nTypeProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
index a1cf6f8..9e041dc 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -1675,6 +1675,20 @@ public final class BitControl {
      * <code>optional int64 running_time = 9;</code>
      */
     long getRunningTime();
+
+    // optional .exec.shared.MinorFragmentProfile profile = 10;
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    boolean hasProfile();
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile();
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder();
   }
   /**
    * Protobuf type {@code exec.bit.control.FragmentStatus}
@@ -1794,6 +1808,19 @@ public final class BitControl {
               runningTime_ = input.readInt64();
               break;
             }
+            case 82: {
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000200) == 0x00000200)) {
+                subBuilder = profile_.toBuilder();
+              }
+              profile_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(profile_);
+                profile_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000200;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -2108,6 +2135,28 @@ public final class BitControl {
       return runningTime_;
     }
 
+    // optional .exec.shared.MinorFragmentProfile profile = 10;
+    public static final int PROFILE_FIELD_NUMBER = 10;
+    private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_;
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    public boolean hasProfile() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
+      return profile_;
+    }
+    /**
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
+      return profile_;
+    }
+
     private void initFields() {
       memoryUse_ = 0L;
       batchesCompleted_ = 0L;
@@ -2118,6 +2167,7 @@ public final class BitControl {
       handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
       error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
       runningTime_ = 0L;
+      profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2158,6 +2208,9 @@ public final class BitControl {
       if (((bitField0_ & 0x00000100) == 0x00000100)) {
         output.writeInt64(9, runningTime_);
       }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeMessage(10, profile_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -2203,6 +2256,10 @@ public final class BitControl {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(9, runningTime_);
       }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(10, profile_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -2313,6 +2370,7 @@ public final class BitControl {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getHandleFieldBuilder();
           getErrorFieldBuilder();
+          getProfileFieldBuilder();
         }
       }
       private static Builder create() {
@@ -2347,6 +2405,12 @@ public final class BitControl {
         bitField0_ = (bitField0_ & ~0x00000080);
         runningTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000100);
+        if (profileBuilder_ == null) {
+          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+        } else {
+          profileBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000200);
         return this;
       }
 
@@ -2419,6 +2483,14 @@ public final class BitControl {
           to_bitField0_ |= 0x00000100;
         }
         result.runningTime_ = runningTime_;
+        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
+          to_bitField0_ |= 0x00000200;
+        }
+        if (profileBuilder_ == null) {
+          result.profile_ = profile_;
+        } else {
+          result.profile_ = profileBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2462,6 +2534,9 @@ public final class BitControl {
         if (other.hasRunningTime()) {
           setRunningTime(other.getRunningTime());
         }
+        if (other.hasProfile()) {
+          mergeProfile(other.getProfile());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -2957,6 +3032,123 @@ public final class BitControl {
         return this;
       }
 
+      // optional .exec.shared.MinorFragmentProfile profile = 10;
+      private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> profileBuilder_;
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public boolean hasProfile() {
+        return ((bitField0_ & 0x00000200) == 0x00000200);
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
+        if (profileBuilder_ == null) {
+          return profile_;
+        } else {
+          return profileBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public Builder setProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (profileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          profile_ = value;
+          onChanged();
+        } else {
+          profileBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public Builder setProfile(
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
+        if (profileBuilder_ == null) {
+          profile_ = builderForValue.build();
+          onChanged();
+        } else {
+          profileBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public Builder mergeProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (profileBuilder_ == null) {
+          if (((bitField0_ & 0x00000200) == 0x00000200) &&
+              profile_ != org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance()) {
+            profile_ =
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.newBuilder(profile_).mergeFrom(value).buildPartial();
+          } else {
+            profile_ = value;
+          }
+          onChanged();
+        } else {
+          profileBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000200;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public Builder clearProfile() {
+        if (profileBuilder_ == null) {
+          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+          onChanged();
+        } else {
+          profileBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000200);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder getProfileBuilder() {
+        bitField0_ |= 0x00000200;
+        onChanged();
+        return getProfileFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
+        if (profileBuilder_ != null) {
+          return profileBuilder_.getMessageOrBuilder();
+        } else {
+          return profile_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> 
+          getProfileFieldBuilder() {
+        if (profileBuilder_ == null) {
+          profileBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder>(
+                  profile_,
+                  getParentForChildren(),
+                  isClean());
+          profile_ = null;
+        }
+        return profileBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.bit.control.FragmentStatus)
     }
 
@@ -5898,7 +6090,7 @@ public final class BitControl {
       "(\0162\027.exec.shared.RpcChannel:\013BIT_CONTROL" +
       "\022(\n\010endpoint\030\003 \001(\0132\026.exec.DrillbitEndpoi" +
       "nt\"F\n\tBitStatus\0229\n\017fragment_status\030\001 \003(\013" +
-      "2 .exec.bit.control.FragmentStatus\"\261\003\n\016F" +
+      "2 .exec.bit.control.FragmentStatus\"\345\003\n\016F" +
       "ragmentStatus\022\022\n\nmemory_use\030\001 \001(\003\022\031\n\021bat" +
       "ches_completed\030\002 \001(\003\022\031\n\021records_complete",
       "d\030\003 \001(\003\022\'\n\037estimated_completion_percenta" +
@@ -5907,30 +6099,31 @@ public final class BitControl {
       "processed\030\006 \001(\003\022(\n\006handle\030\007 \001(\0132\030.exec.b" +
       "it.FragmentHandle\022(\n\005error\030\010 \001(\0132\031.exec." +
       "shared.DrillPBError\022\024\n\014running_time\030\t \001(" +
-      "\003\"k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAIT" +
-      "ING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHE" +
-      "D\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\"\276\003\n\014PlanF" +
-      "ragment\022(\n\006handle\030\001 \001(\0132\030.exec.bit.Fragm",
-      "entHandle\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_c" +
-      "ost\030\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_c" +
-      "ost\030\007 \001(\002\022\025\n\rfragment_json\030\010 \001(\t\022*\n\nassi" +
-      "gnment\030\n \001(\0132\026.exec.DrillbitEndpoint\022\025\n\r" +
-      "leaf_fragment\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.e" +
-      "xec.DrillbitEndpoint\022\035\n\013mem_initial\030\014 \001(" +
-      "\003:\01020000000\022\034\n\007mem_max\030\r \001(\003:\0132000000000" +
-      "0\022\030\n\020query_start_time\030\016 \001(\003\0221\n\013credentia" +
-      "ls\030\017 \001(\0132\034.exec.shared.UserCredentials\022\021" +
-      "\n\ttime_zone\030\020 \001(\005\022\024\n\014options_json\030\021 \001(\t\"",
-      "f\n\017WorkQueueStatus\022(\n\010endpoint\030\001 \001(\0132\026.e" +
-      "xec.DrillbitEndpoint\022\024\n\014queue_length\030\002 \001" +
-      "(\005\022\023\n\013report_time\030\003 \001(\003*\332\001\n\007RpcType\022\r\n\tH" +
-      "ANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027REQ_" +
-      "INIATILIZE_FRAGMENT\020\003\022\027\n\023REQ_CANCEL_FRAG" +
-      "MENT\020\006\022\027\n\023REQ_FRAGMENT_STATUS\020\007\022\022\n\016REQ_B" +
-      "IT_STATUS\020\010\022\030\n\024RESP_FRAGMENT_HANDLE\020\t\022\030\n" +
-      "\024RESP_FRAGMENT_STATUS\020\n\022\023\n\017RESP_BIT_STAT" +
-      "US\020\013B+\n\033org.apache.drill.exec.protoB\nBit" +
-      "ControlH\001"
+      "\003\0222\n\007profile\030\n \001(\0132!.exec.shared.MinorFr" +
+      "agmentProfile\"k\n\rFragmentState\022\013\n\007SENDIN" +
+      "G\020\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020" +
+      "\002\022\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED",
+      "\020\005\"\276\003\n\014PlanFragment\022(\n\006handle\030\001 \001(\0132\030.ex" +
+      "ec.bit.FragmentHandle\022\024\n\014network_cost\030\004 " +
+      "\001(\002\022\020\n\010cpu_cost\030\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002" +
+      "\022\023\n\013memory_cost\030\007 \001(\002\022\025\n\rfragment_json\030\010" +
+      " \001(\t\022*\n\nassignment\030\n \001(\0132\026.exec.Drillbit" +
+      "Endpoint\022\025\n\rleaf_fragment\030\t \001(\010\022\'\n\007forem" +
+      "an\030\013 \001(\0132\026.exec.DrillbitEndpoint\022\035\n\013mem_" +
+      "initial\030\014 \001(\003:\01020000000\022\034\n\007mem_max\030\r \001(\003" +
+      ":\01320000000000\022\030\n\020query_start_time\030\016 \001(\003\022" +
+      "1\n\013credentials\030\017 \001(\0132\034.exec.shared.UserC",
+      "redentials\022\021\n\ttime_zone\030\020 \001(\005\022\024\n\014options" +
+      "_json\030\021 \001(\t\"f\n\017WorkQueueStatus\022(\n\010endpoi" +
+      "nt\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014queu" +
+      "e_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003*\332\001\n\007" +
+      "RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODB" +
+      "YE\020\002\022\033\n\027REQ_INIATILIZE_FRAGMENT\020\003\022\027\n\023REQ" +
+      "_CANCEL_FRAGMENT\020\006\022\027\n\023REQ_FRAGMENT_STATU" +
+      "S\020\007\022\022\n\016REQ_BIT_STATUS\020\010\022\030\n\024RESP_FRAGMENT" +
+      "_HANDLE\020\t\022\030\n\024RESP_FRAGMENT_STATUS\020\n\022\023\n\017R" +
+      "ESP_BIT_STATUS\020\013B+\n\033org.apache.drill.exe",
+      "c.protoB\nBitControlH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5954,7 +6147,7 @@ public final class BitControl {
           internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_FragmentStatus_descriptor,
-              new java.lang.String[] { "MemoryUse", "BatchesCompleted", "RecordsCompleted", "EstimatedCompletionPercentage", "State", "DataProcessed", "Handle", "Error", "RunningTime", });
+              new java.lang.String[] { "MemoryUse", "BatchesCompleted", "RecordsCompleted", "EstimatedCompletionPercentage", "State", "DataProcessed", "Handle", "Error", "RunningTime", "Profile", });
           internal_static_exec_bit_control_PlanFragment_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new


[11/24] git commit: ispan

Posted by ja...@apache.org.
ispan


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/8621b682
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/8621b682
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/8621b682

Branch: refs/heads/diagnostics2
Commit: 8621b682cae0f8c8f58ff2e5b750544113bf52ee
Parents: 5472140
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun May 4 16:58:41 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:12:27 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/pom.xml                          |  10 +
 .../exec/cache/AbstractDataSerializable.java    |  33 ++
 .../exec/cache/AbstractStreamSerializable.java  |  34 ++
 .../drill/exec/cache/CachedVectorContainer.java |  77 +++++
 .../drill/exec/cache/DistributedCache.java      |   4 +-
 .../drill/exec/cache/DrillSerializable.java     |  17 +-
 .../cache/HCVectorAccessibleSerializer.java     |  56 ----
 .../org/apache/drill/exec/cache/HazelCache.java | 252 ---------------
 .../exec/cache/JacksonDrillSerializable.java    |  48 +--
 .../org/apache/drill/exec/cache/LocalCache.java | 305 ------------------
 .../cache/LoopedAbstractDrillSerializable.java  |  80 +++++
 .../apache/drill/exec/cache/ProtoBufImpl.java   |  49 ---
 .../apache/drill/exec/cache/ProtoBufWrap.java   |  67 ----
 .../org/apache/drill/exec/cache/ProtoMap.java   |  52 ----
 .../drill/exec/cache/ProtoSerializable.java     |  65 ++++
 .../cache/VectorAccessibleSerializable.java     |  22 +-
 .../hazel/HCVectorAccessibleSerializer.java     |  58 ++++
 .../drill/exec/cache/hazel/HazelCache.java      | 258 ++++++++++++++++
 .../drill/exec/cache/hazel/ProtoBufImpl.java    |  49 +++
 .../drill/exec/cache/hazel/ProtoBufWrap.java    |  67 ++++
 .../apache/drill/exec/cache/hazel/ProtoMap.java |  52 ++++
 .../drill/exec/cache/infinispan/ICache.java     | 266 ++++++++++++++++
 .../infinispan/VAAdvancedExternalizer.java      |  72 +++++
 .../cache/infinispan/ZookeeperCacheStore.java   |  66 ++++
 .../drill/exec/cache/local/LocalCache.java      | 309 +++++++++++++++++++
 .../OrderedPartitionRecordBatch.java            |  20 +-
 .../org/apache/drill/exec/server/Drillbit.java  |   2 +-
 .../drill/exec/server/RemoteServiceSet.java     |  17 +-
 .../java/org/apache/drill/PlanningBase.java     |   2 +-
 .../java/org/apache/drill/exec/cache/ISpan.java |  94 ++++++
 .../drill/exec/cache/TestVectorCache.java       | 136 +++++---
 .../drill/exec/store/TestOrphanSchema.java      |   2 +-
 .../drill/exec/store/ischema/OrphanSchema.java  |   9 +-
 33 files changed, 1750 insertions(+), 900 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index d693630..6d11614 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -224,6 +224,16 @@
       <version>3.1.4</version>
     </dependency>
     <dependency>
+      <groupId>org.infinispan</groupId>
+      <artifactId>infinispan-core</artifactId>
+      <version>6.0.1.Final</version>
+    </dependency>
+    <dependency>
+      <groupId>org.infinispan</groupId>
+      <artifactId>infinispan-tree</artifactId>
+      <version>6.0.1.Final</version>
+    </dependency>
+    <dependency>
       <groupId>org.codehaus.janino</groupId>
       <artifactId>janino</artifactId>
       <version>2.6.1</version>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractDataSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractDataSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractDataSerializable.java
new file mode 100644
index 0000000..f7b9eed
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractDataSerializable.java
@@ -0,0 +1,33 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractDataSerializable extends LoopedAbstractDrillSerializable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractDataSerializable.class);
+
+  @Override
+  public abstract void read(DataInput input) throws IOException;
+
+  @Override
+  public abstract void write(DataOutput output) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java
new file mode 100644
index 0000000..ef488d6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/AbstractStreamSerializable.java
@@ -0,0 +1,34 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+public abstract class AbstractStreamSerializable extends LoopedAbstractDrillSerializable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStreamSerializable.class);
+
+  @Override
+  public abstract void readFromStream(InputStream input) throws IOException;
+
+  @Override
+  public abstract void writeToStream(OutputStream output) throws IOException;
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
new file mode 100644
index 0000000..1447e28
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
@@ -0,0 +1,77 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+import org.apache.commons.io.output.ByteArrayOutputStream;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class CachedVectorContainer extends LoopedAbstractDrillSerializable {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CachedVectorContainer.class);
+
+  private final byte[] data;
+  private final BufferAllocator allocator;
+  private VectorContainer container;
+
+  public CachedVectorContainer(WritableBatch batch, BufferAllocator allocator) throws IOException {
+    VectorAccessibleSerializable va = new VectorAccessibleSerializable(batch, allocator);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    va.writeToStream(baos);
+    this.allocator = allocator;
+    this.data = baos.toByteArray();
+    va.clear();
+  }
+
+  public CachedVectorContainer(byte[] data, BufferAllocator allocator) {
+    this.data = data;
+    this.allocator = allocator;
+  }
+
+  private void construct() {
+    try {
+      VectorAccessibleSerializable va = new VectorAccessibleSerializable(allocator);
+      va.readFromStream(new ByteArrayInputStream(data));
+      this.container = va.get();
+    } catch (IOException e) {
+      throw new IllegalStateException(e);
+    }
+
+  }
+
+  public VectorAccessible get() {
+    if (container == null) {
+      construct();
+    }
+    return container;
+  }
+
+  public void clear() {
+    container.clear();
+    container = null;
+  }
+
+  public byte[] getData(){
+    return data;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
index 65362e0..aa87162 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
@@ -17,14 +17,12 @@
  */
 package org.apache.drill.exec.cache;
 
-import java.io.Closeable;
-
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 
 
-public interface DistributedCache extends Closeable{
+public interface DistributedCache extends AutoCloseable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedCache.class);
 
   public void run() throws DrillbitStartupException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DrillSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DrillSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DrillSerializable.java
index 4f266f7..21ed37c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DrillSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DrillSerializable.java
@@ -17,19 +17,20 @@
  */
 package org.apache.drill.exec.cache;
 
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
-import com.hazelcast.nio.serialization.DataSerializable;
-
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 /**
  * Classes that can be put in the Distributed Cache must implement this interface.
  */
-public interface DrillSerializable {
+public interface DrillSerializable extends Externalizable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSerializable.class);
-  public void readData(ObjectDataInput input) throws IOException;
+  public void read(DataInput input) throws IOException;
   public void readFromStream(InputStream input) throws IOException;
-  public void writeData(ObjectDataOutput output) throws IOException;
+  public void write(DataOutput output) throws IOException;
   public void writeToStream(OutputStream output) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HCVectorAccessibleSerializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HCVectorAccessibleSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HCVectorAccessibleSerializer.java
deleted file mode 100644
index 0d5ba96..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HCVectorAccessibleSerializer.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
-import com.hazelcast.nio.serialization.StreamSerializer;
-import org.apache.drill.common.util.DataInputInputStream;
-import org.apache.drill.common.util.DataOutputOutputStream;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.server.DrillbitContext;
-
-import java.io.*;
-
-/**
- * Wraps a DrillSerializable object. Objects of this class can be put in the HazelCast implementation of Distributed Cache
- */
-public class HCVectorAccessibleSerializer implements StreamSerializer<VectorAccessibleSerializable> {
-
-  private BufferAllocator allocator;
-
-  public HCVectorAccessibleSerializer(BufferAllocator allocator) {
-    this.allocator = allocator;
-  }
-
-  public VectorAccessibleSerializable read(ObjectDataInput in) throws IOException {
-    VectorAccessibleSerializable va = new VectorAccessibleSerializable(allocator);
-    va.readFromStream(DataInputInputStream.constructInputStream(in));
-    return va;
-  }
-
-  public void write(ObjectDataOutput out, VectorAccessibleSerializable va) throws IOException {
-    va.writeToStream(DataOutputOutputStream.constructOutputStream(out));
-  }
-
-  public void destroy() {}
-
-  public int getTypeId() {
-    return 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
deleted file mode 100644
index 0149a57..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.cache.ProtoBufImpl.HWorkQueueStatus;
-import org.apache.drill.exec.cache.ProtoBufImpl.HandlePlan;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
-import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.collect.Lists;
-import com.hazelcast.config.Config;
-import com.hazelcast.config.MapConfig;
-import com.hazelcast.config.SerializerConfig;
-import com.hazelcast.core.DuplicateInstanceNameException;
-import com.hazelcast.core.Hazelcast;
-import com.hazelcast.core.HazelcastInstance;
-import com.hazelcast.core.IAtomicLong;
-import com.hazelcast.core.IMap;
-import com.hazelcast.core.ITopic;
-import com.hazelcast.core.Message;
-import com.hazelcast.core.MessageListener;
-import com.hazelcast.nio.serialization.StreamSerializer;
-
-public class HazelCache implements DistributedCache {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HazelCache.class);
-
-  private final String instanceName;
-  private HazelcastInstance instance;
-  private ITopic<HWorkQueueStatus> workQueueLengths;
-  private HandlePlan fragments;
-  private Cache<WorkQueueStatus, Integer>  endpoints;
-  private BufferAllocator allocator;
-  private DrillConfig config;
-
-  public HazelCache(DrillConfig config, BufferAllocator allocator) {
-    this.instanceName = config.getString(ExecConstants.SERVICE_NAME);
-    this.allocator = allocator;
-    this.config = config;
-  }
-
-  private <T> void addSer(Config c, StreamSerializer<T> serializer, Class<T> clazz){
-    SerializerConfig sc = new SerializerConfig().setImplementation(serializer).setTypeClass(clazz);
-    c.getSerializationConfig().addSerializerConfig(sc);
-  }
-
-  @SuppressWarnings("rawtypes")
-  private <T> void addJSer(Config c, SerializationDefinition d){
-    SerializerConfig sc = new SerializerConfig().setImplementation(new JacksonAdvancedSerializer(d, config.getMapper())).setTypeClass(d.clazz);
-    c.getSerializationConfig().addSerializerConfig(sc);
-  }
-
-
-  private class Listener implements MessageListener<HWorkQueueStatus>{
-
-    @Override
-    public void onMessage(Message<HWorkQueueStatus> wrapped) {
-      logger.debug("Received new queue length message.");
-      endpoints.put(wrapped.getMessageObject().get(), 0);
-    }
-
-  }
-
-  public void run() {
-    Config c = new Config();
-    addSer(c, new HCVectorAccessibleSerializer(allocator), VectorAccessibleSerializable.class);
-    addJSer(c, SerializationDefinition.OPTION);
-    addJSer(c, SerializationDefinition.STORAGE_PLUGINS);
-
-    c.setInstanceName(instanceName);
-    c.getGroupConfig().setName(instanceName);
-    for (String s : DrillConfig.create().getStringList(ExecConstants.HAZELCAST_SUBNETS)) {
-      logger.debug("Adding interface: {}", s);
-      c.getNetworkConfig().getInterfaces().setEnabled(true).addInterface(s);
-    }
-
-    instance = getInstanceOrCreateNew(c);
-    workQueueLengths = instance.getTopic("queue-length");
-    fragments = new HandlePlan(instance);
-    endpoints = CacheBuilder.newBuilder().maximumSize(2000).build();
-    workQueueLengths.addMessageListener(new Listener());
-  }
-
-  private HazelcastInstance getInstanceOrCreateNew(Config c) {
-    for (HazelcastInstance instance : Hazelcast.getAllHazelcastInstances()){
-      if (instance.getName().equals(this.instanceName))
-        return instance;
-    }
-    try {
-    return Hazelcast.newHazelcastInstance(c);
-    } catch (DuplicateInstanceNameException e) {
-      return getInstanceOrCreateNew(c);
-    }
-  }
-
-//  @Override
-//  public void updateLocalQueueLength(int length) {
-//    workQueueLengths.publish(new HWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
-//        .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
-//  }
-//
-//  @Override
-//  public List<WorkQueueStatus> getQueueLengths() {
-//    return Lists.newArrayList(endpoints.asMap().keySet());
-//  }
-
-  @Override
-  public void close() throws IOException {
-    this.instance.getLifecycleService().shutdown();
-  }
-
-  @Override
-  public PlanFragment getFragment(FragmentHandle handle) {
-    return this.fragments.get(handle);
-  }
-
-  @Override
-  public void storeFragment(PlanFragment fragment) {
-    fragments.put(fragment.getHandle(), fragment);
-  }
-
-
-  @Override
-  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
-    com.hazelcast.core.MultiMap<String, V> mmap = this.instance.getMultiMap(clazz.toString());
-    return new HCDistributedMultiMapImpl<V>(mmap, clazz);
-  }
-
-  @Override
-  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
-    return getNamedMap(clazz.getName(), clazz);
-  }
-
-
-  @Override
-  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
-    IMap<String, V> imap = this.instance.getMap(name);
-    MapConfig myMapConfig = new MapConfig();
-    myMapConfig.setBackupCount(0);
-    myMapConfig.setReadBackupData(true);
-    instance.getConfig().getMapConfigs().put(clazz.toString(), myMapConfig);
-    return new HCDistributedMapImpl<V>(imap);
-  }
-
-  @Override
-  public Counter getCounter(String name) {
-    return new HCCounterImpl(this.instance.getAtomicLong(name));
-  }
-
-
-
-  public static class HCDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
-    private final IMap<String, V> m;
-
-    public HCDistributedMapImpl(IMap<String, V> m) {
-      this.m = m;
-    }
-
-    public V get(String key) {
-      return m.get(key);
-    }
-
-    public void put(String key, V value) {
-      m.put(key, value);
-    }
-
-    public void putIfAbsent(String key, V value) {
-      m.putIfAbsent(key, value);
-    }
-
-    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeunit) {
-      m.putIfAbsent(key, value, ttl, timeunit);
-
-    }
-
-    @Override
-    public Iterator<Entry<String, V>> iterator() {
-      return m.entrySet().iterator();
-    }
-
-
-  }
-
-  public static class HCDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
-    private com.hazelcast.core.MultiMap<String, V> mmap;
-
-    public HCDistributedMultiMapImpl(com.hazelcast.core.MultiMap<String, V> mmap, Class<V> clazz) {
-      this.mmap = mmap;
-    }
-
-    public Collection<V> get(String key) {
-      List<V> list = Lists.newArrayList();
-      for (V v : mmap.get(key)) {
-        list.add(v);
-      }
-      return list;
-    }
-
-    @Override
-    public void put(String key, V value) {
-      mmap.put(key, value);
-    }
-  }
-
-  public static class HCCounterImpl implements Counter {
-    private IAtomicLong n;
-
-    public HCCounterImpl(IAtomicLong n) {
-      this.n = n;
-    }
-
-    public long get() {
-      return n.get();
-    }
-
-    public long incrementAndGet() {
-      return n.incrementAndGet();
-    }
-
-    public long decrementAndGet() {
-      return n.decrementAndGet();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonDrillSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonDrillSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonDrillSerializable.java
index dcfc1ec..617c356 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonDrillSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonDrillSerializable.java
@@ -17,47 +17,39 @@
  */
 package org.apache.drill.exec.cache;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
-import com.hazelcast.nio.serialization.DataSerializable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.DataInputInputStream;
-import org.apache.drill.common.util.DataOutputOutputStream;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.planner.logical.StoragePlugins;
 import org.apache.drill.exec.server.DrillbitContext;
 
-import java.io.*;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
-public abstract class JacksonDrillSerializable<T> implements DrillSerializable, DataSerializable{
+public abstract class JacksonDrillSerializable<T> extends LoopedAbstractDrillSerializable implements DrillSerializable{
   private ObjectMapper mapper;
   private T obj;
+  private Class<T> clazz;
 
-  public JacksonDrillSerializable(DrillbitContext context, T obj) {
+  public JacksonDrillSerializable(DrillbitContext context, T obj, Class<T> clazz) {
+    this(clazz);
     this.mapper = context.getConfig().getMapper();
     this.obj = obj;
   }
 
-  public JacksonDrillSerializable() {
+  public JacksonDrillSerializable(Class<T> clazz) {
+    this.clazz = clazz;
   }
 
   @Override
-  public void readData(ObjectDataInput input) throws IOException {
-    readFromStream(DataInputInputStream.constructInputStream(input));
-  }
-
-  public void readFromStream(InputStream input, Class clazz) throws IOException {
+  public void readFromStream(InputStream input) throws IOException {
     mapper = DrillConfig.create().getMapper();
     obj = (T) mapper.readValue(input, clazz);
   }
 
   @Override
-  public void writeData(ObjectDataOutput output) throws IOException {
-    writeToStream(DataOutputOutputStream.constructOutputStream(output));
-  }
-
-  @Override
   public void writeToStream(OutputStream output) throws IOException {
     output.write(mapper.writeValueAsBytes(obj));
   }
@@ -66,4 +58,20 @@ public abstract class JacksonDrillSerializable<T> implements DrillSerializable,
     return obj;
   }
 
+  public static class StoragePluginsSerializable extends JacksonDrillSerializable<StoragePlugins> {
+
+    public StoragePluginsSerializable(DrillbitContext context, StoragePlugins obj) {
+      super(context, obj, StoragePlugins.class);
+    }
+
+    public StoragePluginsSerializable(BufferAllocator allocator) {
+      super(StoragePlugins.class);
+    }
+
+    public StoragePluginsSerializable() {
+      super(StoragePlugins.class);
+    }
+
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
deleted file mode 100644
index 0fb4b82..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.InvocationTargetException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.DataInputInputStream;
-import org.apache.drill.common.util.DataOutputOutputStream;
-import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.TopLevelAllocator;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.ByteArrayDataInput;
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
-
-public class LocalCache implements DistributedCache {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
-
-  private volatile Map<FragmentHandle, PlanFragment> handles;
-  private volatile ConcurrentMap<String, DistributedMap<?>> namedMaps;
-  private volatile ConcurrentMap<Class<?>, DistributedMap<?>> maps;
-  private volatile ConcurrentMap<Class<?>, DistributedMultiMap<?>> multiMaps;
-  private volatile ConcurrentMap<String, Counter> counters;
-  private static final BufferAllocator allocator = new TopLevelAllocator();
-
-  private static final ObjectMapper mapper = DrillConfig.create().getMapper();
-
-  @Override
-  public void close() throws IOException {
-    handles = null;
-  }
-
-  @Override
-  public void run() throws DrillbitStartupException {
-    handles = Maps.newConcurrentMap();
-    maps = Maps.newConcurrentMap();
-    multiMaps = Maps.newConcurrentMap();
-    counters = Maps.newConcurrentMap();
-    namedMaps = Maps.newConcurrentMap();
-  }
-
-  @Override
-  public PlanFragment getFragment(FragmentHandle handle) {
-//    logger.debug("looking for fragment with handle: {}", handle);
-    return handles.get(handle);
-  }
-
-  @Override
-  public void storeFragment(PlanFragment fragment) {
-//    logger.debug("Storing fragment: {}", fragment);
-    handles.put(fragment.getHandle(), fragment);
-  }
-
-  @Override
-  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
-    DistributedMultiMap<V> mmap = (DistributedMultiMap<V>) multiMaps.get(clazz);
-    if (mmap == null) {
-      multiMaps.putIfAbsent(clazz, new LocalDistributedMultiMapImpl<V>(clazz));
-      return (DistributedMultiMap<V>) multiMaps.get(clazz);
-    } else {
-      return mmap;
-    }
-  }
-
-  @Override
-  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
-    DistributedMap m = maps.get(clazz);
-    if (m == null) {
-      maps.putIfAbsent(clazz, new LocalDistributedMapImpl<V>(clazz));
-      return (DistributedMap<V>) maps.get(clazz);
-    } else {
-      return m;
-    }
-  }
-
-
-  @Override
-  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
-    DistributedMap m = namedMaps.get(clazz);
-    if (m == null) {
-      namedMaps.putIfAbsent(name, new LocalDistributedMapImpl<V>(clazz));
-      return (DistributedMap<V>) namedMaps.get(name);
-    } else {
-      return m;
-    }
-  }
-
-  @Override
-  public Counter getCounter(String name) {
-    Counter c = counters.get(name);
-    if (c == null) {
-      counters.putIfAbsent(name, new LocalCounterImpl());
-      return counters.get(name);
-    } else {
-      return c;
-    }
-  }
-
-  public static ByteArrayDataOutput serialize(DrillSerializable obj) {
-    if(obj instanceof JacksonSerializable){
-      try{
-        ByteArrayDataOutput out = ByteStreams.newDataOutput();
-        out.write(mapper.writeValueAsBytes(obj));
-        return out;
-      }catch(Exception e){
-        throw new RuntimeException(e);
-      }
-    }
-
-    ByteArrayDataOutput out = ByteStreams.newDataOutput();
-    OutputStream outputStream = DataOutputOutputStream.constructOutputStream(out);
-    try {
-      obj.writeToStream(outputStream);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    try {
-      outputStream.flush();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return out;
-  }
-
-  public static <V extends DrillSerializable> V deserialize(byte[] bytes, Class<V> clazz) {
-    if(JacksonSerializable.class.isAssignableFrom(clazz)){
-      try{
-        return (V) mapper.readValue(bytes, clazz);
-      }catch(Exception e){
-        throw new RuntimeException(e);
-      }
-    }
-
-    ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
-    InputStream inputStream = DataInputInputStream.constructInputStream(in);
-    try {
-      V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
-      obj.readFromStream(inputStream);
-      return obj;
-    } catch (InstantiationException | IllegalAccessException | IOException | NoSuchMethodException | InvocationTargetException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public static class LocalDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
-    private ArrayListMultimap<String, ByteArrayDataOutput> mmap;
-    private Class<V> clazz;
-
-    public LocalDistributedMultiMapImpl(Class<V> clazz) {
-      mmap = ArrayListMultimap.create();
-      this.clazz = clazz;
-    }
-
-    @Override
-    public Collection<V> get(String key) {
-      List<V> list = Lists.newArrayList();
-      for (ByteArrayDataOutput o : mmap.get(key)) {
-        list.add(deserialize(o.toByteArray(), this.clazz));
-      }
-      return list;
-    }
-
-    @Override
-    public void put(String key, DrillSerializable value) {
-      mmap.put(key, serialize(value));
-    }
-  }
-
-  public static class LocalDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
-    protected ConcurrentMap<String, ByteArrayDataOutput> m;
-    protected Class<V> clazz;
-
-    public LocalDistributedMapImpl(Class<V> clazz) {
-      m = Maps.newConcurrentMap();
-      this.clazz = clazz;
-    }
-
-    @Override
-    public V get(String key) {
-      if (m.get(key) == null) return null;
-      ByteArrayDataOutput b = m.get(key);
-      byte[] bytes = b.toByteArray();
-      return (V) deserialize(m.get(key).toByteArray(), this.clazz);
-    }
-
-    @Override
-    public void put(String key, V value) {
-      m.put(key, serialize(value));
-    }
-
-    @Override
-    public void putIfAbsent(String key, V value) {
-      m.putIfAbsent(key, serialize(value));
-    }
-
-    @Override
-    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit) {
-      m.putIfAbsent(key, serialize(value));
-      logger.warn("Expiration not implemented in local map cache");
-    }
-
-    private class DeserializingTransformer implements Iterator<Map.Entry<String, V> >{
-      private Iterator<Map.Entry<String, ByteArrayDataOutput>> inner;
-
-      public DeserializingTransformer(Iterator<Entry<String, ByteArrayDataOutput>> inner) {
-        super();
-        this.inner = inner;
-      }
-
-      @Override
-      public boolean hasNext() {
-        return inner.hasNext();
-      }
-
-      @Override
-      public Entry<String, V> next() {
-        return newEntry(inner.next());
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-
-      public Entry<String, V> newEntry(final Entry<String, ByteArrayDataOutput> input) {
-        return new Map.Entry<String, V>(){
-
-          @Override
-          public String getKey() {
-            return input.getKey();
-          }
-
-          @Override
-          public V getValue() {
-            return deserialize(input.getValue().toByteArray(), clazz);
-          }
-
-          @Override
-          public V setValue(V value) {
-            throw new UnsupportedOperationException();
-          }
-
-        };
-      }
-
-    }
-    @Override
-    public Iterator<Entry<String, V>> iterator() {
-      return new DeserializingTransformer(m.entrySet().iterator());
-    }
-  }
-
-  public static class LocalCounterImpl implements Counter {
-    private AtomicLong al = new AtomicLong();
-
-    @Override
-    public long get() {
-      return al.get();
-    }
-
-    @Override
-    public long incrementAndGet() {
-      return al.incrementAndGet();
-    }
-
-    @Override
-    public long decrementAndGet() {
-      return al.decrementAndGet();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java
new file mode 100644
index 0000000..1de030d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LoopedAbstractDrillSerializable.java
@@ -0,0 +1,80 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.OutputStream;
+
+import org.apache.drill.common.util.DataInputInputStream;
+import org.apache.drill.common.util.DataOutputOutputStream;
+
+/**
+ * Helper class that holds the basic functionality to interchangably use the different Drill serializble interfaces.
+ * This is package private as users should utilize either AbstractDataSerializable or AbstractStreamSerializable instead
+ * to avoid infinite loops.
+ */
+abstract class LoopedAbstractDrillSerializable implements DrillSerializable {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LoopedAbstractDrillSerializable.class);
+
+  @Override
+  public void writeExternal(ObjectOutput out) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    writeToStream(baos);
+    byte[] ba = baos.toByteArray();
+    out.write(ba.length);
+    out.write(ba);
+  }
+
+
+  @Override
+  public void read(DataInput input) throws IOException {
+    readFromStream(DataInputInputStream.constructInputStream(input));
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    writeToStream(DataOutputOutputStream.constructOutputStream(output));
+  }
+
+  @Override
+  public void readFromStream(InputStream input) throws IOException {
+    read(new DataInputStream(input));
+  }
+
+  @Override
+  public void writeToStream(OutputStream output) throws IOException {
+    write(new DataOutputStream(output));
+  }
+
+  @Override
+  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+    int len = in.readInt();
+    byte[] bytes = new byte[len];
+    in.readFully(bytes);
+    readFromStream(new ByteArrayInputStream(bytes));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
deleted file mode 100644
index 53b2bfa..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
-import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-
-import com.hazelcast.core.HazelcastInstance;
-
-public class ProtoBufImpl {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufImpl.class);
-  
-  public static class HWorkQueueStatus extends ProtoBufWrap<WorkQueueStatus>{
-    public HWorkQueueStatus() {super(WorkQueueStatus.PARSER);}
-    public HWorkQueueStatus(WorkQueueStatus value) {super(value, WorkQueueStatus.PARSER);}
-  }
-  
-  public static class HFragmentHandle extends ProtoBufWrap<FragmentHandle>{
-    public HFragmentHandle() {super(FragmentHandle.PARSER);}
-    public HFragmentHandle(FragmentHandle value) {super(value, FragmentHandle.PARSER);}
-  }
-  
-  public static class HPlanFragment extends ProtoBufWrap<PlanFragment>{
-    public HPlanFragment() {super(PlanFragment.PARSER);}
-    public HPlanFragment(PlanFragment value) {super(value, PlanFragment.PARSER);}
-  }
-  
-  public static class HandlePlan extends ProtoMap<FragmentHandle, PlanFragment, HFragmentHandle, HPlanFragment>{
-    public HandlePlan(HazelcastInstance instance) {super(instance, "plan-fragment-cache");}
-    public HFragmentHandle getNewKey(FragmentHandle key) {return new HFragmentHandle(key);}
-    public HPlanFragment getNewValue(PlanFragment value) {return new HPlanFragment(value);}
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
deleted file mode 100644
index d55c133..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import java.io.IOException;
-
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
-import com.hazelcast.nio.serialization.DataSerializable;
-
-public abstract class ProtoBufWrap<T extends MessageLite> implements DataSerializable{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufWrap.class);
-  
-  T value;
-  final Parser<T> parser;
-  
-  public ProtoBufWrap(Parser<T> parser){
-    this(null, parser);
-  }
-  
-  public ProtoBufWrap(T value, Parser<T> parser){
-    this.value = value;
-    this.parser = parser;
-  }
-  
-  @Override
-  public void readData(ObjectDataInput arg0) throws IOException {
-    int len = arg0.readShort();
-    byte[] b = new byte[len];
-    arg0.readFully(b);
-    this.value = parser.parseFrom(b);
-  }
-
-  @Override
-  public void writeData(ObjectDataOutput arg0) throws IOException {
-    byte[] b = value.toByteArray();
-    if (b.length > Short.MAX_VALUE) throw new IOException("Unexpectedly long value.");
-    arg0.writeShort(b.length);
-    arg0.write(b);
-  }
-
-  protected T get() {
-    return value;
-  }
-
-  protected void set(T value) {
-    this.value = value;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
deleted file mode 100644
index 1de1c4e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.cache;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
-import com.hazelcast.core.HazelcastInstance;
-import com.hazelcast.core.IMap;
-
-public abstract class ProtoMap<K extends MessageLite, V extends MessageLite, HK extends ProtoBufWrap<K>, HV extends ProtoBufWrap<V>> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoMap.class);
-
-  private IMap<HK, HV> hzMap;
-  
-  public ProtoMap(HazelcastInstance instance, String mapName){
-    hzMap = instance.getMap(mapName);
-  }
-  
-  public V get(K key){
-    Preconditions.checkNotNull(key);
-    HK hk = getNewKey(key);
-    HV hv = hzMap.get(hk);
-    if(hv == null) return null;
-    return hv.get();
-  }
-  
-  public V put(K key, V value){
-    Preconditions.checkNotNull(key);
-    Preconditions.checkNotNull(value);
-    HV oldValue = hzMap.put(getNewKey(key), getNewValue(value));
-    return oldValue == null ? null : oldValue.get();
-  }
-  
-  public abstract HK getNewKey(K key);
-  public abstract HV getNewValue(V key);
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
new file mode 100644
index 0000000..1538a85
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
@@ -0,0 +1,65 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+
+public abstract class ProtoSerializable<V extends Message> extends AbstractStreamSerializable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoSerializable.class);
+
+  private Parser<V> protoParser;
+  private V obj;
+
+  ProtoSerializable(Parser<V> protoParser, V obj) {
+    super();
+    this.protoParser = protoParser;
+    this.obj = obj;
+  }
+
+  public V getObject(){
+    return obj;
+  }
+
+  @Override
+  public void readFromStream(InputStream input) throws IOException {
+    obj = protoParser.parseFrom(input);
+  }
+
+  @Override
+  public void writeToStream(OutputStream output) throws IOException {
+    obj.writeTo(output);
+  }
+
+  public static class PlanFragmentSerializable extends ProtoSerializable<PlanFragment>{
+    public PlanFragmentSerializable(PlanFragment obj) {super(PlanFragment.PARSER, obj);}
+    public PlanFragmentSerializable(){this(null);}
+  }
+  public static class FragmentHandleSerializable extends ProtoSerializable<FragmentHandle>{
+    public FragmentHandleSerializable(FragmentHandle obj) {super(FragmentHandle.PARSER, obj);}
+    public FragmentHandleSerializable(){this(null);}
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
index 073a8d5..bb3f527 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
@@ -19,18 +19,18 @@ package org.apache.drill.exec.cache;
 
 import io.netty.buffer.ByteBuf;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.List;
 
-import org.apache.drill.common.util.DataInputInputStream;
-import org.apache.drill.common.util.DataOutputOutputStream;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.DrillMetrics;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.proto.UserBitShared.SerializedField;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.VectorAccessible;
@@ -43,19 +43,17 @@ import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.hazelcast.nio.ObjectDataInput;
-import com.hazelcast.nio.ObjectDataOutput;
 
 /**
  * A wrapper around a VectorAccessible. Will serialize a VectorAccessible and write to an OutputStream, or can read
  * from an InputStream and construct a new VectorContainer.
  */
-public class VectorAccessibleSerializable implements DrillSerializable {
+public class VectorAccessibleSerializable extends AbstractStreamSerializable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorAccessibleSerializable.class);
   static final MetricRegistry metrics = DrillMetrics.getInstance();
   static final String WRITER_TIMER = MetricRegistry.name(VectorAccessibleSerializable.class, "writerTime");
 
-  private VectorAccessible va;
+  private VectorContainer va;
   private WritableBatch batch;
   private BufferAllocator allocator;
   private int recordCount = -1;
@@ -91,10 +89,6 @@ public class VectorAccessibleSerializable implements DrillSerializable {
     }
   }
 
-  @Override
-  public void readData(ObjectDataInput input) throws IOException {
-    readFromStream(DataInputInputStream.constructInputStream(input));
-  }
 
   /**
    * Reads from an InputStream and parses a RecordBatchDef. From this, we construct a SelectionVector2 if it exits
@@ -134,10 +128,6 @@ public class VectorAccessibleSerializable implements DrillSerializable {
     va = container;
   }
 
-  @Override
-  public void writeData(ObjectDataOutput output) throws IOException {
-    writeToStream(DataOutputOutputStream.constructOutputStream(output));
-  }
 
   public void writeToStreamAndRetain(OutputStream output) throws IOException {
     retain = true;
@@ -208,7 +198,7 @@ public class VectorAccessibleSerializable implements DrillSerializable {
     }
   }
 
-  public VectorAccessible get() {
+  public VectorContainer get() {
     return va;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HCVectorAccessibleSerializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HCVectorAccessibleSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HCVectorAccessibleSerializer.java
new file mode 100644
index 0000000..bac2323
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HCVectorAccessibleSerializer.java
@@ -0,0 +1,58 @@
+/**
+ * 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.drill.exec.cache.hazel;
+
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.StreamSerializer;
+
+import org.apache.drill.common.util.DataInputInputStream;
+import org.apache.drill.common.util.DataOutputOutputStream;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.server.DrillbitContext;
+
+import java.io.*;
+
+/**
+ * Wraps a DrillSerializable object. Objects of this class can be put in the HazelCast implementation of Distributed Cache
+ */
+public class HCVectorAccessibleSerializer implements StreamSerializer<VectorAccessibleSerializable> {
+
+  private BufferAllocator allocator;
+
+  public HCVectorAccessibleSerializer(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  public VectorAccessibleSerializable read(ObjectDataInput in) throws IOException {
+    VectorAccessibleSerializable va = new VectorAccessibleSerializable(allocator);
+    va.readFromStream(DataInputInputStream.constructInputStream(in));
+    return va;
+  }
+
+  public void write(ObjectDataOutput out, VectorAccessibleSerializable va) throws IOException {
+    va.writeToStream(DataOutputOutputStream.constructOutputStream(out));
+  }
+
+  public void destroy() {}
+
+  public int getTypeId() {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
new file mode 100644
index 0000000..06518b6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
@@ -0,0 +1,258 @@
+/**
+ * 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.drill.exec.cache.hazel;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.Counter;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.DistributedMap;
+import org.apache.drill.exec.cache.DistributedMultiMap;
+import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.cache.hazel.ProtoBufImpl.HWorkQueueStatus;
+import org.apache.drill.exec.cache.hazel.ProtoBufImpl.HandlePlan;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.Lists;
+import com.hazelcast.config.Config;
+import com.hazelcast.config.MapConfig;
+import com.hazelcast.config.SerializerConfig;
+import com.hazelcast.core.DuplicateInstanceNameException;
+import com.hazelcast.core.Hazelcast;
+import com.hazelcast.core.HazelcastInstance;
+import com.hazelcast.core.IAtomicLong;
+import com.hazelcast.core.IMap;
+import com.hazelcast.core.ITopic;
+import com.hazelcast.core.Message;
+import com.hazelcast.core.MessageListener;
+import com.hazelcast.nio.serialization.StreamSerializer;
+
+public class HazelCache implements DistributedCache {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HazelCache.class);
+
+  private final String instanceName;
+  private HazelcastInstance instance;
+  private ITopic<HWorkQueueStatus> workQueueLengths;
+  private HandlePlan fragments;
+  private Cache<WorkQueueStatus, Integer>  endpoints;
+  private BufferAllocator allocator;
+  private DrillConfig config;
+
+  public HazelCache(DrillConfig config, BufferAllocator allocator) {
+    this.instanceName = config.getString(ExecConstants.SERVICE_NAME);
+    this.allocator = allocator;
+    this.config = config;
+  }
+
+  private <T> void addSer(Config c, StreamSerializer<T> serializer, Class<T> clazz){
+    SerializerConfig sc = new SerializerConfig().setImplementation(serializer).setTypeClass(clazz);
+    c.getSerializationConfig().addSerializerConfig(sc);
+  }
+
+  @SuppressWarnings("rawtypes")
+  private <T> void addJSer(Config c, SerializationDefinition d){
+    SerializerConfig sc = new SerializerConfig().setImplementation(new JacksonAdvancedSerializer(d, config.getMapper())).setTypeClass(d.clazz);
+    c.getSerializationConfig().addSerializerConfig(sc);
+  }
+
+
+  private class Listener implements MessageListener<HWorkQueueStatus>{
+
+    @Override
+    public void onMessage(Message<HWorkQueueStatus> wrapped) {
+      logger.debug("Received new queue length message.");
+      endpoints.put(wrapped.getMessageObject().get(), 0);
+    }
+
+  }
+
+  public void run() {
+    Config c = new Config();
+    addSer(c, new HCVectorAccessibleSerializer(allocator), VectorAccessibleSerializable.class);
+    addJSer(c, SerializationDefinition.OPTION);
+    addJSer(c, SerializationDefinition.STORAGE_PLUGINS);
+
+    c.setInstanceName(instanceName);
+    c.getGroupConfig().setName(instanceName);
+    for (String s : DrillConfig.create().getStringList(ExecConstants.HAZELCAST_SUBNETS)) {
+      logger.debug("Adding interface: {}", s);
+      c.getNetworkConfig().getInterfaces().setEnabled(true).addInterface(s);
+    }
+
+    instance = getInstanceOrCreateNew(c);
+    workQueueLengths = instance.getTopic("queue-length");
+    fragments = new HandlePlan(instance);
+    endpoints = CacheBuilder.newBuilder().maximumSize(2000).build();
+    workQueueLengths.addMessageListener(new Listener());
+  }
+
+  private HazelcastInstance getInstanceOrCreateNew(Config c) {
+    for (HazelcastInstance instance : Hazelcast.getAllHazelcastInstances()){
+      if (instance.getName().equals(this.instanceName))
+        return instance;
+    }
+    try {
+    return Hazelcast.newHazelcastInstance(c);
+    } catch (DuplicateInstanceNameException e) {
+      return getInstanceOrCreateNew(c);
+    }
+  }
+
+//  @Override
+//  public void updateLocalQueueLength(int length) {
+//    workQueueLengths.publish(new HWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
+//        .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
+//  }
+//
+//  @Override
+//  public List<WorkQueueStatus> getQueueLengths() {
+//    return Lists.newArrayList(endpoints.asMap().keySet());
+//  }
+
+  @Override
+  public void close() throws IOException {
+    this.instance.getLifecycleService().shutdown();
+  }
+
+  @Override
+  public PlanFragment getFragment(FragmentHandle handle) {
+    return this.fragments.get(handle);
+  }
+
+  @Override
+  public void storeFragment(PlanFragment fragment) {
+    fragments.put(fragment.getHandle(), fragment);
+  }
+
+
+  @Override
+  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
+    com.hazelcast.core.MultiMap<String, V> mmap = this.instance.getMultiMap(clazz.toString());
+    return new HCDistributedMultiMapImpl<V>(mmap, clazz);
+  }
+
+  @Override
+  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
+    return getNamedMap(clazz.getName(), clazz);
+  }
+
+
+  @Override
+  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
+    IMap<String, V> imap = this.instance.getMap(name);
+    MapConfig myMapConfig = new MapConfig();
+    myMapConfig.setBackupCount(0);
+    myMapConfig.setReadBackupData(true);
+    instance.getConfig().getMapConfigs().put(clazz.toString(), myMapConfig);
+    return new HCDistributedMapImpl<V>(imap);
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return new HCCounterImpl(this.instance.getAtomicLong(name));
+  }
+
+
+
+  public static class HCDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
+    private final IMap<String, V> m;
+
+    public HCDistributedMapImpl(IMap<String, V> m) {
+      this.m = m;
+    }
+
+    public V get(String key) {
+      return m.get(key);
+    }
+
+    public void put(String key, V value) {
+      m.put(key, value);
+    }
+
+    public void putIfAbsent(String key, V value) {
+      m.putIfAbsent(key, value);
+    }
+
+    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeunit) {
+      m.putIfAbsent(key, value, ttl, timeunit);
+
+    }
+
+    @Override
+    public Iterator<Entry<String, V>> iterator() {
+      return m.entrySet().iterator();
+    }
+
+
+  }
+
+  public static class HCDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
+    private com.hazelcast.core.MultiMap<String, V> mmap;
+
+    public HCDistributedMultiMapImpl(com.hazelcast.core.MultiMap<String, V> mmap, Class<V> clazz) {
+      this.mmap = mmap;
+    }
+
+    public Collection<V> get(String key) {
+      List<V> list = Lists.newArrayList();
+      for (V v : mmap.get(key)) {
+        list.add(v);
+      }
+      return list;
+    }
+
+    @Override
+    public void put(String key, V value) {
+      mmap.put(key, value);
+    }
+  }
+
+  public static class HCCounterImpl implements Counter {
+    private IAtomicLong n;
+
+    public HCCounterImpl(IAtomicLong n) {
+      this.n = n;
+    }
+
+    public long get() {
+      return n.get();
+    }
+
+    public long incrementAndGet() {
+      return n.incrementAndGet();
+    }
+
+    public long decrementAndGet() {
+      return n.decrementAndGet();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufImpl.java
new file mode 100644
index 0000000..d992aa7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufImpl.java
@@ -0,0 +1,49 @@
+/**
+ * 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.drill.exec.cache.hazel;
+
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.BitControl.WorkQueueStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+import com.hazelcast.core.HazelcastInstance;
+
+public class ProtoBufImpl {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufImpl.class);
+  
+  public static class HWorkQueueStatus extends ProtoBufWrap<WorkQueueStatus>{
+    public HWorkQueueStatus() {super(WorkQueueStatus.PARSER);}
+    public HWorkQueueStatus(WorkQueueStatus value) {super(value, WorkQueueStatus.PARSER);}
+  }
+  
+  public static class HFragmentHandle extends ProtoBufWrap<FragmentHandle>{
+    public HFragmentHandle() {super(FragmentHandle.PARSER);}
+    public HFragmentHandle(FragmentHandle value) {super(value, FragmentHandle.PARSER);}
+  }
+  
+  public static class HPlanFragment extends ProtoBufWrap<PlanFragment>{
+    public HPlanFragment() {super(PlanFragment.PARSER);}
+    public HPlanFragment(PlanFragment value) {super(value, PlanFragment.PARSER);}
+  }
+  
+  public static class HandlePlan extends ProtoMap<FragmentHandle, PlanFragment, HFragmentHandle, HPlanFragment>{
+    public HandlePlan(HazelcastInstance instance) {super(instance, "plan-fragment-cache");}
+    public HFragmentHandle getNewKey(FragmentHandle key) {return new HFragmentHandle(key);}
+    public HPlanFragment getNewValue(PlanFragment value) {return new HPlanFragment(value);}
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufWrap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufWrap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufWrap.java
new file mode 100644
index 0000000..23a4e08
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoBufWrap.java
@@ -0,0 +1,67 @@
+/**
+ * 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.drill.exec.cache.hazel;
+
+import java.io.IOException;
+
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+import com.hazelcast.nio.serialization.DataSerializable;
+
+public abstract class ProtoBufWrap<T extends MessageLite> implements DataSerializable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufWrap.class);
+  
+  T value;
+  final Parser<T> parser;
+  
+  public ProtoBufWrap(Parser<T> parser){
+    this(null, parser);
+  }
+  
+  public ProtoBufWrap(T value, Parser<T> parser){
+    this.value = value;
+    this.parser = parser;
+  }
+  
+  @Override
+  public void readData(ObjectDataInput arg0) throws IOException {
+    int len = arg0.readShort();
+    byte[] b = new byte[len];
+    arg0.readFully(b);
+    this.value = parser.parseFrom(b);
+  }
+
+  @Override
+  public void writeData(ObjectDataOutput arg0) throws IOException {
+    byte[] b = value.toByteArray();
+    if (b.length > Short.MAX_VALUE) throw new IOException("Unexpectedly long value.");
+    arg0.writeShort(b.length);
+    arg0.write(b);
+  }
+
+  protected T get() {
+    return value;
+  }
+
+  protected void set(T value) {
+    this.value = value;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoMap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoMap.java
new file mode 100644
index 0000000..72d793a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/ProtoMap.java
@@ -0,0 +1,52 @@
+/**
+ * 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.drill.exec.cache.hazel;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+import com.hazelcast.core.HazelcastInstance;
+import com.hazelcast.core.IMap;
+
+public abstract class ProtoMap<K extends MessageLite, V extends MessageLite, HK extends ProtoBufWrap<K>, HV extends ProtoBufWrap<V>> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoMap.class);
+
+  private IMap<HK, HV> hzMap;
+  
+  public ProtoMap(HazelcastInstance instance, String mapName){
+    hzMap = instance.getMap(mapName);
+  }
+  
+  public V get(K key){
+    Preconditions.checkNotNull(key);
+    HK hk = getNewKey(key);
+    HV hv = hzMap.get(hk);
+    if(hv == null) return null;
+    return hv.get();
+  }
+  
+  public V put(K key, V value){
+    Preconditions.checkNotNull(key);
+    Preconditions.checkNotNull(value);
+    HV oldValue = hzMap.put(getNewKey(key), getNewValue(value));
+    return oldValue == null ? null : oldValue.get();
+  }
+  
+  public abstract HK getNewKey(K key);
+  public abstract HV getNewValue(V key);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
new file mode 100644
index 0000000..b26be7d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
@@ -0,0 +1,266 @@
+/**
+ * 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.drill.exec.cache.infinispan;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.Counter;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.DistributedMap;
+import org.apache.drill.exec.cache.DistributedMultiMap;
+import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.cache.ProtoSerializable.FragmentHandleSerializable;
+import org.apache.drill.exec.cache.ProtoSerializable.PlanFragmentSerializable;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.infinispan.Cache;
+import org.infinispan.atomic.Delta;
+import org.infinispan.atomic.DeltaAware;
+import org.infinispan.configuration.cache.CacheMode;
+import org.infinispan.configuration.cache.Configuration;
+import org.infinispan.configuration.cache.ConfigurationBuilder;
+import org.infinispan.configuration.global.GlobalConfiguration;
+import org.infinispan.configuration.global.GlobalConfigurationBuilder;
+import org.infinispan.manager.DefaultCacheManager;
+import org.infinispan.manager.EmbeddedCacheManager;
+import org.infinispan.remoting.transport.jgroups.JGroupsTransport;
+import org.jgroups.blocks.atomic.CounterService;
+import org.jgroups.fork.ForkChannel;
+import org.jgroups.protocols.COUNTER;
+import org.jgroups.protocols.FRAG2;
+import org.jgroups.stack.ProtocolStack;
+
+public class ICache implements DistributedCache{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ICache.class);
+
+  private EmbeddedCacheManager manager;
+  private ForkChannel cacheChannel;
+  private final CounterService counters;
+  private final Cache<FragmentHandleSerializable, PlanFragmentSerializable> fragments;
+
+  public ICache(DrillConfig config, BufferAllocator allocator) throws Exception {
+    String clusterName = config.getString(ExecConstants.SERVICE_NAME);
+    GlobalConfigurationBuilder gcb = new GlobalConfigurationBuilder();
+    gcb.transport() //
+    .defaultTransport().clusterName(clusterName).build();
+    gcb.serialization() //
+    .addAdvancedExternalizer(new VAAdvancedExternalizer(allocator));
+
+    GlobalConfiguration gc = gcb.build();
+    Configuration c = new ConfigurationBuilder() //
+      .clustering() //
+      .cacheMode(CacheMode.DIST_ASYNC) //
+      .storeAsBinary() //
+      .build();
+    this.manager = new DefaultCacheManager(gc, c);
+    JGroupsTransport transport = (JGroupsTransport) manager.getCache("first").getAdvancedCache().getRpcManager().getTransport();
+    this.cacheChannel = new ForkChannel(transport.getChannel(), "drill-stack", "drill-hijacker", true, ProtocolStack.ABOVE, FRAG2.class, new COUNTER());
+    this.fragments = manager.getCache(PlanFragment.class.getName());
+    this.counters = new CounterService(this.cacheChannel);
+  }
+
+  @Override
+  public void close() throws IOException {
+    manager.stop();
+  }
+
+  @Override
+  public void run() throws DrillbitStartupException {
+    try {
+      cacheChannel.connect("c1");
+    } catch (Exception e) {
+      throw new DrillbitStartupException("Failure while trying to set up JGroups.");
+    }
+  }
+
+  @Override
+  public PlanFragment getFragment(FragmentHandle handle) {
+    PlanFragmentSerializable pfs = fragments.get(new FragmentHandleSerializable(handle));
+    if(pfs == null) return null;
+    return pfs.getObject();
+  }
+
+  @Override
+  public void storeFragment(PlanFragment fragment) {
+    fragments.put(new FragmentHandleSerializable(fragment.getHandle()), new PlanFragmentSerializable(fragment));
+  }
+
+  @Override
+  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
+    Cache<String, DeltaList<V>> cache = manager.getCache(clazz.getName());
+    return new IMulti<V>(cache, clazz);
+  }
+
+  @Override
+  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
+    Cache<String, V> c = manager.getCache(clazz.getName());
+    return new IMap<V>(c);
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    return new JGroupsCounter(counters.getOrCreateCounter(name, 0));
+  }
+
+  private class JGroupsCounter implements Counter{
+    final org.jgroups.blocks.atomic.Counter inner;
+
+    public JGroupsCounter(org.jgroups.blocks.atomic.Counter inner) {
+      super();
+      this.inner = inner;
+    }
+
+    @Override
+    public long get() {
+      return inner.get();
+    }
+
+    @Override
+    public long incrementAndGet() {
+      return inner.incrementAndGet();
+    }
+
+    @Override
+    public long decrementAndGet() {
+      return inner.decrementAndGet();
+    }
+
+  }
+
+  private class IMap<V extends DrillSerializable> implements DistributedMap<V>{
+
+    private Cache<String, V> cache;
+
+
+    public IMap(Cache<String, V> cache) {
+      super();
+      this.cache = cache;
+    }
+
+    @Override
+    public V get(String key) {
+      return cache.get(key);
+    }
+
+    @Override
+    public void put(String key, V value) {
+      cache.put(key,  value);
+    }
+
+    @Override
+    public void putIfAbsent(String key, V value) {
+      cache.putIfAbsent(key,  value);
+    }
+
+    @Override
+    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit) {
+      cache.putIfAbsent(key, value, ttl, timeUnit);
+    }
+
+  }
+
+  private class IMulti<V extends DrillSerializable> implements DistributedMultiMap<V>{
+
+    private Cache<String, DeltaList<V>> cache;
+    private Class<V> clazz;
+
+    public IMulti(Cache<String, DeltaList<V>> cache, Class<V> clazz) {
+      super();
+      this.cache = cache;
+      this.clazz = clazz;
+    }
+
+    @Override
+    public Collection<V> get(String key) {
+      return cache.get(key);
+    }
+
+    @Override
+    public void put(String key, V value) {
+      cache.put(key, new DeltaList<V>(value));
+//      cache.getAdvancedCache().applyDelta(key, new DeltaList<V>(value), key);
+    }
+
+  }
+
+
+
+
+  private static class DeltaList<V extends DrillSerializable> extends LinkedList<V> implements DeltaAware, Delta{
+
+    /** The serialVersionUID */
+    private static final long serialVersionUID = 2176345973026460708L;
+
+    public DeltaList(Collection<? extends V> c) {
+       super(c);
+    }
+
+    public DeltaList(V obj) {
+       super();
+       add(obj);
+    }
+
+    @Override
+    public Delta delta() {
+       return new DeltaList<V>(this);
+    }
+
+    @Override
+    public void commit() {
+       this.clear();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public DeltaAware merge(DeltaAware d) {
+       List<V> other = null;
+       if (d != null && d instanceof DeltaList) {
+          other = (List<V>) d;
+          for (V e : this) {
+             other.add(e);
+          }
+          return (DeltaAware) other;
+       } else {
+          return this;
+       }
+    }
+ }
+
+
+//  public void run() {
+//    Config c = new Config();
+//    SerializerConfig sc = new SerializerConfig() //
+//      .setImplementation(new HCVectorAccessibleSerializer(allocator)) //
+//      .setTypeClass(VectorAccessibleSerializable.class);
+//    c.setInstanceName(instanceName);
+//    c.getSerializationConfig().addSerializerConfig(sc);
+//    instance = getInstanceOrCreateNew(c);
+//    workQueueLengths = instance.getTopic("queue-length");
+//    fragments = new HandlePlan(instance);
+//    endpoints = CacheBuilder.newBuilder().maximumSize(2000).build();
+//    workQueueLengths.addMessageListener(new Listener());
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/VAAdvancedExternalizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/VAAdvancedExternalizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/VAAdvancedExternalizer.java
new file mode 100644
index 0000000..5f54f74
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/VAAdvancedExternalizer.java
@@ -0,0 +1,72 @@
+/**
+ * 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.drill.exec.cache.infinispan;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Set;
+
+import org.apache.drill.exec.cache.CachedVectorContainer;
+import org.apache.drill.exec.cache.VectorAccessibleSerializable;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.infinispan.commons.marshall.AdvancedExternalizer;
+
+import com.google.common.collect.ImmutableSet;
+
+public class VAAdvancedExternalizer implements AdvancedExternalizer<CachedVectorContainer> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VAAdvancedExternalizer.class);
+
+  private BufferAllocator allocator;
+
+
+  public VAAdvancedExternalizer(BufferAllocator allocator) {
+    super();
+    this.allocator = allocator;
+  }
+
+  static final Set<Class<? extends CachedVectorContainer>> CLASSES = //
+      (Set<Class<? extends CachedVectorContainer>>) //
+      (Object) ImmutableSet.of(CachedVectorContainer.class);
+
+  @Override
+  public CachedVectorContainer readObject(ObjectInput in) throws IOException, ClassNotFoundException {
+    int length = in.readInt();
+    byte[] b = new byte[length];
+    in.read(b);
+    CachedVectorContainer va = new CachedVectorContainer(b, allocator);
+    return va;
+  }
+
+  @Override
+  public void writeObject(ObjectOutput out, CachedVectorContainer va) throws IOException {
+    out.writeInt(va.getData().length);
+    out.write(va.getData());
+  }
+
+  @Override
+  public Integer getId() {
+    // magic number for this class, assume drill uses 3001-3100.
+    return 3001;
+  }
+
+  @Override
+  public Set<Class<? extends CachedVectorContainer>> getTypeClasses() {
+    return CLASSES;
+  }
+}


[23/24] git commit: exclude jersey 1.8 from base client dependency list.

Posted by ja...@apache.org.
exclude jersey 1.8 from base client dependency list.


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

Branch: refs/heads/diagnostics2
Commit: e2740df1d1017831cb2d3ac72c8de30ade4f5caf
Parents: 3eb4f75
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 17:29:41 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 17:29:41 2014 -0700

----------------------------------------------------------------------
 contrib/storage-hbase/pom.xml | 32 ++++++++++++++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e2740df1/contrib/storage-hbase/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/pom.xml b/contrib/storage-hbase/pom.xml
index 700a91d..09b4f0f 100644
--- a/contrib/storage-hbase/pom.xml
+++ b/contrib/storage-hbase/pom.xml
@@ -112,6 +112,22 @@
               <groupId>org.slf4j</groupId>
             </exclusion>
             <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-server</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-json</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-client</artifactId>
+            </exclusion>
+            <exclusion>
               <artifactId>asm</artifactId>
               <groupId>asm</groupId>
             </exclusion>
@@ -179,6 +195,22 @@
               <groupId>org.slf4j</groupId>
             </exclusion>
             <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-server</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-json</artifactId>
+            </exclusion>            
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-client</artifactId>
+            </exclusion>
+            <exclusion>
               <artifactId>asm</artifactId>
               <groupId>asm</groupId>
             </exclusion>


[18/24] git commit: Fix for project column ordering is wrong.

Posted by ja...@apache.org.
Fix for project column ordering is wrong.


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

Branch: refs/heads/diagnostics2
Commit: f4c37bfe1b213d250f7feb73ca7ccf9b88e49255
Parents: b8cef6b
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue May 20 16:39:48 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue May 20 16:39:48 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/physical/ExchangePrel.java     |  1 +
 .../drill/exec/planner/physical/FilterPrel.java |  1 +
 .../exec/planner/physical/HashAggPrel.java      |  1 +
 .../drill/exec/planner/physical/JoinPrel.java   |  1 +
 .../planner/physical/JoinPrelRenameVisitor.java | 75 ------------------
 .../drill/exec/planner/physical/LimitPrel.java  |  1 +
 .../drill/exec/planner/physical/Prel.java       |  1 +
 .../exec/planner/physical/PrelVisitor.java      | 30 -------
 .../exec/planner/physical/ProjectPrel.java      |  1 +
 .../drill/exec/planner/physical/ScanPrel.java   |  1 +
 .../drill/exec/planner/physical/ScreenPrel.java |  5 +-
 .../physical/SelectionVectorPrelVisitor.java    | 66 ----------------
 .../drill/exec/planner/physical/SinglePrel.java |  1 +
 .../drill/exec/planner/physical/SortPrel.java   |  1 +
 .../exec/planner/physical/StreamAggPrel.java    |  1 +
 .../drill/exec/planner/physical/WriterPrel.java | 18 ++++-
 .../planner/physical/explain/PrelSequencer.java |  9 +--
 .../physical/visitor/BasePrelVisitor.java       | 53 +++++++++++++
 .../physical/visitor/FinalColumnReorderer.java  | 83 ++++++++++++++++++++
 .../physical/visitor/JoinPrelRenameVisitor.java | 73 +++++++++++++++++
 .../planner/physical/visitor/PrelVisitor.java   | 38 +++++++++
 .../visitor/SelectionVectorPrelVisitor.java     | 58 ++++++++++++++
 .../planner/sql/handlers/DefaultSqlHandler.java | 18 ++++-
 .../planner/sql/handlers/ExplainHandler.java    |  3 +-
 24 files changed, 353 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
index 21bf3b1..5a75adb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
@@ -18,6 +18,7 @@
 
 package org.apache.drill.exec.planner.physical;
 
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
index 9632911..1b13dcd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.planner.common.DrillFilterRelBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.metadata.RelMetadataQuery;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
index 6377e35..b33805c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
@@ -36,6 +36,7 @@ import org.apache.drill.exec.physical.config.HashAggregate;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.AggregateCall;
 import org.eigenbase.rel.AggregateRelBase;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
index 9e1f6fb..0db9a97 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
@@ -22,6 +22,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.JoinRelType;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
deleted file mode 100644
index ed4b8f8..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.drill.exec.planner.physical;
-
-import java.util.List;
-
-import org.eigenbase.rel.RelNode;
-
-import com.google.common.collect.Lists;
-
-public class JoinPrelRenameVisitor implements PrelVisitor<Prel, Void, RuntimeException>{
-
-  private static JoinPrelRenameVisitor INSTANCE = new JoinPrelRenameVisitor();
-
-  public static Prel insertRenameProject(Prel prel){
-    return prel.accept(INSTANCE, null);
-  }
-
-  @Override
-  public Prel visitExchange(ExchangePrel prel, Void value) throws RuntimeException {
-    return visitPrel(prel, value);
-  }
-
-  @Override
-  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
-    List<RelNode> children = Lists.newArrayList();
-    for(Prel child : prel){
-      child = child.accept(this, null);
-      children.add(child);
-    }
-
-    return (Prel) prel.copy(prel.getTraitSet(), children);
-
-  }
-
-  @Override
-  public Prel visitJoin(JoinPrel prel, Void value) throws RuntimeException {
-
-    List<RelNode> children = Lists.newArrayList();
-
-    for(Prel child : prel){
-      child = child.accept(this, null);
-      children.add(child);
-    }
-
-    final int leftCount = children.get(0).getRowType().getFieldCount();
-
-    List<RelNode> reNamedChildren = Lists.newArrayList();
-
-    RelNode left = prel.getJoinInput(0, children.get(0));
-    RelNode right = prel.getJoinInput(leftCount, children.get(1));
-
-    reNamedChildren.add(left);
-    reNamedChildren.add(right);
-
-    return (Prel) prel.copy(prel.getTraitSet(), reNamedChildren);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
index 794593a..376f334 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
@@ -30,6 +30,7 @@ import org.apache.drill.exec.planner.common.DrillLimitRelBase;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.InvalidRelException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
index a6c6b7c..1b4eb38 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillRelNode;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.relopt.Convention;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
deleted file mode 100644
index 93f95a8..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.physical;
-
-
-public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
-
-  public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
-
-  public RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
-
-  public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
index 70dca25..55f9f32 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -28,6 +28,7 @@ import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.planner.common.DrillProjectRelBase;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.ProjectRelBase;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index 8461e24..445ecd5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -30,6 +30,7 @@ import org.eigenbase.rel.AbstractRelNode;
 import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.store.StoragePlugin;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
index d02ed44..37823ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.planner.common.DrillScreenRelBase;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
@@ -40,7 +41,7 @@ public class ScreenPrel extends DrillScreenRelBase implements Prel {
   }
 
   @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  public ScreenPrel copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new ScreenPrel(getCluster(), traitSet, sole(inputs));
   }
 
@@ -63,7 +64,7 @@ public class ScreenPrel extends DrillScreenRelBase implements Prel {
 
   @Override
   public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitPrel(this, value);
+    return logicalVisitor.visitScreen(this, value);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
deleted file mode 100644
index 8f2ecd4..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.planner.physical;
-
-import java.util.List;
-
-import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
-import org.eigenbase.rel.RelNode;
-
-import com.google.common.collect.Lists;
-
-
-public class SelectionVectorPrelVisitor implements PrelVisitor<Prel, Void, RuntimeException>{
-
-  private static SelectionVectorPrelVisitor INSTANCE = new SelectionVectorPrelVisitor();
-
-  public static Prel addSelectionRemoversWhereNecessary(Prel prel){
-    return prel.accept(INSTANCE, null);
-  }
-
-  @Override
-  public Prel visitExchange(ExchangePrel prel, Void value) throws RuntimeException {
-    return visitPrel(prel, value);
-  }
-
-  @Override
-  public Prel visitJoin(JoinPrel prel, Void value) throws RuntimeException {
-    return visitPrel(prel, value);
-  }
-
-  @Override
-  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
-    SelectionVectorMode[] encodings = prel.getSupportedEncodings();
-    List<RelNode> children = Lists.newArrayList();
-    for(Prel child : prel){
-      child = child.accept(this, null);
-      children.add(convert(encodings, child));
-    }
-
-    return (Prel) prel.copy(prel.getTraitSet(), children);
-  }
-
-  private Prel convert(SelectionVectorMode[] encodings, Prel prel){
-    for(SelectionVectorMode m : encodings){
-      if(prel.getEncoding() == m) return prel;
-    }
-    return new SelectionVectorRemoverPrel(prel);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SinglePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SinglePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SinglePrel.java
index e47dc7f..c23e2a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SinglePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SinglePrel.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.planner.physical;
 import java.util.Collections;
 import java.util.Iterator;
 
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.SingleRel;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index fa5e900..464e1bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.config.SingleMergeExchange;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index a95d926..b35d1bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -37,6 +37,7 @@ import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.AggregateCall;
 import org.eigenbase.rel.AggregateRelBase;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
index e948125..a7f611c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
@@ -24,19 +24,31 @@ import java.util.List;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.common.DrillWriterRelBase;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.sql.type.SqlTypeName;
+
+import com.google.common.collect.ImmutableList;
+import com.google.hive12.common.collect.Lists;
 
 public class WriterPrel extends DrillWriterRelBase implements Prel {
 
+  private static final List<String> FIELD_NAMES = ImmutableList.of("Fragment", "Number of records written");
+
   public WriterPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, CreateTableEntry createTableEntry) {
     super(Prel.DRILL_PHYSICAL, cluster, traits, child, createTableEntry);
+    List<RelDataType> fields = Lists.newArrayList();
+    fields.add(cluster.getTypeFactory().createSqlType(SqlTypeName.VARCHAR, 255));
+    fields.add(cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT));
+    this.rowType = cluster.getTypeFactory().createStructType(fields, FIELD_NAMES);
   }
 
   @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  public WriterPrel copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new WriterPrel(getCluster(), traitSet, sole(inputs), getCreateTableEntry());
   }
 
@@ -49,6 +61,7 @@ public class WriterPrel extends DrillWriterRelBase implements Prel {
     return g;
   }
 
+
   @Override
   public Iterator<Prel> iterator() {
     return PrelUtil.iter(getChild());
@@ -56,7 +69,7 @@ public class WriterPrel extends DrillWriterRelBase implements Prel {
 
   @Override
   public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitPrel(this, value);
+    return logicalVisitor.visitWriter(this, value);
   }
 
   @Override
@@ -64,6 +77,7 @@ public class WriterPrel extends DrillWriterRelBase implements Prel {
     return SelectionVectorMode.DEFAULT;
   }
 
+
   @Override
   public SelectionVectorMode getEncoding() {
     return SelectionVectorMode.NONE;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
index 771546a..ebff287 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
@@ -27,14 +27,14 @@ import java.util.Queue;
 import org.apache.drill.exec.planner.physical.ExchangePrel;
 import org.apache.drill.exec.planner.physical.JoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.planner.physical.PrelVisitor;
+import org.apache.drill.exec.planner.physical.visitor.BasePrelVisitor;
 import org.eigenbase.rel.RelWriter;
 import org.eigenbase.sql.SqlExplainLevel;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, RuntimeException>{
+public class PrelSequencer extends BasePrelVisitor<Void, PrelSequencer.Frag, RuntimeException>{
 
   private List<Frag> frags = Lists.newLinkedList();
 
@@ -235,11 +235,6 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
   }
 
   @Override
-  public Void visitJoin(JoinPrel prel, Frag value) throws RuntimeException {
-    return visitPrel(prel, value);
-  }
-
-  @Override
   public Void visitPrel(Prel prel, Frag value) throws RuntimeException {
     for(Prel children : prel){
       children.accept(this, value);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
new file mode 100644
index 0000000..e3cfecc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/BasePrelVisitor.java
@@ -0,0 +1,53 @@
+/**
+ * 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.drill.exec.planner.physical.visitor;
+
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.JoinPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ScreenPrel;
+import org.apache.drill.exec.planner.physical.WriterPrel;
+
+public class BasePrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> implements PrelVisitor<RETURN, EXTRA, EXCEP> {
+
+  @Override
+  public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP {
+    return visitPrel(prel, value);
+  }
+
+  @Override
+  public RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP {
+    return visitPrel(prel, value);
+  }
+
+  @Override
+  public RETURN visitScreen(ScreenPrel prel, EXTRA value) throws EXCEP {
+    return visitPrel(prel, value);
+  }
+
+  @Override
+  public RETURN visitWriter(WriterPrel prel, EXTRA value) throws EXCEP {
+    return visitPrel(prel, value);
+  }
+
+  @Override
+  public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP {
+    throw new UnsupportedOperationException(String.format("No visit method defined for prel %s in visitor %s.", prel, this.getClass().getName()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
new file mode 100644
index 0000000..6ed3c1f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
@@ -0,0 +1,83 @@
+/**
+ * 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.drill.exec.planner.physical.visitor;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ProjectPrel;
+import org.apache.drill.exec.planner.physical.ScreenPrel;
+import org.apache.drill.exec.planner.physical.WriterPrel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexBuilder;
+import org.eigenbase.rex.RexNode;
+
+import com.google.common.collect.Lists;
+
+public class FinalColumnReorderer extends BasePrelVisitor<Prel, Void, RuntimeException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FinalColumnReorderer.class);
+
+  private static FinalColumnReorderer INSTANCE = new FinalColumnReorderer();
+
+  public static Prel addFinalColumnOrdering(Prel prel){
+    return prel.accept(INSTANCE, null);
+  }
+
+  @Override
+  public Prel visitScreen(ScreenPrel prel, Void value) throws RuntimeException {
+    Prel newChild = ((Prel) prel.getChild()).accept(this, value);
+    return prel.copy(prel.getTraitSet(), Collections.singletonList( (RelNode) addTrivialOrderedProjectPrel( newChild )));
+  }
+
+  private Prel addTrivialOrderedProjectPrel(Prel prel){
+    RelDataType t = prel.getRowType();
+    RexBuilder b = prel.getCluster().getRexBuilder();
+    List<RexNode> projections = Lists.newArrayList();
+    int projectCount = t.getFieldList().size();
+    for(int i =0; i < projectCount; i++){
+      projections.add(b.makeInputRef(prel, i));
+    }
+    return new ProjectPrel(prel.getCluster(), prel.getTraitSet(), prel, projections, prel.getRowType());
+  }
+
+  @Override
+  public Prel visitWriter(WriterPrel prel, Void value) throws RuntimeException {
+    Prel newChild = ((Prel) prel.getChild()).accept(this, null);
+    return prel.copy(prel.getTraitSet(), Collections.singletonList( (RelNode) addTrivialOrderedProjectPrel( newChild )));
+  }
+
+  @Override
+  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
+    List<RelNode> children = Lists.newArrayList();
+    boolean changed = false;
+    for(Prel p : prel){
+      Prel newP = p.accept(this, null);
+      if(newP != p) changed = true;
+      children.add(newP);
+    }
+    if(changed){
+      return (Prel) prel.copy(prel.getTraitSet(), children);
+    }else{
+      return prel;
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
new file mode 100644
index 0000000..3d38484
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/JoinPrelRenameVisitor.java
@@ -0,0 +1,73 @@
+/**
+ * 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.drill.exec.planner.physical.visitor;
+
+import java.util.List;
+
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.JoinPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.eigenbase.rel.RelNode;
+
+import com.google.common.collect.Lists;
+
+public class JoinPrelRenameVisitor extends BasePrelVisitor<Prel, Void, RuntimeException>{
+
+  private static JoinPrelRenameVisitor INSTANCE = new JoinPrelRenameVisitor();
+
+  public static Prel insertRenameProject(Prel prel){
+    return prel.accept(INSTANCE, null);
+  }
+
+  @Override
+  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
+    List<RelNode> children = Lists.newArrayList();
+    for(Prel child : prel){
+      child = child.accept(this, null);
+      children.add(child);
+    }
+
+    return (Prel) prel.copy(prel.getTraitSet(), children);
+
+  }
+
+  @Override
+  public Prel visitJoin(JoinPrel prel, Void value) throws RuntimeException {
+
+    List<RelNode> children = Lists.newArrayList();
+
+    for(Prel child : prel){
+      child = child.accept(this, null);
+      children.add(child);
+    }
+
+    final int leftCount = children.get(0).getRowType().getFieldCount();
+
+    List<RelNode> reNamedChildren = Lists.newArrayList();
+
+    RelNode left = prel.getJoinInput(0, children.get(0));
+    RelNode right = prel.getJoinInput(leftCount, children.get(1));
+
+    reNamedChildren.add(left);
+    reNamedChildren.add(right);
+
+    return (Prel) prel.copy(prel.getTraitSet(), reNamedChildren);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
new file mode 100644
index 0000000..b834464
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/PrelVisitor.java
@@ -0,0 +1,38 @@
+/**
+ * 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.drill.exec.planner.physical.visitor;
+
+import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.JoinPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.ScreenPrel;
+import org.apache.drill.exec.planner.physical.WriterPrel;
+
+
+public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
+
+  public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
+  public RETURN visitScreen(ScreenPrel prel, EXTRA value) throws EXCEP;
+  public RETURN visitWriter(WriterPrel prel, EXTRA value) throws EXCEP;
+
+  public RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
+
+  public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SelectionVectorPrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SelectionVectorPrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SelectionVectorPrelVisitor.java
new file mode 100644
index 0000000..8d0b155
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/SelectionVectorPrelVisitor.java
@@ -0,0 +1,58 @@
+/**
+ * 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.drill.exec.planner.physical.visitor;
+
+import java.util.List;
+
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.SelectionVectorRemoverPrel;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelNode;
+
+import com.google.common.collect.Lists;
+
+
+public class SelectionVectorPrelVisitor extends BasePrelVisitor<Prel, Void, RuntimeException>{
+
+  private static SelectionVectorPrelVisitor INSTANCE = new SelectionVectorPrelVisitor();
+
+  public static Prel addSelectionRemoversWhereNecessary(Prel prel){
+    return prel.accept(INSTANCE, null);
+  }
+
+  @Override
+  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
+    SelectionVectorMode[] encodings = prel.getSupportedEncodings();
+    List<RelNode> children = Lists.newArrayList();
+    for(Prel child : prel){
+      child = child.accept(this, null);
+      children.add(convert(encodings, child));
+    }
+
+    return (Prel) prel.copy(prel.getTraitSet(), children);
+  }
+
+  private Prel convert(SelectionVectorMode[] encodings, Prel prel){
+    for(SelectionVectorMode m : encodings){
+      if(prel.getEncoding() == m) return prel;
+    }
+    return new SelectionVectorRemoverPrel(prel);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index e8bd837..29ed1ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -38,11 +38,12 @@ import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
 import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
-import org.apache.drill.exec.planner.physical.JoinPrelRenameVisitor;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.planner.physical.SelectionVectorPrelVisitor;
 import org.apache.drill.exec.planner.physical.explain.PrelSequencer;
+import org.apache.drill.exec.planner.physical.visitor.FinalColumnReorderer;
+import org.apache.drill.exec.planner.physical.visitor.JoinPrelRenameVisitor;
+import org.apache.drill.exec.planner.physical.visitor.SelectionVectorPrelVisitor;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
 import org.apache.drill.exec.util.Pointer;
 import org.eigenbase.rel.RelNode;
@@ -80,9 +81,13 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
   }
 
   protected void log(String name, Prel node) {
-    if(textPlan != null) textPlan.value = PrelSequencer.printWithIds(node, SqlExplainLevel.ALL_ATTRIBUTES);
+    String plan = PrelSequencer.printWithIds(node, SqlExplainLevel.ALL_ATTRIBUTES);;
+    if(textPlan != null){
+      textPlan.value = plan;
+    }
+
     if (logger.isDebugEnabled()) {
-      logger.debug(name + " : \n" + textPlan.value);
+      logger.debug(name + " : \n" + plan);
     }
   }
 
@@ -138,6 +143,11 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     // In such case, we have to insert Project to rename the conflicting names.
     phyRelNode = JoinPrelRenameVisitor.insertRenameProject(phyRelNode);
 
+    // Since our operators work via names rather than indices, we have to make to reorder any output
+    // before we return data to the user as we may have accindentally shuffled things.  This adds
+    // a trivial project to reorder columns prior to output.
+    phyRelNode = FinalColumnReorderer.addFinalColumnOrdering(phyRelNode);
+
     // the last thing we do is add any required selection vector removers given the supported encodings of each
     // operator. This will ultimately move to a new trait but we're managing here for now to avoid introducing new
     // issues in planning before the next release

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f4c37bfe/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
index 86ce6c5..63db153 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.explain.PrelSequencer;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
@@ -111,7 +112,7 @@ public class ExplainHandler extends DefaultSqlHandler{
     public String json;
 
     public PhysicalExplain(RelNode node, PhysicalPlan plan, SqlExplainLevel level, QueryContext context){
-      this.text = RelOptUtil.toString(node, level);
+      this.text = PrelSequencer.printWithIds((Prel) node, level);
       this.json = plan.unparse(context.getConfig().getMapper().writer());
     }
   }


[04/24] git commit: Use PrelVisitor to add Project when Join has conflicting names from inputs. Move common code of MergeJoin, HashJoin to base class.

Posted by ja...@apache.org.
Use PrelVisitor to add Project when Join has conflicting names from inputs. Move common code of MergeJoin, HashJoin to base class.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/73d3bd06
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/73d3bd06
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/73d3bd06

Branch: refs/heads/diagnostics2
Commit: 73d3bd063e2286cc1a71ebf7ef71ae6171bd592e
Parents: ebf3d34
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Fri May 16 16:02:53 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:11:22 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/logical/DrillProjectRel.java   |  6 +-
 .../exec/planner/physical/ExchangePrel.java     | 18 ++++
 .../exec/planner/physical/HashJoinPrel.java     | 64 +------------
 .../drill/exec/planner/physical/JoinPrel.java   | 98 ++++++++++++++++++++
 .../planner/physical/JoinPrelRenameVisitor.java | 75 +++++++++++++++
 .../exec/planner/physical/MergeJoinPrel.java    | 67 +------------
 .../exec/planner/physical/PrelVisitor.java      |  3 +
 .../physical/SelectionVectorPrelVisitor.java    |  5 +
 .../physical/SelectionVectorRemoverPrel.java    | 14 +++
 .../planner/physical/explain/PrelSequencer.java |  6 ++
 .../planner/sql/handlers/DefaultSqlHandler.java |  5 +
 .../apache/drill/exec/TestOpSerialization.java  | 18 ++++
 12 files changed, 250 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
index aa75f52..c3eb3fb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.common.logical.data.LogicalOperator;
@@ -28,10 +27,7 @@ import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.ProjectRelBase;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
 import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.reltype.RelDataTypeField;
@@ -78,5 +74,5 @@ public class DrillProjectRel extends DrillProjectRelBase implements DrillRel {
     }
     return new DrillProjectRel(context.getCluster(), context.getLogicalTraits(), input, exps, new RelRecordType(fields));
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
index 360fa80..21bf3b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ExchangePrel.java
@@ -1,3 +1,21 @@
+/**
+ * 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.drill.exec.planner.physical;
 
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
index 87da31e..b03ef62 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
@@ -18,16 +18,12 @@
 package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.JoinCondition;
-import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -46,7 +42,7 @@ import org.eigenbase.util.Pair;
 
 import com.beust.jcommander.internal.Lists;
 
-public class HashJoinPrel  extends DrillJoinRelBase implements Prel {
+public class HashJoinPrel  extends JoinPrel {
 
   public HashJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType) throws InvalidRelException {
@@ -94,8 +90,8 @@ public class HashJoinPrel  extends DrillJoinRelBase implements Prel {
     final List<String> leftFields = fields.subList(0, leftCount);
     final List<String> rightFields = fields.subList(leftCount, fields.size());
 
-    PhysicalOperator leftPop = implementInput(creator, 0, left);
-    PhysicalOperator rightPop = implementInput(creator, leftCount, right);
+    PhysicalOperator leftPop = ((Prel)left).getPhysicalOperator(creator);
+    PhysicalOperator rightPop = ((Prel)right).getPhysicalOperator(creator);
 
     JoinRelType jtype = this.getJoinType();
 
@@ -111,60 +107,6 @@ public class HashJoinPrel  extends DrillJoinRelBase implements Prel {
     return hjoin;
   }
 
-  public List<Integer> getLeftKeys() {
-    return this.leftKeys;
-  }
-
-  public List<Integer> getRightKeys() {
-    return this.rightKeys;
-  }
-
-  /**
-   * Check to make sure that the fields of the inputs are the same as the output field names.  If not, insert a project renaming them.
-   * @param implementor
-   * @param i
-   * @param offset
-   * @param input
-   * @return
-   */
-  private PhysicalOperator implementInput(PhysicalPlanCreator creator, int offset, RelNode input) throws IOException {
-    final PhysicalOperator inputOp = ((Prel) input).getPhysicalOperator(creator);
-    assert uniqueFieldNames(input.getRowType());
-    final List<String> fields = getRowType().getFieldNames();
-    final List<String> inputFields = input.getRowType().getFieldNames();
-    final List<String> outputFields = fields.subList(offset, offset + inputFields.size());
-    if (!outputFields.equals(inputFields)) {
-      // Ensure that input field names are the same as output field names.
-      // If there are duplicate field names on left and right, fields will get
-      // lost.
-      return rename(creator, inputOp, inputFields, outputFields);
-    } else {
-      return inputOp;
-    }
-  }
-
-  private PhysicalOperator rename(PhysicalPlanCreator creator, PhysicalOperator inputOp, List<String> inputFields, List<String> outputFields) {
-    List<NamedExpression> exprs = Lists.newArrayList();
-
-    for (Pair<String, String> pair : Pair.zip(inputFields, outputFields)) {
-      exprs.add(new NamedExpression(new FieldReference(pair.left), new FieldReference(pair.right)));
-    }
-
-    Project proj = new Project(exprs, inputOp);
-
-    return proj;
-  }
-
-  @Override
-  public Iterator<Prel> iterator() {
-    return PrelUtil.iter(getLeft(), getRight());
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitPrel(this, value);
-  }
-
   @Override
   public SelectionVectorMode[] getSupportedEncodings() {
     return SelectionVectorMode.DEFAULT;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
new file mode 100644
index 0000000..9e1f6fb
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrel.java
@@ -0,0 +1,98 @@
+/**
+ * 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.drill.exec.planner.physical;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinRelType;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.rex.RexUtil;
+
+import com.google.common.collect.Lists;
+
+/**
+ *
+ * Base class for MergeJoinPrel and HashJoinPrel
+ *
+ */
+public abstract class JoinPrel extends DrillJoinRelBase implements Prel{
+
+  public JoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
+      JoinRelType joinType) throws InvalidRelException{
+    super(cluster, traits, left, right, condition, joinType);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitJoin(this, value);
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return PrelUtil.iter(getLeft(), getRight());
+  }
+
+  /**
+   * Check to make sure that the fields of the inputs are the same as the output field names.  If not, insert a project renaming them.
+   * @param implementor
+   * @param i
+   * @param offset
+   * @param input
+   * @return
+   */
+  public RelNode getJoinInput(int offset, RelNode input) {
+    assert uniqueFieldNames(input.getRowType());
+    final List<String> fields = getRowType().getFieldNames();
+    final List<String> inputFields = input.getRowType().getFieldNames();
+    final List<String> outputFields = fields.subList(offset, offset + inputFields.size());
+    if (!outputFields.equals(inputFields)) {
+      // Ensure that input field names are the same as output field names.
+      // If there are duplicate field names on left and right, fields will get
+      // lost.
+      // In such case, we need insert a rename Project on top of the input.
+      return rename(input, input.getRowType().getFieldList(), outputFields);
+    } else {
+      return input;
+    }
+  }
+
+  private RelNode rename(RelNode input, List<RelDataTypeField> inputFields, List<String> outputFieldNames) {
+    List<RexNode> exprs = Lists.newArrayList();
+
+    for (RelDataTypeField field : inputFields) {
+      RexNode expr = input.getCluster().getRexBuilder().makeInputRef(field.getType(), field.getIndex());
+      exprs.add(expr);
+    }
+
+    RelDataType rowType = RexUtil.createStructType(input.getCluster().getTypeFactory(), exprs, outputFieldNames);
+
+    ProjectPrel proj = new ProjectPrel(input.getCluster(), input.getTraitSet(), input, exprs, rowType);
+
+    return proj;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
new file mode 100644
index 0000000..ed4b8f8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/JoinPrelRenameVisitor.java
@@ -0,0 +1,75 @@
+/**
+ * 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.drill.exec.planner.physical;
+
+import java.util.List;
+
+import org.eigenbase.rel.RelNode;
+
+import com.google.common.collect.Lists;
+
+public class JoinPrelRenameVisitor implements PrelVisitor<Prel, Void, RuntimeException>{
+
+  private static JoinPrelRenameVisitor INSTANCE = new JoinPrelRenameVisitor();
+
+  public static Prel insertRenameProject(Prel prel){
+    return prel.accept(INSTANCE, null);
+  }
+
+  @Override
+  public Prel visitExchange(ExchangePrel prel, Void value) throws RuntimeException {
+    return visitPrel(prel, value);
+  }
+
+  @Override
+  public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
+    List<RelNode> children = Lists.newArrayList();
+    for(Prel child : prel){
+      child = child.accept(this, null);
+      children.add(child);
+    }
+
+    return (Prel) prel.copy(prel.getTraitSet(), children);
+
+  }
+
+  @Override
+  public Prel visitJoin(JoinPrel prel, Void value) throws RuntimeException {
+
+    List<RelNode> children = Lists.newArrayList();
+
+    for(Prel child : prel){
+      child = child.accept(this, null);
+      children.add(child);
+    }
+
+    final int leftCount = children.get(0).getRowType().getFieldCount();
+
+    List<RelNode> reNamedChildren = Lists.newArrayList();
+
+    RelNode left = prel.getJoinInput(0, children.get(0));
+    RelNode right = prel.getJoinInput(leftCount, children.get(1));
+
+    reNamedChildren.add(left);
+    reNamedChildren.add(right);
+
+    return (Prel) prel.copy(prel.getTraitSet(), reNamedChildren);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index 400c6a8..1987e99 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -18,17 +18,12 @@
 package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.JoinCondition;
-import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.MergeJoinPOP;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.physical.config.SelectionVectorRemover;
-import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
 import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -47,9 +42,7 @@ import org.eigenbase.util.Pair;
 
 import com.beust.jcommander.internal.Lists;
 
-public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
-
-  //private final JoinCondition[] joinConditions; // Drill's representation of join conditions
+public class MergeJoinPrel  extends JoinPrel {
 
   /** Creates a MergeJoiPrel. */
   public MergeJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
@@ -94,12 +87,13 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     final List<String> fields = getRowType().getFieldNames();
     assert isUnique(fields);
+
     final int leftCount = left.getRowType().getFieldCount();
     final List<String> leftFields = fields.subList(0, leftCount);
     final List<String> rightFields = fields.subList(leftCount, fields.size());
 
-    PhysicalOperator leftPop = implementInput(creator, 0, left);
-    PhysicalOperator rightPop = implementInput(creator, leftCount, right);
+    PhysicalOperator leftPop = ((Prel)left).getPhysicalOperator(creator);
+    PhysicalOperator rightPop = ((Prel)right).getPhysicalOperator(creator);
 
     JoinRelType jtype = this.getJoinType();
 
@@ -113,60 +107,7 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     mjoin.setOperatorId(creator.getOperatorId(this));
 
     return mjoin;
-  }
-
-  public List<Integer> getLeftKeys() {
-    return this.leftKeys;
-  }
-
-  public List<Integer> getRightKeys() {
-    return this.rightKeys;
-  }
-
-  /**
-   * Check to make sure that the fields of the inputs are the same as the output field names.  If not, insert a project renaming them.
-   * @param implementor
-   * @param i
-   * @param offset
-   * @param input
-   * @return
-   */
-  private PhysicalOperator implementInput(PhysicalPlanCreator creator, int offset, RelNode input) throws IOException {
-    final PhysicalOperator inputOp = ((Prel) input).getPhysicalOperator(creator);
-    assert uniqueFieldNames(input.getRowType());
-    final List<String> fields = getRowType().getFieldNames();
-    final List<String> inputFields = input.getRowType().getFieldNames();
-    final List<String> outputFields = fields.subList(offset, offset + inputFields.size());
-    if (!outputFields.equals(inputFields)) {
-      // Ensure that input field names are the same as output field names.
-      // If there are duplicate field names on left and right, fields will get
-      // lost.
-      return rename(creator, inputOp, inputFields, outputFields);
-    } else {
-      return inputOp;
-    }
-  }
 
-  private PhysicalOperator rename(PhysicalPlanCreator creator, PhysicalOperator inputOp, List<String> inputFields, List<String> outputFields) {
-    List<NamedExpression> exprs = Lists.newArrayList();
-
-    for (Pair<String, String> pair : Pair.zip(inputFields, outputFields)) {
-      exprs.add(new NamedExpression(new FieldReference(pair.left), new FieldReference(pair.right)));
-    }
-
-    Project proj = new Project(exprs, inputOp);
-
-    return proj;
-  }
-
-  @Override
-  public Iterator<Prel> iterator() {
-    return PrelUtil.iter(getLeft(), getRight());
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
-    return logicalVisitor.visitPrel(this, value);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
index 862b133..93f95a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelVisitor.java
@@ -22,6 +22,9 @@ public interface PrelVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PrelVisitor.class);
 
   public RETURN visitExchange(ExchangePrel prel, EXTRA value) throws EXCEP;
+
+  public RETURN visitJoin(JoinPrel prel, EXTRA value) throws EXCEP;
+
   public RETURN visitPrel(Prel prel, EXTRA value) throws EXCEP;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
index 224ab76..8f2ecd4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorPrelVisitor.java
@@ -39,6 +39,11 @@ public class SelectionVectorPrelVisitor implements PrelVisitor<Prel, Void, Runti
   }
 
   @Override
+  public Prel visitJoin(JoinPrel prel, Void value) throws RuntimeException {
+    return visitPrel(prel, value);
+  }
+
+  @Override
   public Prel visitPrel(Prel prel, Void value) throws RuntimeException {
     SelectionVectorMode[] encodings = prel.getSupportedEncodings();
     List<RelNode> children = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
index fd07749..4332188 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
@@ -18,10 +18,16 @@
 package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexNode;
 
 public class SelectionVectorRemoverPrel extends SinglePrel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorRemoverPrel.class);
@@ -30,6 +36,14 @@ public class SelectionVectorRemoverPrel extends SinglePrel{
     super(child.getCluster(), child.getTraitSet(), child);
   }
 
+  public SelectionVectorRemoverPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new SelectionVectorRemoverPrel(this.getCluster(), traitSet, inputs.get(0));
+  }
+
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     SelectionVectorRemover r =  new SelectionVectorRemover( ((Prel)getChild()).getPhysicalOperator(creator));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
index 2ab6c74..771546a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.Queue;
 
 import org.apache.drill.exec.planner.physical.ExchangePrel;
+import org.apache.drill.exec.planner.physical.JoinPrel;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.PrelVisitor;
 import org.eigenbase.rel.RelWriter;
@@ -234,6 +235,11 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
   }
 
   @Override
+  public Void visitJoin(JoinPrel prel, Frag value) throws RuntimeException {
+    return visitPrel(prel, value);
+  }
+
+  @Override
   public Void visitPrel(Prel prel, Frag value) throws RuntimeException {
     for(Prel children : prel){
       children.accept(this, value);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index b06432a..36ec0e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -38,6 +38,7 @@ import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
 import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.JoinPrelRenameVisitor;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.SelectionVectorPrelVisitor;
@@ -125,6 +126,10 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     RelTraitSet traits = drel.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
     Prel phyRelNode = (Prel) planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
 
+    // Join might cause naming conflicts from its left and right child.
+    // In such case, we have to insert Project to rename the conflicting names.
+    phyRelNode = JoinPrelRenameVisitor.insertRenameProject(phyRelNode);
+
     // the last thing we do is add any required selection vector removers given the supported encodings of each
     // operator. This will ultimately move to a new trait but we're managing here for now to avoid introducing new
     // issues in planning before the next release

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/73d3bd06/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
index ad1d6b6..906361a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
@@ -1,3 +1,21 @@
+/**
+ * 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.drill.exec;
 
 import static org.junit.Assert.assertEquals;


[08/24] status changes

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
index 88bada5..5b61a82 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/StreamingAggBatch.java
@@ -79,64 +79,69 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
 
   @Override
   public IterOutcome next() {
-
-    // this is only called on the first batch. Beyond this, the aggregator manages batches.
-    if (aggregator == null) {
-      IterOutcome outcome = incoming.next();
-      logger.debug("Next outcome of {}", outcome);
-      switch (outcome) {
-      case NONE:
-      case NOT_YET:
-      case STOP:
-        return outcome;
-      case OK_NEW_SCHEMA:
-        if (!createAggregator()){
-          done = true;
-          return IterOutcome.STOP;
+    stats.startProcessing();
+    try{
+      // this is only called on the first batch. Beyond this, the aggregator manages batches.
+      if (aggregator == null) {
+        IterOutcome outcome = next(incoming);
+        logger.debug("Next outcome of {}", outcome);
+        switch (outcome) {
+        case NONE:
+        case NOT_YET:
+        case STOP:
+          return outcome;
+        case OK_NEW_SCHEMA:
+          if (!createAggregator()){
+            done = true;
+            return IterOutcome.STOP;
+          }
+          break;
+        case OK:
+          throw new IllegalStateException("You should never get a first batch without a new schema");
+        default:
+          throw new IllegalStateException(String.format("unknown outcome %s", outcome));
         }
-        break;
-      case OK:
-        throw new IllegalStateException("You should never get a first batch without a new schema");
-      default:
-        throw new IllegalStateException(String.format("unknown outcome %s", outcome));
       }
-    }
 
-    while(true){
-      AggOutcome out = aggregator.doWork();
-      logger.debug("Aggregator response {}, records {}", out, aggregator.getOutputCount());
-      switch(out){
-      case CLEANUP_AND_RETURN:
-        container.clear();
-        done = true;
-        return aggregator.getOutcome();
-      case RETURN_OUTCOME:
-        return aggregator.getOutcome();
-      case UPDATE_AGGREGATOR:
-        aggregator = null;
-        if(!createAggregator()){
-          return IterOutcome.STOP;
+      while(true){
+        AggOutcome out = aggregator.doWork();
+        logger.debug("Aggregator response {}, records {}", out, aggregator.getOutputCount());
+        switch(out){
+        case CLEANUP_AND_RETURN:
+          container.clear();
+          done = true;
+          return aggregator.getOutcome();
+        case RETURN_OUTCOME:
+          return aggregator.getOutcome();
+        case UPDATE_AGGREGATOR:
+          aggregator = null;
+          if(!createAggregator()){
+            return IterOutcome.STOP;
+          }
+          continue;
+        default:
+          throw new IllegalStateException(String.format("Unknown state %s.", out));
         }
-        continue;
-      default:
-        throw new IllegalStateException(String.format("Unknown state %s.", out));
       }
+    }finally{
+      stats.stopProcessing();
     }
-    
   }
 
-  
-  
+
+
   /**
    * Creates a new Aggregator based on the current schema. If setup fails, this method is responsible for cleaning up
    * and informing the context of the failure state, as well is informing the upstream operators.
-   * 
+   *
    * @return true if the aggregator was setup successfully. false if there was a failure.
    */
   private boolean createAggregator() {
     logger.debug("Creating new aggregator.");
     try{
+      stats.startSetup();
       this.aggregator = createAggregatorInternal();
+      stats.stopSetup();
       return true;
     }catch(SchemaChangeException | ClassTransformationException | IOException ex){
       context.fail(ex);
@@ -153,13 +158,13 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     ClassGenerator<StreamingAggregator> cg = CodeGenerator.getRoot(StreamingAggTemplate.TEMPLATE_DEFINITION, context.getFunctionRegistry());
     container.clear();
     List<VectorAllocator> allocators = Lists.newArrayList();
-    
+
     LogicalExpression[] keyExprs = new LogicalExpression[popConfig.getKeys().length];
     LogicalExpression[] valueExprs = new LogicalExpression[popConfig.getExprs().length];
     TypedFieldId[] keyOutputIds = new TypedFieldId[popConfig.getKeys().length];
-    
+
     ErrorCollector collector = new ErrorCollectorImpl();
-    
+
     for(int i =0; i < keyExprs.length; i++){
       NamedExpression ne = popConfig.getKeys()[i];
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector,context.getFunctionRegistry() );
@@ -170,38 +175,38 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       allocators.add(VectorAllocator.getAllocator(vector, 50));
       keyOutputIds[i] = container.add(vector);
     }
-    
+
     for(int i =0; i < valueExprs.length; i++){
       NamedExpression ne = popConfig.getExprs()[i];
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry());
       if(expr == null) continue;
-      
+
       final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType());
       ValueVector vector = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       allocators.add(VectorAllocator.getAllocator(vector, 50));
       TypedFieldId id = container.add(vector);
       valueExprs[i] = new ValueVectorWriteExpression(id, expr, true);
     }
-    
+
     if(collector.hasErrors()) throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
-    
+
     setupIsSame(cg, keyExprs);
     setupIsSameApart(cg, keyExprs);
     addRecordValues(cg, valueExprs);
     outputRecordKeys(cg, keyOutputIds, keyExprs);
     outputRecordKeysPrev(cg, keyOutputIds, keyExprs);
-    
+
     cg.getBlock("resetValues")._return(JExpr.TRUE);
     getIndex(cg);
-    
+
     container.buildSchema(SelectionVectorMode.NONE);
     StreamingAggregator agg = context.getImplementationClass(cg);
     agg.setup(context, incoming, this, allocators.toArray(new VectorAllocator[allocators.size()]));
     return agg;
   }
-  
-  
-  
+
+
+
   private final GeneratorMapping IS_SAME = GeneratorMapping.create("setupInterior", "isSame", null, null);
   private final MappingSet IS_SAME_I1 = new MappingSet("index1", null, IS_SAME, IS_SAME);
   private final MappingSet IS_SAME_I2 = new MappingSet("index2", null, IS_SAME, IS_SAME);
@@ -214,19 +219,19 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       HoldingContainer first = cg.addExpr(expr, false);
       cg.setMappingSet(IS_SAME_I2);
       HoldingContainer second = cg.addExpr(expr, false);
-      
+
       LogicalExpression fh = FunctionGenerationHelper.getComparator(first, second, context.getFunctionRegistry());
       HoldingContainer out = cg.addExpr(fh, false);
       cg.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)))._then()._return(JExpr.FALSE);
     }
     cg.getEvalBlock()._return(JExpr.TRUE);
   }
-  
+
   private final GeneratorMapping IS_SAME_PREV_INTERNAL_BATCH_READ = GeneratorMapping.create("isSamePrev", "isSamePrev", null, null); // the internal batch changes each time so we need to redo setup.
   private final GeneratorMapping IS_SAME_PREV = GeneratorMapping.create("setupInterior", "isSamePrev", null, null);
   private final MappingSet ISA_B1 = new MappingSet("b1Index", null, "b1", null, IS_SAME_PREV_INTERNAL_BATCH_READ, IS_SAME_PREV_INTERNAL_BATCH_READ);
   private final MappingSet ISA_B2 = new MappingSet("b2Index", null, "incoming", null, IS_SAME_PREV, IS_SAME_PREV);
-  
+
   private void setupIsSameApart(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] keyExprs){
     cg.setMappingSet(ISA_B1);
     for(LogicalExpression expr : keyExprs){
@@ -242,11 +247,11 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     }
     cg.getEvalBlock()._return(JExpr.TRUE);
   }
-  
+
   private final GeneratorMapping EVAL_INSIDE = GeneratorMapping.create("setupInterior", "addRecord", null, null);
   private final GeneratorMapping EVAL_OUTSIDE = GeneratorMapping.create("setupInterior", "outputRecordValues", "resetValues", "cleanup");
   private final MappingSet EVAL = new MappingSet("index", "outIndex", "incoming", "outgoing", EVAL_INSIDE, EVAL_OUTSIDE, EVAL_INSIDE);
-  
+
   private void addRecordValues(ClassGenerator<StreamingAggregator> cg, LogicalExpression[] valueExprs){
     cg.setMappingSet(EVAL);
     for(LogicalExpression ex : valueExprs){
@@ -255,9 +260,9 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     }
     cg.getBlock(BlockType.EVAL)._return(JExpr.TRUE);
   }
-  
+
   private final MappingSet RECORD_KEYS = new MappingSet(GeneratorMapping.create("setupInterior", "outputRecordKeys", null, null));
-  
+
   private void outputRecordKeys(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs){
     cg.setMappingSet(RECORD_KEYS);
     for(int i =0; i < keyExprs.length; i++){
@@ -266,13 +271,13 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     }
     cg.getBlock(BlockType.EVAL)._return(JExpr.TRUE);
   }
-  
+
   private final GeneratorMapping PREVIOUS_KEYS_OUT = GeneratorMapping.create("setupInterior", "outputRecordKeysPrev", null, null);
   private final MappingSet RECORD_KEYS_PREV_OUT = new MappingSet("previousIndex", "outIndex", "previous", "outgoing", PREVIOUS_KEYS_OUT, PREVIOUS_KEYS_OUT);
 
   private final GeneratorMapping PREVIOUS_KEYS = GeneratorMapping.create("outputRecordKeysPrev", "outputRecordKeysPrev", null, null);
   private final MappingSet RECORD_KEYS_PREV = new MappingSet("previousIndex", "outIndex", "previous", null, PREVIOUS_KEYS, PREVIOUS_KEYS);
-  
+
   private void outputRecordKeysPrev(ClassGenerator<StreamingAggregator> cg, TypedFieldId[] keyOutputIds, LogicalExpression[] keyExprs){
     cg.setMappingSet(RECORD_KEYS_PREV);
 
@@ -285,11 +290,11 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       cg.setMappingSet(RECORD_KEYS_PREV_OUT);
       HoldingContainer outerExpression = cg.addExpr(new ValueVectorWriteExpression(keyOutputIds[i], new HoldingContainerExpression(innerExpression), true), false);
       cg.getBlock(BlockType.EVAL)._if(outerExpression.getValue().eq(JExpr.lit(0)))._then()._return(JExpr.FALSE);
-      
+
     }
     cg.getBlock(BlockType.EVAL)._return(JExpr.TRUE);
   }
-  
+
   private void getIndex(ClassGenerator<StreamingAggregator> g){
     switch(incoming.getSchema().getSelectionVectorMode()){
     case FOUR_BYTE: {
@@ -308,12 +313,12 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
       g.getBlock("getVectorIndex")._return(var.invoke("getIndex").arg(JExpr.direct("recordIndex")));;
       return;
     }
-     
+
     default:
       throw new IllegalStateException();
-      
+
     }
-   
+
   }
 
   @Override
@@ -321,7 +326,7 @@ public class StreamingAggBatch extends AbstractRecordBatch<StreamingAggregate> {
     super.cleanup();
     incoming.cleanup();
   }
-  
+
   @Override
   protected void killIncoming() {
     incoming.kill();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index 5eec3bb..6b768c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -17,25 +17,16 @@
  */
 package org.apache.drill.exec.physical.impl.join;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.JoinCondition;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.OutOfMemoryException;
-import org.apache.drill.exec.record.*;
-import org.eigenbase.rel.JoinRelType;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.sun.codemodel.JExpression;
-import com.sun.codemodel.JVar;
-import com.sun.codemodel.JExpr;
-
 import org.apache.drill.exec.compile.sig.GeneratorMapping;
 import org.apache.drill.exec.compile.sig.MappingSet;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -44,6 +35,7 @@ import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.HashJoinPOP;
 import org.apache.drill.exec.physical.impl.common.ChainedHashTable;
@@ -51,14 +43,29 @@ import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.physical.impl.common.HashTableConfig;
 import org.apache.drill.exec.physical.impl.sort.RecordBatchData;
 import org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.ExpandableHyperContainer;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.allocator.VectorAllocator;
+import org.eigenbase.rel.JoinRelType;
+
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+import com.sun.codemodel.JVar;
 
 public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
 
   public static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024;
   public static final long ALLOCATOR_MAX_RESERVATION = 20L*1000*1000*1000;
 
+    private static final int LEFT_INPUT = 0;
+    private static final int RIGHT_INPUT = 1;
+
     // Probe side record batch
     private final RecordBatch left;
 
@@ -137,7 +144,7 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
 
     @Override
     public IterOutcome next() {
-
+        stats.startProcessing();
         try {
             /* If we are here for the first time, execute the build phase of the
              * hash join and setup the run time generated class for the probe side
@@ -153,7 +160,7 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
                  * as well, for the materialization to be successful. This batch will not be used
                  * till we complete the build phase.
                  */
-                leftUpstream = left.next();
+                leftUpstream = next(LEFT_INPUT, left);
 
                 // Build the hash table, using the build side record batches.
                 executeBuildPhase();
@@ -200,12 +207,12 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
                     for (VectorWrapper<?> wrapper : left) {
                       wrapper.getValueVector().clear();
                     }
-                    leftUpstream = left.next();
+                    leftUpstream = next(LEFT_INPUT, left);
                     while (leftUpstream == IterOutcome.OK_NEW_SCHEMA || leftUpstream == IterOutcome.OK) {
                       for (VectorWrapper<?> wrapper : left) {
                         wrapper.getValueVector().clear();
                       }
-                      leftUpstream = left.next();
+                      leftUpstream = next(LEFT_INPUT, left);
                     }
                 }
             }
@@ -214,6 +221,7 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
             return IterOutcome.NONE;
 
         } catch (ClassTransformationException | SchemaChangeException | IOException e) {
+            stats.stopProcessing();
             context.fail(e);
             killIncoming();
             return IterOutcome.STOP;
@@ -256,7 +264,7 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
     public void executeBuildPhase() throws SchemaChangeException, ClassTransformationException, IOException {
 
         //Setup the underlying hash table
-        IterOutcome rightUpstream = right.next();
+        IterOutcome rightUpstream = next(RIGHT_INPUT, right);
 
         boolean moreData = true;
 
@@ -323,7 +331,7 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
                     break;
             }
             // Get the next record batch
-            rightUpstream = right.next();
+            rightUpstream = next(RIGHT_INPUT, right);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
index c07878a..faca32a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
@@ -35,6 +35,9 @@ public final class JoinStatus {
     INCOMING, SV4;
   }
 
+  private static final int LEFT_INPUT = 0;
+  private static final int RIGHT_INPUT = 1;
+
   public final RecordBatch left;
   private int leftPosition;
   private IterOutcome lastLeft;
@@ -63,10 +66,18 @@ public final class JoinStatus {
     this.joinType = output.getJoinType();
   }
 
+  private final IterOutcome nextLeft(){
+    return outputBatch.next(LEFT_INPUT, left);
+  }
+
+  private final IterOutcome nextRight(){
+    return outputBatch.next(RIGHT_INPUT, right);
+  }
+
   public final void ensureInitial(){
     if(!initialSet){
-      this.lastLeft = left.next();
-      this.lastRight = right.next();
+      this.lastLeft = nextLeft();
+      this.lastRight = nextRight();
       initialSet = true;
     }
   }
@@ -148,7 +159,7 @@ public final class JoinStatus {
     if (!isLeftPositionInCurrentBatch()) {
       leftPosition = 0;
       releaseData(left);
-      lastLeft = left.next();
+      lastLeft = nextLeft();
       return lastLeft == IterOutcome.OK;
     }
     lastLeft = IterOutcome.OK;
@@ -167,7 +178,7 @@ public final class JoinStatus {
     if (!isRightPositionInCurrentBatch()) {
       rightPosition = 0;
       releaseData(right);
-      lastRight = right.next();
+      lastRight = nextRight();
       return lastRight == IterOutcome.OK;
     }
     lastRight = IterOutcome.OK;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index 48b7fea..46dea64 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -66,9 +66,9 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
 
   public static final long ALLOCATOR_INITIAL_RESERVATION = 1*1024*1024;
   public static final long ALLOCATOR_MAX_RESERVATION = 20L*1000*1000*1000;
-    
+
   public final MappingSet setupMapping =
-      new MappingSet("null", "null", 
+      new MappingSet("null", "null",
                      GM("doSetup", "doSetup", null, null),
                      GM("doSetup", "doSetup", null, null));
   public final MappingSet copyLeftMapping =
@@ -96,7 +96,7 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
                      GM("doSetup", "doCompareNextLeftKey", null, null),
                      GM("doSetup", "doCompareNextLeftKey", null, null));
 
-  
+
   private final RecordBatch left;
   private final RecordBatch right;
   private final JoinStatus status;
@@ -104,7 +104,7 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
   private final JoinRelType joinType;
   private JoinWorker worker;
   public MergeJoinBatchBuilder batchBuilder;
-  
+
   protected MergeJoinBatch(MergeJoinPOP popConfig, FragmentContext context, RecordBatch left, RecordBatch right) throws OutOfMemoryException {
     super(popConfig, context);
 
@@ -130,10 +130,10 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
 
   @Override
   public IterOutcome next() {
-    
+
     // we do this in the here instead of the constructor because don't necessary want to start consuming on construction.
     status.ensureInitial();
-    
+
     // loop so we can start over again if we find a new batch was created.
     while(true){
 
@@ -153,14 +153,17 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
         logger.debug("NO MORE DATA; returning {}  NONE");
         return IterOutcome.NONE;
       }
-      
+
       boolean first = false;
       if(worker == null){
         try {
           logger.debug("Creating New Worker");
+          stats.startSetup();
           this.worker = generateNewWorker();
           first = true;
+          stats.stopSetup();
         } catch (ClassTransformationException | IOException | SchemaChangeException e) {
+          stats.stopSetup();
           context.fail(new SchemaChangeException(e));
           kill();
           return IterOutcome.STOP;
@@ -222,13 +225,13 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
       left.cleanup();
       right.cleanup();
   }
-  
-  private void generateDoCompareNextLeft(ClassGenerator<JoinWorker> cg, JVar incomingRecordBatch, 
+
+  private void generateDoCompareNextLeft(ClassGenerator<JoinWorker> cg, JVar incomingRecordBatch,
       JVar incomingLeftRecordBatch, JVar joinStatus, ErrorCollector collector) throws ClassTransformationException {
     boolean nextLeftIndexDeclared = false;
 
     cg.setMappingSet(compareLeftMapping);
-    
+
     for (JoinCondition condition : conditions) {
       final LogicalExpression leftFieldExpr = condition.getLeft();
 
@@ -242,52 +245,52 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
       ////////////////////////////////
       cg.setMappingSet(compareLeftMapping);
       cg.getSetupBlock().assign(JExpr._this().ref(incomingRecordBatch), JExpr._this().ref(incomingLeftRecordBatch));
-  
+
       if (!nextLeftIndexDeclared) {
         // int nextLeftIndex = leftIndex + 1;
         cg.getEvalBlock().decl(JType.parse(cg.getModel(), "int"), "nextLeftIndex", JExpr.direct("leftIndex").plus(JExpr.lit(1)));
         nextLeftIndexDeclared = true;
-      } 
+      }
       // check if the next key is in this batch
       cg.getEvalBlock()._if(joinStatus.invoke("isNextLeftPositionInCurrentBatch").eq(JExpr.lit(false)))
                        ._then()
                          ._return(JExpr.lit(-1));
-  
+
       // generate VV read expressions
       ClassGenerator.HoldingContainer compareThisLeftExprHolder = cg.addExpr(materializedLeftExpr, false);
       cg.setMappingSet(compareNextLeftMapping); // change mapping from 'leftIndex' to 'nextLeftIndex'
       ClassGenerator.HoldingContainer compareNextLeftExprHolder = cg.addExpr(materializedLeftExpr, false);
-  
+
       if (compareThisLeftExprHolder.isOptional()) {
         // handle null == null
         cg.getEvalBlock()._if(compareThisLeftExprHolder.getIsSet().eq(JExpr.lit(0))
                               .cand(compareNextLeftExprHolder.getIsSet().eq(JExpr.lit(0))))
                          ._then()
                            ._return(JExpr.lit(0));
-    
+
         // handle null == !null
         cg.getEvalBlock()._if(compareThisLeftExprHolder.getIsSet().eq(JExpr.lit(0))
                               .cor(compareNextLeftExprHolder.getIsSet().eq(JExpr.lit(0))))
                          ._then()
                            ._return(JExpr.lit(1));
       }
-  
+
       // check value equality
-  
+
       LogicalExpression gh = FunctionGenerationHelper.getComparator(compareThisLeftExprHolder,
         compareNextLeftExprHolder,
         context.getFunctionRegistry());
       HoldingContainer out = cg.addExpr(gh, false);
-      
-      //If not 0, it means not equal. We return this out value. 
+
+      //If not 0, it means not equal. We return this out value.
       JConditional jc = cg.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
       jc._then()._return(out.getValue());
     }
-    
+
     //Pass the equality check for all the join conditions. Finally, return 0.
     cg.getEvalBlock()._return(JExpr.lit(0));
-  }  
-  
+  }
+
   private JoinWorker generateNewWorker() throws ClassTransformationException, IOException, SchemaChangeException{
 
     final ClassGenerator<JoinWorker> cg = CodeGenerator.getRoot(JoinWorker.TEMPLATE_DEFINITION, context.getFunctionRegistry());
@@ -322,11 +325,11 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     //generate doCompare() method
     /////////////////////////////////////////
     generateDoCompare(cg, incomingRecordBatch, incomingLeftRecordBatch, incomingRightRecordBatch, collector);
-    
+
     //generate doCompareNextLeftKey() method
     /////////////////////////////////////////
     generateDoCompareNextLeft(cg, incomingRecordBatch, incomingLeftRecordBatch, joinStatus, collector);
-    
+
     // generate copyLeft()
     //////////////////////
     cg.setMappingSet(copyLeftMapping);
@@ -394,12 +397,12 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
   private void allocateBatch() {
     // allocate new batch space.
     container.clear();
-    
+
     //estimation of joinBatchSize : max of left/right size, expanded by a factor of 16, which is then bounded by MAX_BATCH_SIZE.
     int leftCount = status.isLeftPositionAllowed() ? left.getRecordCount() : 0;
     int rightCount = status.isRightPositionAllowed() ? right.getRecordCount() : 0;
     int joinBatchSize = Math.min(Math.max(leftCount, rightCount) * 16, MAX_BATCH_SIZE);
-    
+
     // add fields from both batches
     if (leftCount > 0) {
 
@@ -436,11 +439,11 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     logger.debug("Built joined schema: {}", container.getSchema());
   }
 
-  private void generateDoCompare(ClassGenerator<JoinWorker> cg, JVar incomingRecordBatch, 
+  private void generateDoCompare(ClassGenerator<JoinWorker> cg, JVar incomingRecordBatch,
       JVar incomingLeftRecordBatch, JVar incomingRightRecordBatch, ErrorCollector collector) throws ClassTransformationException {
-    
+
     cg.setMappingSet(compareMapping);
-    
+
     for (JoinCondition condition : conditions) {
       final LogicalExpression leftFieldExpr = condition.getLeft();
       final LogicalExpression rightFieldExpr = condition.getRight();
@@ -482,37 +485,37 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
             .cand(compareRightExprHolder.getIsSet().eq(JExpr.lit(0))))
             ._then()
             ._return(JExpr.lit(0));
-    
+
         // handle null == !null
         cg.getEvalBlock()._if(compareLeftExprHolder.getIsSet().eq(JExpr.lit(0))
             .cor(compareRightExprHolder.getIsSet().eq(JExpr.lit(0))))
             ._then()
             ._return(JExpr.lit(1));
-  
+
       } else if (compareLeftExprHolder.isOptional()) {
         // handle null == required (null is less than any value)
         cg.getEvalBlock()._if(compareLeftExprHolder.getIsSet().eq(JExpr.lit(0)))
             ._then()
             ._return(JExpr.lit(-1));
-  
+
       } else if (compareRightExprHolder.isOptional()) {
         // handle required == null (null is less than any value)
         cg.getEvalBlock()._if(compareRightExprHolder.getIsSet().eq(JExpr.lit(0)))
             ._then()
             ._return(JExpr.lit(1));
       }
-  
+
       LogicalExpression fh = FunctionGenerationHelper.getComparator(compareLeftExprHolder,
         compareRightExprHolder,
-        context.getFunctionRegistry()); 
+        context.getFunctionRegistry());
       HoldingContainer out = cg.addExpr(fh, false);
-      
-      //If not 0, it means not equal. We return this out value.       
+
+      //If not 0, it means not equal. We return this out value.
       JConditional jc = cg.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
       jc._then()._return(out.getValue());
     }
-    
-    //Pass the equality check for all the join conditions. Finally, return 0.    
-    cg.getEvalBlock()._return(JExpr.lit(0));  
+
+    //Pass the equality check for all the join conditions. Finally, return 0.
+    cg.getEvalBlock()._return(JExpr.lit(0));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index 3d496d3..cc38cbe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -27,8 +27,6 @@ import java.util.PriorityQueue;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.Order.Ordering;
@@ -41,19 +39,28 @@ import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.expr.ValueVectorReadExpression;
 import org.apache.drill.exec.expr.fn.FunctionGenerationHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.MetricDef;
 import org.apache.drill.exec.physical.config.MergingReceiverPOP;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.record.*;
+import org.apache.drill.exec.record.AbstractRecordBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.TypedFieldId;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.eigenbase.rel.RelFieldCollation.Direction;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.sun.codemodel.JArray;
 import com.sun.codemodel.JClass;
@@ -92,6 +99,15 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
   private List<VectorAllocator> allocators;
   private MergingReceiverPOP config;
 
+  public static enum Metric implements MetricDef{
+    NEXT_WAIT_NANOS;
+
+    @Override
+    public int metricId() {
+      return ordinal();
+    }
+  }
+
   public MergingRecordBatch(FragmentContext context,
                             MergingReceiverPOP config,
                             RawFragmentBatchProvider[] fragProviders) throws OutOfMemoryException {
@@ -104,8 +120,21 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
     this.outgoingContainer = new VectorContainer();
   }
 
+  private RawFragmentBatch getNext(RawFragmentBatchProvider provider) throws IOException{
+    long startNext = System.nanoTime();
+    RawFragmentBatch b = provider.getNext();
+    if(b != null){
+      stats.batchReceived(0, b.getHeader().getDef().getRecordCount(), false);
+    }
+    stats.addLongStat(Metric.NEXT_WAIT_NANOS, System.nanoTime() - startNext);
+    return b;
+  }
+
   @Override
   public IterOutcome next() {
+    stats.startProcessing();
+    try{
+
     if (fragProviders.length == 0) return IterOutcome.NONE;
     boolean schemaChanged = false;
 
@@ -131,7 +160,7 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
       for (RawFragmentBatchProvider provider : fragProviders) {
         RawFragmentBatch rawBatch = null;
         try {
-          rawBatch = provider.getNext();
+          rawBatch = getNext(provider);
         } catch (IOException e) {
           context.fail(e);
           return IterOutcome.STOP;
@@ -238,10 +267,10 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
         // reached the end of an incoming record batch
         RawFragmentBatch nextBatch = null;
         try {
-          nextBatch = fragProviders[node.batchId].getNext();
+          nextBatch = getNext(fragProviders[node.batchId]);
 
           while (nextBatch != null && nextBatch.getHeader().getDef().getRecordCount() == 0) {
-            nextBatch = fragProviders[node.batchId].getNext();
+            nextBatch = getNext(fragProviders[node.batchId]);
           }
         } catch (IOException e) {
           context.fail(e);
@@ -301,6 +330,11 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
       return IterOutcome.OK_NEW_SCHEMA;
     else
       return IterOutcome.OK;
+
+    }finally{
+      stats.stopProcessing();
+    }
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index 4641de6..f105363 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -166,7 +166,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     recordsSampled += incoming.getRecordCount();
 
     outer: while (recordsSampled < recordsToSample) {
-      upstream = incoming.next();
+      upstream = next(incoming);
       switch (upstream) {
       case NONE:
       case NOT_YET:
@@ -414,97 +414,102 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
 
   @Override
   public IterOutcome next() {
-    container.zeroVectors();
-
-    // if we got IterOutcome.NONE while getting partition vectors, and there are no batches on the queue, then we are
-    // done
-    if (upstreamNone && (batchQueue == null || batchQueue.size() == 0))
-      return IterOutcome.NONE;
-
-    // if there are batches on the queue, process them first, rather than calling incoming.next()
-    if (batchQueue != null && batchQueue.size() > 0) {
-      VectorContainer vc = batchQueue.poll();
-      recordCount = vc.getRecordCount();
-      try {
-
-        // Must set up a new schema each time, because ValueVectors are not reused between containers in queue
-        setupNewSchema(vc);
-      } catch (SchemaChangeException ex) {
-        kill();
-        logger.error("Failure during query", ex);
-        context.fail(ex);
-        return IterOutcome.STOP;
+    stats.startProcessing();
+    try{
+      container.zeroVectors();
+
+      // if we got IterOutcome.NONE while getting partition vectors, and there are no batches on the queue, then we are
+      // done
+      if (upstreamNone && (batchQueue == null || batchQueue.size() == 0))
+        return IterOutcome.NONE;
+
+      // if there are batches on the queue, process them first, rather than calling incoming.next()
+      if (batchQueue != null && batchQueue.size() > 0) {
+        VectorContainer vc = batchQueue.poll();
+        recordCount = vc.getRecordCount();
+        try {
+
+          // Must set up a new schema each time, because ValueVectors are not reused between containers in queue
+          setupNewSchema(vc);
+        } catch (SchemaChangeException ex) {
+          kill();
+          logger.error("Failure during query", ex);
+          context.fail(ex);
+          return IterOutcome.STOP;
+        }
+        doWork(vc);
+        vc.zeroVectors();
+        return IterOutcome.OK_NEW_SCHEMA;
       }
-      doWork(vc);
-      vc.zeroVectors();
-      return IterOutcome.OK_NEW_SCHEMA;
-    }
 
-    // Reaching this point, either this is the first iteration, or there are no batches left on the queue and there are
-    // more incoming
-    IterOutcome upstream = incoming.next();
+      // Reaching this point, either this is the first iteration, or there are no batches left on the queue and there are
+      // more incoming
+      IterOutcome upstream = next(incoming);
 
-    if (this.first && upstream == IterOutcome.OK) {
-      throw new RuntimeException("Invalid state: First batch should have OK_NEW_SCHEMA");
-    }
-
-    // If this is the first iteration, we need to generate the partition vectors before we can proceed
-    if (this.first && upstream == IterOutcome.OK_NEW_SCHEMA) {
-      if (!getPartitionVectors()){
-        cleanup();
-        return IterOutcome.STOP;
+      if (this.first && upstream == IterOutcome.OK) {
+        throw new RuntimeException("Invalid state: First batch should have OK_NEW_SCHEMA");
       }
 
-      batchQueue = new LinkedBlockingQueue<>(this.sampledIncomingBatches);
-      first = false;
-
-      // Now that we have the partition vectors, we immediately process the first batch on the queue
-      VectorContainer vc = batchQueue.poll();
-      try {
-        setupNewSchema(vc);
-      } catch (SchemaChangeException ex) {
-        kill();
-        logger.error("Failure during query", ex);
-        context.fail(ex);
-        return IterOutcome.STOP;
+      // If this is the first iteration, we need to generate the partition vectors before we can proceed
+      if (this.first && upstream == IterOutcome.OK_NEW_SCHEMA) {
+        if (!getPartitionVectors()){
+          cleanup();
+          return IterOutcome.STOP;
+        }
+
+        batchQueue = new LinkedBlockingQueue<>(this.sampledIncomingBatches);
+        first = false;
+
+        // Now that we have the partition vectors, we immediately process the first batch on the queue
+        VectorContainer vc = batchQueue.poll();
+        try {
+          setupNewSchema(vc);
+        } catch (SchemaChangeException ex) {
+          kill();
+          logger.error("Failure during query", ex);
+          context.fail(ex);
+          return IterOutcome.STOP;
+        }
+        doWork(vc);
+        vc.zeroVectors();
+        recordCount = vc.getRecordCount();
+        return IterOutcome.OK_NEW_SCHEMA;
       }
-      doWork(vc);
-      vc.zeroVectors();
-      recordCount = vc.getRecordCount();
-      return IterOutcome.OK_NEW_SCHEMA;
-    }
 
-    // if this now that all the batches on the queue are processed, we begin processing the incoming batches. For the
-    // first one
-    // we need to generate a new schema, even if the outcome is IterOutcome.OK After that we can reuse the schema.
-    if (this.startedUnsampledBatches == false) {
-      this.startedUnsampledBatches = true;
-      if (upstream == IterOutcome.OK)
-        upstream = IterOutcome.OK_NEW_SCHEMA;
-    }
-    switch (upstream) {
-    case NONE:
-    case NOT_YET:
-    case STOP:
-      cleanup();
-      recordCount = 0;
-      return upstream;
-    case OK_NEW_SCHEMA:
-      try {
-        setupNewSchema(incoming);
-      } catch (SchemaChangeException ex) {
-        kill();
-        logger.error("Failure during query", ex);
-        context.fail(ex);
-        return IterOutcome.STOP;
+      // if this now that all the batches on the queue are processed, we begin processing the incoming batches. For the
+      // first one
+      // we need to generate a new schema, even if the outcome is IterOutcome.OK After that we can reuse the schema.
+      if (this.startedUnsampledBatches == false) {
+        this.startedUnsampledBatches = true;
+        if (upstream == IterOutcome.OK)
+          upstream = IterOutcome.OK_NEW_SCHEMA;
+      }
+      switch (upstream) {
+      case NONE:
+      case NOT_YET:
+      case STOP:
+        cleanup();
+        recordCount = 0;
+        return upstream;
+      case OK_NEW_SCHEMA:
+        try {
+          setupNewSchema(incoming);
+        } catch (SchemaChangeException ex) {
+          kill();
+          logger.error("Failure during query", ex);
+          context.fail(ex);
+          return IterOutcome.STOP;
+        }
+        // fall through.
+      case OK:
+        doWork(incoming);
+        recordCount = incoming.getRecordCount();
+        return upstream; // change if upstream changed, otherwise normal.
+      default:
+        throw new UnsupportedOperationException();
       }
-      // fall through.
-    case OK:
-      doWork(incoming);
-      recordCount = incoming.getRecordCount();
-      return upstream; // change if upstream changed, otherwise normal.
-    default:
-      throw new UnsupportedOperationException();
+    }finally{
+      stats.stopProcessing();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index 844d6db..b6b4c33 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -20,11 +20,10 @@ package org.apache.drill.exec.record;
 import java.util.Iterator;
 
 import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
@@ -36,12 +35,14 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
   protected final T popConfig;
   protected final FragmentContext context;
   protected final OperatorContext oContext;
+  protected final OperatorStats stats;
 
   protected AbstractRecordBatch(T popConfig, FragmentContext context) throws OutOfMemoryException {
     super();
     this.context = context;
     this.popConfig = popConfig;
     this.oContext = new OperatorContext(popConfig, context);
+    this.stats = oContext.getStats();
   }
 
   @Override
@@ -58,6 +59,27 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     return popConfig;
   }
 
+  public final IterOutcome next(RecordBatch b){
+    return next(0, b);
+  }
+
+  public final IterOutcome next(int inputIndex, RecordBatch b){
+    stats.stopProcessing();
+    IterOutcome next = b.next();
+
+    switch(next){
+    case OK_NEW_SCHEMA:
+      stats.batchReceived(inputIndex, b.getRecordCount(), true);
+      break;
+    case OK:
+      stats.batchReceived(inputIndex, b.getRecordCount(), false);
+      break;
+    }
+
+    stats.startProcessing();
+    return next;
+  }
+
   @Override
   public BatchSchema getSchema() {
     return container.getSchema();
@@ -75,6 +97,7 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
     oContext.close();
   }
 
+
   @Override
   public SelectionVector2 getSelectionVector2() {
     throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
index dd2cfe0..13e4ac8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
@@ -29,7 +29,7 @@ public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> exte
   protected final RecordBatch incoming;
   private boolean first = true;
   protected boolean outOfMemory = false;
-  
+
   public AbstractSingleRecordBatch(T popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException {
     super(popConfig, context);
     this.incoming = incoming;
@@ -42,36 +42,46 @@ public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> exte
 
   @Override
   public IterOutcome next() {
-    IterOutcome upstream = incoming.next();
-    if(first && upstream == IterOutcome.OK) upstream = IterOutcome.OK_NEW_SCHEMA;
-    first = false;
-    switch(upstream){
-    case NONE:
-    case NOT_YET:
-    case STOP:
-      return upstream;
-    case OUT_OF_MEMORY:
-      return upstream;
-    case OK_NEW_SCHEMA:
-      try{
-        setupNewSchema();
-      }catch(SchemaChangeException ex){
-        kill();
-        logger.error("Failure during query", ex);
-        context.fail(ex);
-        return IterOutcome.STOP;
-      }
-      // fall through.
-    case OK:
-      doWork();
-      if (outOfMemory) {
-        outOfMemory = false;
-        return IterOutcome.OUT_OF_MEMORY;
+    try{
+      stats.startProcessing();
+      IterOutcome upstream = next(incoming);
+      if(first && upstream == IterOutcome.OK) upstream = IterOutcome.OK_NEW_SCHEMA;
+      first = false;
+      switch(upstream){
+      case NONE:
+      case NOT_YET:
+      case STOP:
+        return upstream;
+      case OUT_OF_MEMORY:
+        return upstream;
+      case OK_NEW_SCHEMA:
+        try{
+          stats.startSetup();
+          setupNewSchema();
+          stats.stopSetup();
+        }catch(SchemaChangeException ex){
+          stats.stopSetup();
+          kill();
+          logger.error("Failure during query", ex);
+          context.fail(ex);
+          return IterOutcome.STOP;
+        }
+        // fall through.
+      case OK:
+        doWork();
+        if (outOfMemory) {
+          outOfMemory = false;
+          return IterOutcome.OUT_OF_MEMORY;
+        }
+        return upstream; // change if upstream changed, otherwise normal.
+      default:
+        throw new UnsupportedOperationException();
       }
-      return upstream; // change if upstream changed, otherwise normal.
-    default:
-      throw new UnsupportedOperationException();
+    }finally{
+      stats.stopProcessing();
     }
+
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 7f607a3..7297dc3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -22,7 +22,6 @@ import java.io.Closeable;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
-import org.apache.drill.exec.cache.DistributedMultiMap;
 import org.apache.drill.exec.cache.HazelCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
@@ -31,6 +30,8 @@ import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.service.ServiceEngine;
 import org.apache.drill.exec.work.WorkManager;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.ServletHandler;
 
 import com.google.common.io.Closeables;
 
@@ -71,6 +72,7 @@ public class Drillbit implements Closeable{
   final DistributedCache cache;
   final WorkManager manager;
   final BootStrapContext context;
+  final Server embeddedJetty;
 
   private volatile RegistrationHandle handle;
 
@@ -89,8 +91,17 @@ public class Drillbit implements Closeable{
       this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
       this.cache = new HazelCache(config, context.getAllocator());
     }
+    this.embeddedJetty = new Server(474747);
   }
 
+  private void setupJetty(){
+    ServletHandler handler = new ServletHandler();
+    embeddedJetty.setHandler(handler);
+
+  }
+
+
+
   public void run() throws Exception {
     coord.start(10000);
     DrillbitEndpoint md = engine.start();
@@ -99,6 +110,7 @@ public class Drillbit implements Closeable{
     manager.getContext().getStorage().init();
     manager.getContext().getOptionManager().init();
     handle = coord.register(md);
+    embeddedJetty.start();
   }
 
   public void close() {
@@ -109,7 +121,11 @@ public class Drillbit implements Closeable{
     } catch (InterruptedException e) {
       logger.warn("Interrupted while sleeping during coordination deregistration.");
     }
-
+    try {
+      embeddedJetty.stop();
+    } catch (Exception e) {
+      logger.warn("Failure while shutting down embedded jetty server.");
+    }
     Closeables.closeQuietly(engine);
     Closeables.closeQuietly(coord);
     Closeables.closeQuietly(manager);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 2965e79..718da23 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -35,6 +35,7 @@ import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.physical.impl.WriterRecordBatch;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
@@ -65,7 +66,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   private final String name;
   protected final CompressionCodecFactory codecFactory;
   private final boolean compressible;
-  
+
   protected EasyFormatPlugin(String name, DrillbitContext context, DrillFileSystem fs, StoragePluginConfig storageConfig,
                              T formatConfig, boolean readable, boolean writable, boolean blockSplittable, boolean compressible, List<String> extensions, String defaultName){
     this.matcher = new BasicFormatMatcher(this, fs, extensions, compressible);
@@ -80,7 +81,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     this.name = name == null ? defaultName : name;
     this.codecFactory = new CompressionCodecFactory(new Configuration(fs.getUnderlying().getConf()));
   }
-  
+
   @Override
   public DrillFileSystem getFileSystem() {
     return fs;
@@ -90,7 +91,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   public DrillbitContext getContext() {
     return context;
   }
-  
+
   @Override
   public String getName() {
     return name;
@@ -99,7 +100,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   /**
    * Whether or not you can split the format based on blocks within file boundaries. If not, the simple format engine will
    * only split on file boundaries.
-   * 
+   *
    * @return True if splittable.
    */
   public boolean isBlockSplittable() {
@@ -184,7 +185,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   public AbstractGroupScan getGroupScan(FileSelection selection, List<SchemaPath> columns) throws IOException {
     return new EasyGroupScan(selection, this, columns, selection.selectionRoot);
   }
-  
+
   @Override
   public FormatPluginConfig getConfig() {
     return formatConfig;
@@ -214,5 +215,8 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   public Set<StoragePluginOptimizerRule> getOptimizerRules() {
     return ImmutableSet.of();
   }
-  
+
+  public abstract int getReaderOperatorType();
+  public abstract int getWriterOperatorType();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
index 0b3fe0f..5f9226e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
@@ -26,6 +26,7 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.store.dfs.NamedFormatPluginConfig;
 import org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
@@ -45,13 +46,13 @@ public class EasySubScan extends AbstractSubScan{
   private final EasyFormatPlugin<?> formatPlugin;
   private final List<SchemaPath> columns;
   private String selectionRoot;
-  
+
   @JsonCreator
   public EasySubScan(
       @JsonProperty("files") List<FileWorkImpl> files, //
       @JsonProperty("storage") StoragePluginConfig storageConfig, //
       @JsonProperty("format") FormatPluginConfig formatConfig, //
-      @JacksonInject StoragePluginRegistry engineRegistry, // 
+      @JacksonInject StoragePluginRegistry engineRegistry, //
       @JsonProperty("columns") List<SchemaPath> columns, //
       @JsonProperty("selectionRoot") String selectionRoot
       ) throws IOException, ExecutionSetupException {
@@ -62,7 +63,7 @@ public class EasySubScan extends AbstractSubScan{
     this.columns = columns;
     this.selectionRoot = selectionRoot;
   }
-  
+
   public EasySubScan(List<FileWorkImpl> files, EasyFormatPlugin<?> plugin, List<SchemaPath> columns, String selectionRoot){
     this.formatPlugin = plugin;
     this.files = files;
@@ -74,7 +75,7 @@ public class EasySubScan extends AbstractSubScan{
   public String getSelectionRoot() {
     return selectionRoot;
   }
-  
+
   @JsonIgnore
   public EasyFormatPlugin<?> getFormatPlugin(){
     return formatPlugin;
@@ -100,11 +101,15 @@ public class EasySubScan extends AbstractSubScan{
       return formatPlugin.getConfig();
     }
   }
-  
+
   @JsonProperty("columns")
   public List<SchemaPath> getColumns(){
     return columns;
   }
 
-   
+  @Override
+  public int getOperatorType() {
+    return formatPlugin.getReaderOperatorType();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
index 864ae48..5ca781b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyWriter.java
@@ -94,4 +94,9 @@ public class EasyWriter extends AbstractWriter {
     // TODO:
     return new OperatorCost(1,1,1,1);
   }
+
+  @Override
+  public int getOperatorType() {
+    return formatPlugin.getReaderOperatorType();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
index 0c50898..89694f8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.direct;
 
 import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.RecordReader;
 
 public class DirectSubScan extends AbstractSubScan{
@@ -34,4 +35,10 @@ public class DirectSubScan extends AbstractSubScan{
     return reader;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.DIRECT_SUB_SCAN_VALUE;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index 04a9768..e410306 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -22,12 +22,14 @@ import java.util.List;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.google.common.collect.Lists;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.RecordWriter;
@@ -80,4 +82,16 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
     }
 
   }
+
+  @Override
+  public int getReaderOperatorType() {
+    return CoreOperatorType.JSON_SUB_SCAN_VALUE;
+  }
+
+  @Override
+  public int getWriterOperatorType() {
+    throw new UnsupportedOperationException();
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
index f6cc58e..cd28d30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/text/TextFormatPlugin.java
@@ -28,6 +28,7 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.RecordReader;
 import org.apache.drill.exec.store.RecordWriter;
@@ -71,7 +72,7 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
 
   @Override
   public AbstractGroupScan getGroupScan(FileSelection selection, List<SchemaPath> columns) throws IOException {
-    return new EasyGroupScan(selection, this, null, selection.selectionRoot); //TODO : textformat supports project? 
+    return new EasyGroupScan(selection, this, null, selection.selectionRoot); //TODO : textformat supports project?
   }
 
   @Override
@@ -127,6 +128,16 @@ public class TextFormatPlugin extends EasyFormatPlugin<TextFormatPlugin.TextForm
         return true;
       return false;
     }
-    
+
+  }
+
+  @Override
+  public int getReaderOperatorType() {
+    return CoreOperatorType.TEXT_SUB_SCAN_VALUE;
+  }
+
+  @Override
+  public int getWriterOperatorType() {
+    return CoreOperatorType.TEXT_WRITER_VALUE;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
index 7f2d0f1..ecd952c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveSubScan.java
@@ -25,11 +25,13 @@ import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.io.ByteArrayDataInput;
 import com.google.common.io.ByteStreams;
+
 import org.apache.commons.codec.binary.Base64;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -144,4 +146,9 @@ public class HiveSubScan extends AbstractBase implements SubScan {
   public Iterator<PhysicalOperator> iterator() {
     return Iterators.emptyIterator();
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.HIVE_SUB_SCAN_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaSubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaSubScan.java
index dce4d3b..70e1258 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaSubScan.java
@@ -18,16 +18,17 @@
 package org.apache.drill.exec.store.ischema;
 
 import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
 public class InfoSchemaSubScan extends AbstractSubScan{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InfoSchemaSubScan.class);
-  
+
 
   private final SelectedTable table;
-  
+
   @JsonCreator
   public InfoSchemaSubScan(@JsonProperty("table") SelectedTable table) {
     this.table = table;
@@ -36,6 +37,10 @@ public class InfoSchemaSubScan extends AbstractSubScan{
   public SelectedTable getTable() {
     return table;
   }
-  
-  
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.INFO_SCHEMA_SUB_SCAN_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
index f616bca..869e40c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.base.AbstractStore;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Store;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -51,7 +52,7 @@ public class MockStorePOP extends AbstractStore {
 
   @Override
   public void applyAssignments(List<DrillbitEndpoint> endpoints) {
-    
+
   }
 
   @Override
@@ -69,7 +70,10 @@ public class MockStorePOP extends AbstractStore {
     return new MockStorePOP(child);
   }
 
+  @Override
+  public int getOperatorType() {
+    throw new UnsupportedOperationException();
+  }
 
-  
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
index 0753be5..517ad3e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -86,7 +87,7 @@ public class MockSubScanPOP extends AbstractBase implements SubScan {
   // will want to replace these two methods with an interface above for AbstractSubScan
   @Override
   public boolean isExecutable() {
-    return true;  
+    return true;
   }
 
   @Override
@@ -103,4 +104,9 @@ public class MockSubScanPOP extends AbstractBase implements SubScan {
 
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.MOCK_SUB_SCAN_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index dd5c91c..1f66e9f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
@@ -140,4 +141,9 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     return columns;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.PARQUET_ROW_GROUP_SCAN_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
index 51e9219..762942d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetWriter.java
@@ -19,12 +19,14 @@ package org.apache.drill.exec.store.parquet;
 
 import com.fasterxml.jackson.annotation.*;
 import com.google.common.base.Preconditions;
+
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.AbstractWriter;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 import java.io.IOException;
@@ -88,4 +90,10 @@ public class ParquetWriter extends AbstractWriter {
     // TODO:
     return new OperatorCost(1,1,1,1);
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.PARQUET_WRITER_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 36b7509..3dbb98e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -99,7 +99,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     this.queryRequest = queryRequest;
     this.context = new QueryContext(connection.getSession(), queryId, dContext);
     this.initiatingClient = connection;
-    this.fragmentManager = new QueryManager(new ForemanManagerListener(), dContext.getController());
+    this.fragmentManager = new QueryManager(bee.getContext().getCache(), new ForemanManagerListener(), dContext.getController());
     this.bee = bee;
 
     this.state = new AtomicState<QueryState>(QueryState.PENDING) {
@@ -168,7 +168,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
    * Called by execution pool to do foreman setup. Actual query execution is a separate phase (and can be scheduled).
    */
   public void run() {
-    
+
     final String originalThread = Thread.currentThread().getName();
     Thread.currentThread().setName(QueryIdHelper.getQueryId(queryId) + ":foreman");
     // convert a run query request into action

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
new file mode 100644
index 0000000..509000f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
@@ -0,0 +1,43 @@
+package org.apache.drill.exec.work.foreman;
+
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+public class FragmentData {
+  private final boolean isLocal;
+  private volatile FragmentStatus status;
+  private volatile long lastStatusUpdate = 0;
+  private final DrillbitEndpoint endpoint;
+
+  public FragmentData(FragmentHandle handle, DrillbitEndpoint endpoint, boolean isLocal) {
+    super();
+    this.status = FragmentStatus.newBuilder().setHandle(handle).setState(FragmentState.SENDING).build();
+    this.endpoint = endpoint;
+    this.isLocal = isLocal;
+  }
+
+  public void setStatus(FragmentStatus status){
+    this.status = status;
+    lastStatusUpdate = System.currentTimeMillis();
+  }
+
+  public FragmentStatus getStatus() {
+    return status;
+  }
+
+  public boolean isLocal() {
+    return isLocal;
+  }
+
+  public long getLastStatusUpdate() {
+    return lastStatusUpdate;
+  }
+
+  public DrillbitEndpoint getEndpoint() {
+    return endpoint;
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 01b0df8..c9c769a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -20,17 +20,13 @@ package org.apache.drill.exec.work.foreman;
 import io.netty.buffer.ByteBuf;
 
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.base.FragmentRoot;
-import org.apache.drill.exec.physical.impl.ImplCreator;
-import org.apache.drill.exec.physical.impl.RootExec;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
@@ -51,28 +47,26 @@ import org.apache.drill.exec.work.fragment.AbstractStatusReporter;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.exec.work.fragment.RootFragmentManager;
 
-import com.google.common.collect.Maps;
-
 /**
- * Each Foreman holds its own fragment manager.  This manages the events associated with execution of a particular query across all fragments.  
+ * Each Foreman holds its own fragment manager.  This manages the events associated with execution of a particular query across all fragments.
  */
 public class QueryManager implements FragmentStatusListener{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryManager.class);
-  
-  public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.
+
+  private final QueryStatus status;
   private final Controller controller;
   private ForemanManagerListener foreman;
   private AtomicInteger remainingFragmentCount;
   private WorkEventBus workBus;
   private FragmentExecutor rootRunner;
   private volatile QueryId queryId;
-  
-  public QueryManager(ForemanManagerListener foreman, Controller controller) {
+
+  public QueryManager(DistributedCache cache, ForemanManagerListener foreman, Controller controller) {
     super();
     this.foreman = foreman;
     this.controller = controller;
     this.remainingFragmentCount = new AtomicInteger(0);
-    
+    this.status = new QueryStatus(cache);
   }
 
   public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments, List<PlanFragment> intermediateFragments) throws ExecutionSetupException{
@@ -90,44 +84,44 @@ public class QueryManager implements FragmentStatusListener{
       logger.debug("Setting buffers on root context.");
       rootContext.setBuffers(buffers);
       // add fragment to local node.
-      map.put(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
+      status.add(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
       logger.debug("Fragment added to local node.");
       rootRunner = new FragmentExecutor(rootContext, rootOperator, new RootStatusHandler(rootContext, rootFragment));
       RootFragmentManager fragmentManager = new RootFragmentManager(rootFragment.getHandle(), buffers, rootRunner);
-      
+
       if(buffers.isDone()){
         // if we don't have to wait for any incoming data, start the fragment runner.
         bee.addFragmentRunner(fragmentManager.getRunnable());
       }else{
         // if we do, record the fragment manager in the workBus.
-        workBus.setRootFragmentManager(fragmentManager);  
+        workBus.setRootFragmentManager(fragmentManager);
       }
-      
-      
+
+
     }
 
     // keep track of intermediate fragments (not root or leaf)
     for (PlanFragment f : intermediateFragments) {
       logger.debug("Tracking intermediate remote node {} with data {}", f.getAssignment(), f.getFragmentJson());
-      map.put(f.getHandle(), new FragmentData(f.getHandle(), f.getAssignment(), false));
+      status.add(f.getHandle(), new FragmentData(f.getHandle(), f.getAssignment(), false));
     }
 
     // send remote (leaf) fragments.
     for (PlanFragment f : leafFragments) {
       sendRemoteFragment(f);
     }
-    
+
     logger.debug("Fragment runs setup is complete.");
   }
-    
+
   private void sendRemoteFragment(PlanFragment fragment){
     logger.debug("Sending remote fragment to node {} with data {}", fragment.getAssignment(), fragment.getFragmentJson());
-    map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
+    status.add(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
     FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
     controller.getTunnel(fragment.getAssignment()).sendFragment(listener, fragment);
   }
-  
-  
+
+
   @Override
   public void statusUpdate(FragmentStatus status) {
     logger.debug("New fragment status was provided to Foreman of {}", status);
@@ -151,11 +145,11 @@ public class QueryManager implements FragmentStatusListener{
       throw new UnsupportedOperationException();
     }
   }
-  
+
   private void updateStatus(FragmentStatus status){
-    map.get(status.getHandle()).setStatus(status);
+    this.status.update(status);
   }
-  
+
   private void finished(FragmentStatus status){
     updateStatus(status);
     int remaining = remainingFragmentCount.decrementAndGet();
@@ -167,15 +161,15 @@ public class QueryManager implements FragmentStatusListener{
       foreman.cleanupAndSendResult(result);
     }
   }
-  
+
   private void fail(FragmentStatus status){
     updateStatus(status);
     stopQuery();
     QueryResult result = QueryResult.newBuilder().setQueryId(queryId).setQueryState(QueryState.FAILED).addError(status.getError()).build();
     foreman.cleanupAndSendResult(result);
   }
- 
-  
+
+
   private void stopQuery(){
     // Stop all queries with a currently active status.
 //    for(FragmentData data: map.values()){
@@ -195,13 +189,13 @@ public class QueryManager implements FragmentStatusListener{
 //      }
 //    }
   }
-  
+
   public void cancel(){
     stopQuery();
   }
 
   private class CancelListener extends EndpointListener<Ack, FragmentHandle>{
-    
+
     public CancelListener(DrillbitEndpoint endpoint, FragmentHandle handle) {
       super(endpoint, handle);
     }
@@ -220,15 +214,13 @@ public class QueryManager implements FragmentStatusListener{
     }
 
   };
-  
+
   public RpcOutcomeListener<Ack> getSubmitListener(DrillbitEndpoint endpoint, PlanFragment value){
     return new FragmentSubmitListener(endpoint, value);
   }
-  
-  
-  
+
   private class FragmentSubmitListener extends EndpointListener<Ack, PlanFragment>{
-    
+
     public FragmentSubmitListener(DrillbitEndpoint endpoint, PlanFragment value) {
       super(endpoint, value);
     }
@@ -240,44 +232,6 @@ public class QueryManager implements FragmentStatusListener{
     }
 
   }
-  
-  
-  private class FragmentData{
-    private final boolean isLocal;
-    private volatile FragmentStatus status;
-    private volatile long lastStatusUpdate = 0;
-    private final DrillbitEndpoint endpoint;
-    
-    public FragmentData(FragmentHandle handle, DrillbitEndpoint endpoint, boolean isLocal) {
-      super();
-      this.status = FragmentStatus.newBuilder().setHandle(handle).setState(FragmentState.SENDING).build();
-      this.endpoint = endpoint;
-      this.isLocal = isLocal;
-    }
-    
-    public void setStatus(FragmentStatus status){
-      this.status = status;
-      lastStatusUpdate = System.currentTimeMillis();
-    }
-
-    public FragmentStatus getStatus() {
-      return status;
-    }
-
-    public boolean isLocal() {
-      return isLocal;
-    }
-
-    public long getLastStatusUpdate() {
-      return lastStatusUpdate;
-    }
-
-    public DrillbitEndpoint getEndpoint() {
-      return endpoint;
-    }
-    
-    
-  }
 
   private class RootStatusHandler extends AbstractStatusReporter{
 
@@ -290,7 +244,7 @@ public class QueryManager implements FragmentStatusListener{
       QueryManager.this.statusUpdate(status);
     }
 
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
new file mode 100644
index 0000000..09858d5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
@@ -0,0 +1,33 @@
+package org.apache.drill.exec.work.foreman;
+
+import java.util.Map;
+
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
+
+import com.google.common.collect.Maps;
+
+public class QueryStatus {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryStatus.class);
+
+  public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.
+
+  private final String queryId;
+
+  public QueryStatus(QueryId id, DistributedCache cache){
+    this.queryId = QueryIdHelper.getQueryId(id);
+    cache.getMultiMap(QueryStatus.class);
+
+  }
+
+  void add(FragmentHandle handle, FragmentData data){
+    if(map.put(handle,  data) != null) throw new IllegalStateException();
+  }
+
+  void update(FragmentStatus status){
+    map.get(status.getHandle()).setStatus(status);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
deleted file mode 100644
index c75808d..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/PerformanceTests.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.physical.impl;
-
-import com.google.caliper.runner.CaliperMain;
-
-
-public class PerformanceTests {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PerformanceTests.class);
-
-
-  public static void main(String[] args){
-    CaliperMain.main(TestExecutionAbstractions.class, args);
-    System.out.println("Hello");
-  }
-}


[05/24] status changes

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index e1ced30..79d54a4 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -200,97 +200,6 @@ public final class UserProtos {
   }
 
   /**
-   * Protobuf enum {@code exec.user.QueryType}
-   */
-  public enum QueryType
-      implements com.google.protobuf.ProtocolMessageEnum {
-    /**
-     * <code>SQL = 1;</code>
-     */
-    SQL(0, 1),
-    /**
-     * <code>LOGICAL = 2;</code>
-     */
-    LOGICAL(1, 2),
-    /**
-     * <code>PHYSICAL = 3;</code>
-     */
-    PHYSICAL(2, 3),
-    ;
-
-    /**
-     * <code>SQL = 1;</code>
-     */
-    public static final int SQL_VALUE = 1;
-    /**
-     * <code>LOGICAL = 2;</code>
-     */
-    public static final int LOGICAL_VALUE = 2;
-    /**
-     * <code>PHYSICAL = 3;</code>
-     */
-    public static final int PHYSICAL_VALUE = 3;
-
-
-    public final int getNumber() { return value; }
-
-    public static QueryType valueOf(int value) {
-      switch (value) {
-        case 1: return SQL;
-        case 2: return LOGICAL;
-        case 3: return PHYSICAL;
-        default: return null;
-      }
-    }
-
-    public static com.google.protobuf.Internal.EnumLiteMap<QueryType>
-        internalGetValueMap() {
-      return internalValueMap;
-    }
-    private static com.google.protobuf.Internal.EnumLiteMap<QueryType>
-        internalValueMap =
-          new com.google.protobuf.Internal.EnumLiteMap<QueryType>() {
-            public QueryType findValueByNumber(int number) {
-              return QueryType.valueOf(number);
-            }
-          };
-
-    public final com.google.protobuf.Descriptors.EnumValueDescriptor
-        getValueDescriptor() {
-      return getDescriptor().getValues().get(index);
-    }
-    public final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-    public static final com.google.protobuf.Descriptors.EnumDescriptor
-        getDescriptor() {
-      return org.apache.drill.exec.proto.UserProtos.getDescriptor().getEnumTypes().get(1);
-    }
-
-    private static final QueryType[] VALUES = values();
-
-    public static QueryType valueOf(
-        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-      if (desc.getType() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "EnumValueDescriptor is not for this type.");
-      }
-      return VALUES[desc.getIndex()];
-    }
-
-    private final int index;
-    private final int value;
-
-    private QueryType(int index, int value) {
-      this.index = index;
-      this.value = value;
-    }
-
-    // @@protoc_insertion_point(enum_scope:exec.user.QueryType)
-  }
-
-  /**
    * Protobuf enum {@code exec.user.QueryResultsMode}
    */
   public enum QueryResultsMode
@@ -346,7 +255,7 @@ public final class UserProtos {
     }
     public static final com.google.protobuf.Descriptors.EnumDescriptor
         getDescriptor() {
-      return org.apache.drill.exec.proto.UserProtos.getDescriptor().getEnumTypes().get(2);
+      return org.apache.drill.exec.proto.UserProtos.getDescriptor().getEnumTypes().get(1);
     }
 
     private static final QueryResultsMode[] VALUES = values();
@@ -3264,15 +3173,15 @@ public final class UserProtos {
      */
     org.apache.drill.exec.proto.UserProtos.QueryResultsMode getResultsMode();
 
-    // optional .exec.user.QueryType type = 2;
+    // optional .exec.shared.QueryType type = 2;
     /**
-     * <code>optional .exec.user.QueryType type = 2;</code>
+     * <code>optional .exec.shared.QueryType type = 2;</code>
      */
     boolean hasType();
     /**
-     * <code>optional .exec.user.QueryType type = 2;</code>
+     * <code>optional .exec.shared.QueryType type = 2;</code>
      */
-    org.apache.drill.exec.proto.UserProtos.QueryType getType();
+    org.apache.drill.exec.proto.UserBitShared.QueryType getType();
 
     // optional string plan = 3;
     /**
@@ -3353,7 +3262,7 @@ public final class UserProtos {
             }
             case 16: {
               int rawValue = input.readEnum();
-              org.apache.drill.exec.proto.UserProtos.QueryType value = org.apache.drill.exec.proto.UserProtos.QueryType.valueOf(rawValue);
+              org.apache.drill.exec.proto.UserBitShared.QueryType value = org.apache.drill.exec.proto.UserBitShared.QueryType.valueOf(rawValue);
               if (value == null) {
                 unknownFields.mergeVarintField(2, rawValue);
               } else {
@@ -3423,19 +3332,19 @@ public final class UserProtos {
       return resultsMode_;
     }
 
-    // optional .exec.user.QueryType type = 2;
+    // optional .exec.shared.QueryType type = 2;
     public static final int TYPE_FIELD_NUMBER = 2;
-    private org.apache.drill.exec.proto.UserProtos.QueryType type_;
+    private org.apache.drill.exec.proto.UserBitShared.QueryType type_;
     /**
-     * <code>optional .exec.user.QueryType type = 2;</code>
+     * <code>optional .exec.shared.QueryType type = 2;</code>
      */
     public boolean hasType() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional .exec.user.QueryType type = 2;</code>
+     * <code>optional .exec.shared.QueryType type = 2;</code>
      */
-    public org.apache.drill.exec.proto.UserProtos.QueryType getType() {
+    public org.apache.drill.exec.proto.UserBitShared.QueryType getType() {
       return type_;
     }
 
@@ -3484,7 +3393,7 @@ public final class UserProtos {
 
     private void initFields() {
       resultsMode_ = org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
-      type_ = org.apache.drill.exec.proto.UserProtos.QueryType.SQL;
+      type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
       plan_ = "";
     }
     private byte memoizedIsInitialized = -1;
@@ -3647,7 +3556,7 @@ public final class UserProtos {
         super.clear();
         resultsMode_ = org.apache.drill.exec.proto.UserProtos.QueryResultsMode.STREAM_FULL;
         bitField0_ = (bitField0_ & ~0x00000001);
-        type_ = org.apache.drill.exec.proto.UserProtos.QueryType.SQL;
+        type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
         bitField0_ = (bitField0_ & ~0x00000002);
         plan_ = "";
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -3781,24 +3690,24 @@ public final class UserProtos {
         return this;
       }
 
-      // optional .exec.user.QueryType type = 2;
-      private org.apache.drill.exec.proto.UserProtos.QueryType type_ = org.apache.drill.exec.proto.UserProtos.QueryType.SQL;
+      // optional .exec.shared.QueryType type = 2;
+      private org.apache.drill.exec.proto.UserBitShared.QueryType type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
       /**
-       * <code>optional .exec.user.QueryType type = 2;</code>
+       * <code>optional .exec.shared.QueryType type = 2;</code>
        */
       public boolean hasType() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .exec.user.QueryType type = 2;</code>
+       * <code>optional .exec.shared.QueryType type = 2;</code>
        */
-      public org.apache.drill.exec.proto.UserProtos.QueryType getType() {
+      public org.apache.drill.exec.proto.UserBitShared.QueryType getType() {
         return type_;
       }
       /**
-       * <code>optional .exec.user.QueryType type = 2;</code>
+       * <code>optional .exec.shared.QueryType type = 2;</code>
        */
-      public Builder setType(org.apache.drill.exec.proto.UserProtos.QueryType value) {
+      public Builder setType(org.apache.drill.exec.proto.UserBitShared.QueryType value) {
         if (value == null) {
           throw new NullPointerException();
         }
@@ -3808,11 +3717,11 @@ public final class UserProtos {
         return this;
       }
       /**
-       * <code>optional .exec.user.QueryType type = 2;</code>
+       * <code>optional .exec.shared.QueryType type = 2;</code>
        */
       public Builder clearType() {
         bitField0_ = (bitField0_ & ~0x00000002);
-        type_ = org.apache.drill.exec.proto.UserProtos.QueryType.SQL;
+        type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
         onChanged();
         return this;
       }
@@ -6952,32 +6861,31 @@ public final class UserProtos {
       "redentials\022-\n\nproperties\030\005 \001(\0132\031.exec.us" +
       "er.UserProperties\"S\n\016RequestResults\022&\n\010q",
       "uery_id\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n\021m" +
-      "aximum_responses\030\002 \001(\005\"o\n\010RunQuery\0221\n\014re" +
+      "aximum_responses\030\002 \001(\005\"q\n\010RunQuery\0221\n\014re" +
       "sults_mode\030\001 \001(\0162\033.exec.user.QueryResult" +
-      "sMode\022\"\n\004type\030\002 \001(\0162\024.exec.user.QueryTyp" +
-      "e\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022\023\n" +
-      "\013rpc_version\030\002 \001(\005\"7\n\nNodeStatus\022\017\n\007node" +
-      "_id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\331\003\n\013" +
-      "QueryResult\0226\n\013query_state\030\001 \001(\0162!.exec." +
-      "user.QueryResult.QueryState\022&\n\010query_id\030" +
-      "\002 \001(\0132\024.exec.shared.QueryId\022\025\n\ris_last_c",
-      "hunk\030\003 \001(\010\022\021\n\trow_count\030\004 \001(\005\022\024\n\014records" +
-      "_scan\030\005 \001(\003\022\025\n\rrecords_error\030\006 \001(\003\022\027\n\017su" +
-      "bmission_time\030\007 \001(\003\022*\n\013node_status\030\010 \003(\013" +
-      "2\025.exec.user.NodeStatus\022(\n\005error\030\t \003(\0132\031" +
-      ".exec.shared.DrillPBError\022(\n\003def\030\n \001(\0132\033" +
-      ".exec.shared.RecordBatchDef\022\026\n\016schema_ch" +
-      "anged\030\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000\022\013" +
-      "\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED\020\003" +
-      "\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\270\001\n\007RpcT" +
-      "ype\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002",
-      "\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQ" +
-      "UEST_RESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QUER" +
-      "Y_HANDLE\020\007\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022RE" +
-      "SP_FUNCTION_LIST\020\t*/\n\tQueryType\022\007\n\003SQL\020\001" +
-      "\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*#\n\020QueryResu" +
-      "ltsMode\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.d" +
-      "rill.exec.protoB\nUserProtosH\001"
+      "sMode\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryT" +
+      "ype\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022" +
+      "\023\n\013rpc_version\030\002 \001(\005\"7\n\nNodeStatus\022\017\n\007no" +
+      "de_id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\331\003" +
+      "\n\013QueryResult\0226\n\013query_state\030\001 \001(\0162!.exe" +
+      "c.user.QueryResult.QueryState\022&\n\010query_i" +
+      "d\030\002 \001(\0132\024.exec.shared.QueryId\022\025\n\ris_last",
+      "_chunk\030\003 \001(\010\022\021\n\trow_count\030\004 \001(\005\022\024\n\014recor" +
+      "ds_scan\030\005 \001(\003\022\025\n\rrecords_error\030\006 \001(\003\022\027\n\017" +
+      "submission_time\030\007 \001(\003\022*\n\013node_status\030\010 \003" +
+      "(\0132\025.exec.user.NodeStatus\022(\n\005error\030\t \003(\013" +
+      "2\031.exec.shared.DrillPBError\022(\n\003def\030\n \001(\013" +
+      "2\033.exec.shared.RecordBatchDef\022\026\n\016schema_" +
+      "changed\030\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000" +
+      "\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED" +
+      "\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\270\001\n\007Rp" +
+      "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE",
+      "\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017R" +
+      "EQUEST_RESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QU" +
+      "ERY_HANDLE\020\007\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022" +
+      "RESP_FUNCTION_LIST\020\t*#\n\020QueryResultsMode" +
+      "\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.ex" +
+      "ec.protoB\nUserProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/protobuf/BitControl.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
index 77d7e9d..d9fa4b4 100644
--- a/protocol/src/main/protobuf/BitControl.proto
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -60,6 +60,7 @@ message FragmentStatus {
   optional FragmentHandle handle = 7;
   optional exec.shared.DrillPBError error = 8;
   optional int64 running_time = 9;
+  optional exec.shared.MinorFragmentProfile profile = 10;
 }
 
 message PlanFragment {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index 52ac627..7809fdc 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -55,16 +55,10 @@ message RequestResults {
 
 message RunQuery {
   optional QueryResultsMode results_mode = 1;
-  optional QueryType type = 2;
+  optional exec.shared.QueryType type = 2;
   optional string plan = 3;
 }
 
-enum QueryType {
-  SQL = 1;
-  LOGICAL = 2;
-  PHYSICAL = 3;
-}
-
 enum QueryResultsMode {
 	STREAM_FULL = 1; // Server will inform the client regularly on the status of the query. Once the query is completed, service will inform the client as each query chunk is made available.
 	// STREAM_FIRST = 2; // Server will inform the client regularly on the status of the query.  Once the query is completed, server will inform the client of the first query chunk.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index a3a734b..c379112 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -14,6 +14,12 @@ enum RpcChannel {
   USER = 2;
 }
 
+enum QueryType {
+  SQL = 1;
+  LOGICAL = 2;
+  PHYSICAL = 3;
+}
+
 message UserCredentials {
   optional string user_name = 1;
 }
@@ -66,3 +72,85 @@ message SerializedField {
   optional int32 group_count = 6; // number of groups.  (number of repeated records)
   optional int32 buffer_length = 7;
 }
+
+
+
+
+message QueryProfile {
+  optional QueryId id = 1;
+  optional QueryType type = 2;
+  optional int64 start = 3;
+  optional int64 end = 4;
+  optional string query = 5;
+  repeated MajorFragmentProfile fragment_profile = 6;
+}
+
+message MajorFragmentProfile {
+  optional int32 major_fragment_id = 1;
+  repeated MinorFragmentProfile minor_fragment_profile = 2;
+} 
+
+message MinorFragmentProfile {
+  optional int32 minor_fragment_id = 1;
+  repeated OperatorProfile operator_profile = 2; 
+  optional int64 start_time = 3;
+  optional int64 end_time = 4;  
+}
+
+message OperatorProfile {
+  repeated StreamProfile input_profile = 1;
+  optional StreamProfile output_profile = 2;
+  optional int32 operator_id = 3;
+  optional int32 operator_type = 4;
+  optional int64 setup_nanos = 5;
+  optional int64 process_nanos = 6;
+  optional int64 local_memory_allocated = 7;  
+  repeated MetricValue metric = 8;
+}
+
+message StreamProfile {
+  optional int64 records = 1;
+  optional int64 batches = 2;
+  optional int64 schemas = 3;
+}
+
+message MetricValue {
+  optional int32 metric_id = 1;
+  optional int64 long_value = 2;
+  optional double double_value = 3;
+}
+
+
+enum CoreOperatorType {
+  SINGLE_SENDER = 0;
+  BROADCAST_SENDER = 1;
+  FILTER = 2;
+  HASH_AGGREGATE = 3;
+  HASH_JOIN = 4;
+  MERGE_JOIN = 5;
+  HASH_PARTITION_SENDER = 6;
+  LIMIT = 7;
+  MERGING_RECEIVER = 8;
+  ORDERED_PARTITION_SENDER = 9;
+  PROJECT = 10;
+  RANDOM_RECEIVER = 11;
+  RANGE_SENDER = 12;
+  SCREEN = 13;
+  SELECTION_VECTOR_REMOVER = 14;
+  STREAMING_AGGREGATE = 15;
+  TOP_N_SORT = 16;
+  EXTERNAL_SORT = 17;
+  TRACE = 18;
+  UNION = 19;
+  OLD_SORT = 20;
+  PARQUET_ROW_GROUP_SCAN = 21;
+  HIVE_SUB_SCAN = 22;
+  SYSTEM_TABLE_SCAN = 23;
+  MOCK_SUB_SCAN = 24;
+  PARQUET_WRITER = 25;
+  DIRECT_SUB_SCAN = 26;
+  TEXT_WRITER = 27;
+  TEXT_SUB_SCAN = 28;
+  JSON_SUB_SCAN = 29;
+  INFO_SCHEMA_SUB_SCAN = 30;
+}


[12/24] git commit: merge fixes

Posted by ja...@apache.org.
merge fixes


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

Branch: refs/heads/diagnostics2
Commit: e6121d05ed3e5afe776486383a90e66e5af99351
Parents: 8621b68
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon May 19 08:58:59 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:12:34 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/store/hbase/HBaseSubScan.java    |  5 +++
 exec/java-exec/pom.xml                          |  5 +++
 .../src/main/codegen/templates/TypeHelper.java  |  1 -
 .../drill/exec/cache/JacksonSerializable.java   | 20 ++++++++--
 .../cache/VectorAccessibleSerializable.java     |  5 +--
 .../drill/exec/cache/hazel/HazelCache.java      |  2 +
 .../drill/exec/cache/infinispan/ICache.java     | 17 ++++++++-
 .../drill/exec/cache/local/LocalCache.java      |  2 +
 .../apache/drill/exec/ops/OperatorStats.java    |  4 +-
 .../exec/physical/impl/join/HashJoinBatch.java  |  4 +-
 .../drill/exec/record/AbstractRecordBatch.java  | 27 ++++++++------
 .../exec/record/AbstractSingleRecordBatch.java  |  4 +-
 .../org/apache/drill/exec/server/Drillbit.java  | 39 ++++++++++++--------
 .../drill/exec/server/rest/DrillRestServer.java | 28 ++++++++++++++
 .../drill/exec/server/rest/DrillRoot.java       | 15 ++++++++
 .../apache/drill/exec/work/foreman/Foreman.java |  2 +-
 .../drill/exec/work/foreman/QueryManager.java   |  4 +-
 .../drill/exec/work/foreman/QueryStatus.java    |  2 +-
 .../java/org/apache/drill/BaseTestQuery.java    |  2 +-
 .../java/org/apache/drill/PlanTestBase.java     |  2 +-
 .../drill/exec/cache/TestVectorCache.java       | 32 ++++++++--------
 .../exec/client/DrillClientSystemTest.java      |  2 +-
 .../exec/fn/impl/TestAggregateFunction.java     |  5 +--
 .../drill/exec/fn/impl/TestDateFunctions.java   |  2 +-
 .../drill/exec/fn/impl/TestMultiInputAdd.java   |  2 +-
 .../exec/physical/impl/TestCastFunctions.java   |  2 +-
 .../physical/impl/TestCastVarCharToBigInt.java  |  2 +-
 .../drill/exec/physical/impl/TestDecimal.java   | 14 +++----
 .../impl/TestDistributedFragmentRun.java        |  8 +---
 .../physical/impl/TestExtractFunctions.java     |  2 +-
 .../physical/impl/TestHashToRandomExchange.java |  2 +-
 .../exec/physical/impl/TestOptiqPlans.java      | 10 ++---
 .../physical/impl/TestReverseImplicitCast.java  |  2 +-
 .../physical/impl/TestSimpleFragmentRun.java    |  2 +-
 .../exec/physical/impl/TestUnionExchange.java   |  2 +-
 .../exec/physical/impl/TopN/TestSimpleTopN.java |  2 +-
 .../exec/physical/impl/join/TestHashJoin.java   |  8 ++--
 .../exec/physical/impl/join/TestMergeJoin.java  |  6 +--
 .../impl/join/TestMergeJoinMulCondition.java    |  2 +-
 .../impl/mergereceiver/TestMergingReceiver.java |  6 +--
 .../TestOrderedPartitionExchange.java           |  2 +-
 .../impl/xsort/TestSimpleExternalSort.java      |  6 +--
 .../drill/exec/record/vector/TestDateTypes.java | 12 +++---
 .../store/parquet/ParquetRecordReaderTest.java  |  8 +---
 .../store/parquet/TestParquetPhysicalPlan.java  |  4 +-
 .../exec/store/text/TextRecordReaderTest.java   |  2 +-
 .../exec/work/batch/TestSpoolingBuffer.java     |  2 +-
 exec/java-exec/src/test/resources/logback.xml   |  2 +-
 48 files changed, 215 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
index 3f20087..d9f2b7c 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
@@ -209,4 +209,9 @@ public class HBaseSubScan extends AbstractBase implements SubScan {
 
   }
 
+  @Override
+  public int getOperatorType() {
+    return 1001;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 6d11614..83220ab 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -92,6 +92,11 @@
       <version>2.8</version>
     </dependency>
     <dependency>
+        <groupId>org.glassfish.jersey.ext</groupId>
+        <artifactId>jersey-mvc-freemarker</artifactId>
+        <version>2.8</version>
+    </dependency>    
+    <dependency>
       <groupId>net.hydromatic</groupId>
       <artifactId>optiq-core</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/codegen/templates/TypeHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/TypeHelper.java b/exec/java-exec/src/main/codegen/templates/TypeHelper.java
index f2d66d2..5f24a30 100644
--- a/exec/java-exec/src/main/codegen/templates/TypeHelper.java
+++ b/exec/java-exec/src/main/codegen/templates/TypeHelper.java
@@ -67,7 +67,6 @@ public class TypeHelper {
     </#list>
     </#list>
     case MAP:
-    case REPEATMAP:
     case LIST:
       return new GenericAccessor(vector);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonSerializable.java
index 831db84..247c79e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/JacksonSerializable.java
@@ -17,8 +17,12 @@
  */
 package org.apache.drill.exec.cache;
 
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.io.OutputStream;
 
 import com.hazelcast.nio.ObjectDataInput;
@@ -33,17 +37,27 @@ public abstract class JacksonSerializable implements DrillSerializable{
   }
 
   @Override
-  public void readData(ObjectDataInput input) throws IOException {
+  public void writeExternal(ObjectOutput out) throws IOException {
     fail();
   }
 
   @Override
-  public void readFromStream(InputStream input) throws IOException {
+  public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+    fail();
+  }
+
+  @Override
+  public void read(DataInput input) throws IOException {
     fail();
   }
 
   @Override
-  public void writeData(ObjectDataOutput output) throws IOException {
+  public void write(DataOutput output) throws IOException {
+    fail();
+  }
+
+  @Override
+  public void readFromStream(InputStream input) throws IOException {
     fail();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
index bb3f527..63ed592 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/VectorAccessibleSerializable.java
@@ -19,8 +19,6 @@ package org.apache.drill.exec.cache;
 
 import io.netty.buffer.ByteBuf;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -30,10 +28,9 @@ import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.metrics.DrillMetrics;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorContainer;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
index 06518b6..f83456b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/hazel/HazelCache.java
@@ -31,6 +31,8 @@ import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.DistributedMap;
 import org.apache.drill.exec.cache.DistributedMultiMap;
 import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.cache.JacksonAdvancedSerializer;
+import org.apache.drill.exec.cache.SerializationDefinition;
 import org.apache.drill.exec.cache.VectorAccessibleSerializable;
 import org.apache.drill.exec.cache.hazel.ProtoBufImpl.HWorkQueueStatus;
 import org.apache.drill.exec.cache.hazel.ProtoBufImpl.HandlePlan;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
index b26be7d..5533dd7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
@@ -19,8 +19,10 @@ package org.apache.drill.exec.cache.infinispan;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.config.DrillConfig;
@@ -114,13 +116,19 @@ public class ICache implements DistributedCache{
     return new IMulti<V>(cache, clazz);
   }
 
+
   @Override
-  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
-    Cache<String, V> c = manager.getCache(clazz.getName());
+  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
+    Cache<String, V> c = manager.getCache(name);
     return new IMap<V>(c);
   }
 
   @Override
+  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
+    return getNamedMap(clazz.getName(), clazz);
+  }
+
+  @Override
   public Counter getCounter(String name) {
     return new JGroupsCounter(counters.getOrCreateCounter(name, 0));
   }
@@ -180,6 +188,11 @@ public class ICache implements DistributedCache{
       cache.putIfAbsent(key, value, ttl, timeUnit);
     }
 
+    @Override
+    public Iterator<Entry<String, V>> iterator() {
+      return cache.entrySet().iterator();
+    }
+
   }
 
   private class IMulti<V extends DrillSerializable> implements DistributedMultiMap<V>{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
index e66cc90..942e09e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
@@ -30,6 +30,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.DataInputInputStream;
 import org.apache.drill.common.util.DataOutputOutputStream;
 import org.apache.drill.exec.cache.Counter;
@@ -37,6 +38,7 @@ import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.DistributedMap;
 import org.apache.drill.exec.cache.DistributedMultiMap;
 import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.cache.JacksonSerializable;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
index dc463b3..bd00560 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -61,6 +61,7 @@ public class OperatorStats {
     this.operatorType = operatorType;
     this.recordsReceivedByInput = new long[inputCount];
     this.batchesReceivedByInput = new long[inputCount];
+    this.schemaCountByInput = new long[inputCount];
   }
 
   public void startSetup() {
@@ -86,6 +87,7 @@ public class OperatorStats {
   public void stopProcessing() {
     assert inProcessing;
     processingNanos += System.nanoTime() - processingMark;
+    inProcessing = false;
   }
 
   public void batchReceived(int inputIndex, long records, boolean newSchema) {
@@ -97,7 +99,7 @@ public class OperatorStats {
   }
 
   public OperatorProfile getProfile() {
-    OperatorProfile.Builder b = OperatorProfile //
+    final OperatorProfile.Builder b = OperatorProfile //
         .newBuilder() //
         .setOperatorType(operatorType) //
         .setOperatorId(operatorId) //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
index 6b768c9..3b8d713 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/HashJoinBatch.java
@@ -221,11 +221,13 @@ public class HashJoinBatch extends AbstractRecordBatch<HashJoinPOP> {
             return IterOutcome.NONE;
 
         } catch (ClassTransformationException | SchemaChangeException | IOException e) {
-            stats.stopProcessing();
             context.fail(e);
             killIncoming();
             return IterOutcome.STOP;
+        } finally{
+          stats.stopProcessing();
         }
+
     }
 
     public void setupHashTable() throws IOException, SchemaChangeException, ClassTransformationException {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index b6b4c33..53b223e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -65,19 +65,24 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
 
   public final IterOutcome next(int inputIndex, RecordBatch b){
     stats.stopProcessing();
-    IterOutcome next = b.next();
-
-    switch(next){
-    case OK_NEW_SCHEMA:
-      stats.batchReceived(inputIndex, b.getRecordCount(), true);
-      break;
-    case OK:
-      stats.batchReceived(inputIndex, b.getRecordCount(), false);
-      break;
+    try{
+      IterOutcome next = b.next();
+
+      switch(next){
+      case OK_NEW_SCHEMA:
+        stats.batchReceived(inputIndex, b.getRecordCount(), true);
+        break;
+      case OK:
+        stats.batchReceived(inputIndex, b.getRecordCount(), false);
+        break;
+      }
+
+      return next;
+
+    }finally{
+      stats.startProcessing();
     }
 
-    stats.startProcessing();
-    return next;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
index 13e4ac8..d897a78 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractSingleRecordBatch.java
@@ -58,13 +58,13 @@ public abstract class AbstractSingleRecordBatch<T extends PhysicalOperator> exte
         try{
           stats.startSetup();
           setupNewSchema();
-          stats.stopSetup();
         }catch(SchemaChangeException ex){
-          stats.stopSetup();
           kill();
           logger.error("Failure during query", ex);
           context.fail(ex);
           return IterOutcome.STOP;
+        }finally{
+          stats.stopSetup();
         }
         // fall through.
       case OK:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 0e3181d..eba6e92 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -22,16 +22,20 @@ import java.io.Closeable;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
-import org.apache.drill.exec.cache.hazel.HazelCache;
+import org.apache.drill.exec.cache.infinispan.ICache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.rest.DrillRestServer;
 import org.apache.drill.exec.service.ServiceEngine;
 import org.apache.drill.exec.work.WorkManager;
 import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.glassfish.jersey.server.ServerProperties;
+import org.glassfish.jersey.servlet.ServletContainer;
 
 import com.google.common.io.Closeables;
 
@@ -77,27 +81,32 @@ public class Drillbit implements Closeable{
   private volatile RegistrationHandle handle;
 
   public Drillbit(DrillConfig config, RemoteServiceSet serviceSet) throws Exception {
+
+    this.context = new BootStrapContext(config);
+    this.manager = new WorkManager(context);
+    this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
+    this.embeddedJetty = new Server(8047);
+
     if(serviceSet != null){
-      this.context = new BootStrapContext(config);
-      this.manager = new WorkManager(context);
       this.coord = serviceSet.getCoordinator();
-      this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
       this.cache = serviceSet.getCache();
     }else{
       Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
-      this.context = new BootStrapContext(config);
-      this.manager = new WorkManager(context);
       this.coord = new ZKClusterCoordinator(config);
-      this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
-      this.cache = new HazelCache(config, context.getAllocator());
+      this.cache = new ICache(config, context.getAllocator());
+//      this.cache = new HazelCache(config, context.getAllocator());
     }
-    this.embeddedJetty = new Server(474747);
   }
 
-  private void setupJetty(){
-    ServletHandler handler = new ServletHandler();
-    embeddedJetty.setHandler(handler);
-
+  private void startJetty() throws Exception{
+    ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS);
+    context.setContextPath("/");
+    embeddedJetty.setHandler(context);
+    ServletHolder h = new ServletHolder(new ServletContainer(new DrillRestServer(manager)));
+//    h.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.drill.exec.server");
+    h.setInitOrder(1);
+    context.addServlet(h, "/*");
+    embeddedJetty.start();
   }
 
 
@@ -110,7 +119,7 @@ public class Drillbit implements Closeable{
     manager.getContext().getStorage().init();
     manager.getContext().getOptionManager().init();
     handle = coord.register(md);
-    embeddedJetty.start();
+    startJetty();
   }
 
   public void close() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
new file mode 100644
index 0000000..dfc10c2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
@@ -0,0 +1,28 @@
+package org.apache.drill.exec.server.rest;
+
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.server.HelloResource;
+import org.apache.drill.exec.work.WorkManager;
+import org.glassfish.hk2.utilities.binding.AbstractBinder;
+import org.glassfish.jersey.jackson.JacksonFeature;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.server.ServerProperties;
+import org.glassfish.jersey.server.mvc.freemarker.FreemarkerMvcFeature;
+
+public class DrillRestServer extends ResourceConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRestServer.class);
+
+  public DrillRestServer(final WorkManager workManager) {
+//    registerClasses(HelloResource.class);
+    register(JacksonFeature.class);
+    register(FreemarkerMvcFeature.class);
+    property(ServerProperties.METAINF_SERVICES_LOOKUP_DISABLE, true);
+    register(new AbstractBinder() {
+      @Override
+      protected void configure() {
+        bind(workManager).to(WorkManager.class);
+        bind(new DrillClient()).to(DrillClient.class);
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
new file mode 100644
index 0000000..dae6b43
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
@@ -0,0 +1,15 @@
+package org.apache.drill.exec.server.rest;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+
+@Path("/")
+public class DrillRoot {
+    @GET
+    @Produces("text/plain")
+    public String getHello() {
+      return "hello world";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 3dbb98e..3e8d3e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -99,7 +99,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     this.queryRequest = queryRequest;
     this.context = new QueryContext(connection.getSession(), queryId, dContext);
     this.initiatingClient = connection;
-    this.fragmentManager = new QueryManager(bee.getContext().getCache(), new ForemanManagerListener(), dContext.getController());
+    this.fragmentManager = new QueryManager(queryId, bee.getContext().getCache(), new ForemanManagerListener(), dContext.getController());
     this.bee = bee;
 
     this.state = new AtomicState<QueryState>(QueryState.PENDING) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index c9c769a..62d9375 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -61,12 +61,12 @@ public class QueryManager implements FragmentStatusListener{
   private FragmentExecutor rootRunner;
   private volatile QueryId queryId;
 
-  public QueryManager(DistributedCache cache, ForemanManagerListener foreman, Controller controller) {
+  public QueryManager(QueryId queryId, DistributedCache cache, ForemanManagerListener foreman, Controller controller) {
     super();
     this.foreman = foreman;
     this.controller = controller;
     this.remainingFragmentCount = new AtomicInteger(0);
-    this.status = new QueryStatus(cache);
+    this.status = new QueryStatus(queryId, cache);
   }
 
   public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments, List<PlanFragment> intermediateFragments) throws ExecutionSetupException{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
index 09858d5..64ec671 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
@@ -19,7 +19,7 @@ public class QueryStatus {
 
   public QueryStatus(QueryId id, DistributedCache cache){
     this.queryId = QueryIdHelper.getQueryId(id);
-    cache.getMultiMap(QueryStatus.class);
+//    cache.getMultiMap(QueryStatus.class);
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 062511e..b18141c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.client.QuerySubmitter;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
index 356b63d..89452a1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanTestBase.java
@@ -33,7 +33,7 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.eigenbase.sql.SqlExplain.Depth;
 import org.eigenbase.sql.SqlExplainLevel;
 
-import com.google.monitoring.runtime.instrumentation.common.com.google.common.base.Strings;
+import com.google.common.base.Strings;
 
 public class PlanTestBase extends BaseTestQuery {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
index 7686614..49f41e3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
@@ -69,14 +69,14 @@ public class TestVectorCache extends ExecTest{
       vectorList.add(intVector);
       vectorList.add(binVector);
 
-      intVector.getMutator().set(0, 0);
-      binVector.getMutator().set(0, "ZERO".getBytes());
-      intVector.getMutator().set(1, 1);
-      binVector.getMutator().set(1, "ONE".getBytes());
-      intVector.getMutator().set(2, 2);
-      binVector.getMutator().set(2, "TWO".getBytes());
-      intVector.getMutator().set(3, 3);
-      binVector.getMutator().set(3, "THREE".getBytes());
+      intVector.getMutator().setSafe(0, 0);
+      binVector.getMutator().setSafe(0, "ZERO".getBytes());
+      intVector.getMutator().setSafe(1, 1);
+      binVector.getMutator().setSafe(1, "ONE".getBytes());
+      intVector.getMutator().setSafe(2, 2);
+      binVector.getMutator().setSafe(2, "TWO".getBytes());
+      intVector.getMutator().setSafe(3, 3);
+      binVector.getMutator().setSafe(3, "THREE".getBytes());
       intVector.getMutator().setValueCount(4);
       binVector.getMutator().setValueCount(4);
 
@@ -110,14 +110,14 @@ public class TestVectorCache extends ExecTest{
 
   }
 
-  @Test
-  public void testHazelVectorCache() throws Exception {
-    DrillConfig c = DrillConfig.create();
-    HazelCache cache = new HazelCache(c, new TopLevelAllocator());
-    cache.run();
-    testCache(c, cache);
-    cache.close();
-  }
+//  @Test
+//  public void testHazelVectorCache() throws Exception {
+//    DrillConfig c = DrillConfig.create();
+//    HazelCache cache = new HazelCache(c, new TopLevelAllocator());
+//    cache.run();
+//    testCache(c, cache);
+//    cache.close();
+//  }
 
   @Test
   public void testICache() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index f3789d0..ec8a5e6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.client;
 import java.util.List;
 
 import org.apache.drill.exec.DrillSystemTestBase;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.junit.After;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
index 1159982..ffb372d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestAggregateFunction.java
@@ -25,13 +25,12 @@ import java.util.List;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.NullableBigIntVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
@@ -48,7 +47,7 @@ public class TestAggregateFunction extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/functions/test_stddev_variance.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/simple_stddev_variance_input.json"));
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
index 8d065d4..db026b6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
@@ -48,7 +48,7 @@ public class TestDateFunctions extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile(physicalPlan), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", resourceFile));
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
index d4714c0..8e6813c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMultiInputAdd.java
@@ -62,7 +62,7 @@ public class TestMultiInputAdd extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/functions/multi_input_add_test.json"), Charsets.UTF_8));
 
             RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
index 1e88774..702edb6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
@@ -428,7 +428,7 @@ public class TestCastFunctions extends PopUnitTestBase{
       bit.run();
 
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarCharNull.json"), Charsets.UTF_8).replace("#{TEST_FILE}", "/jsoninput/input1.json"));
 
       QueryResultBatch batch = results.get(0);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
index 2dfeb5d..c0fd23a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
@@ -51,7 +51,7 @@ public class TestCastVarCharToBigInt extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/functions/cast/test_cast_varchar_to_bigint.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/scan_json_test_cast.json")
             );

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
index d35a371..2df4d18 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDecimal.java
@@ -55,7 +55,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/cast_simple_decimal.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
@@ -100,7 +100,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/cast_float_decimal.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
@@ -145,7 +145,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/simple_decimal_arithmetic.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );
@@ -196,7 +196,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_complex.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_complex_decimal.json")
             );
@@ -243,7 +243,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_sort_complex.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_sort_complex_decimal.json")
             );
@@ -295,7 +295,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_dense_sparse.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_complex_decimal.json")
             );
@@ -343,7 +343,7 @@ public class TestDecimal extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/decimal/test_decimal_sparse_dense_dense.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/input_simple_decimal.json")
             );

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index b18d996..3de9c63 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -17,21 +17,17 @@
  */
 package org.apache.drill.exec.physical.impl;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
 
 import java.util.List;
 
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
-import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
 
 import com.google.common.base.Charsets;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
index a4f10d0..7a983ad 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
@@ -112,7 +112,7 @@ public class TestExtractFunctions extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
         Files.toString(FileUtils.getResourceAsFile("/functions/extractFrom.json"), Charsets.UTF_8)
         .replace("#{TEST_TYPE}", fromType)
         .replace("#{TEST_FILE}", testDataFile)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
index 6ed2d25..f769f1e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
@@ -49,7 +49,7 @@ public class TestHashToRandomExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/hash_exchange.json"),
               Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
index fe82cd1..9a02d0e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
@@ -132,7 +132,7 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_filter.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       for (QueryResultBatch b : results) {
@@ -162,7 +162,7 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_join.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       for (QueryResultBatch b : results) {
@@ -192,7 +192,7 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_string_filter.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       for (QueryResultBatch b : results) {
@@ -232,7 +232,7 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.LOGICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.LOGICAL,
           Resources.toString(Resources.getResource("logical_json_scan.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       for (QueryResultBatch b : results) {
@@ -272,7 +272,7 @@ public class TestOptiqPlans extends ExecTest {
         DrillClient client = new DrillClient(config, serviceSet.getCoordinator());) {
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Resources.toString(Resources.getResource("physical_order_varbinary.json"), Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       for (QueryResultBatch b : results) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
index 62afa12..c8209ac 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestReverseImplicitCast.java
@@ -54,7 +54,7 @@ public class TestReverseImplicitCast extends PopUnitTestBase {
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/functions/cast/two_way_implicit_cast.json"), Charsets.UTF_8));
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index c8126c4..7ea01a6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -30,7 +30,7 @@ import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.SchemaDefProtos;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
index 6de4ada..6240e81 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
@@ -47,7 +47,7 @@ public class TestUnionExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/union_exchange.json"),
               Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
index 7ff165b..eb290e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
@@ -72,7 +72,7 @@ public class TestSimpleTopN extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/topN/one_key_sort.json"),
                       Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
index 13180db..d4a86ca 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestHashJoin.java
@@ -127,7 +127,7 @@ public class TestHashJoin extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/join/hash_join.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE_1}", FileUtils.getResourceAsFile("/build_side_input.json").toURI().toString())
                             .replace("#{TEST_FILE_2}", FileUtils.getResourceAsFile("/probe_side_input.json").toURI().toString()));
@@ -170,7 +170,7 @@ public class TestHashJoin extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/join/hj_exchanges.json"), Charsets.UTF_8));
 
             int count = 0;
@@ -197,7 +197,7 @@ public class TestHashJoin extends PopUnitTestBase{
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/join/hj_multi_condition_join.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE_1}", FileUtils.getResourceAsFile("/build_side_input.json").toURI().toString())
                             .replace("#{TEST_FILE_2}", FileUtils.getResourceAsFile("/probe_side_input.json").toURI().toString()));
@@ -244,7 +244,7 @@ public class TestHashJoin extends PopUnitTestBase{
       // run query.
       bit.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/hj_exchanges1.json"), Charsets.UTF_8));
 
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 1754c5c..5ca7e72 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -312,7 +312,7 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
                       Charsets.UTF_8)
                       .replace("${JOIN_TYPE}", "INNER"));
@@ -335,7 +335,7 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
               Charsets.UTF_8)
               .replace("${JOIN_TYPE}", "LEFT"));
@@ -358,7 +358,7 @@ public class TestMergeJoin extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
                       Charsets.UTF_8)
                       .replace("${JOIN_TYPE}", "RIGHT"));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
index 1994987..d9cfa5c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
@@ -58,7 +58,7 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/mj_multi_condition.json"),
               Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index b05c778..14d6fe2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -54,7 +54,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
         Files.toString(FileUtils.getResourceAsFile("/mergerecv/merging_receiver.json"),
           Charsets.UTF_8));
       int count = 0;
@@ -97,7 +97,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                                                         Files.toString(FileUtils.getResourceAsFile("/mergerecv/multiple_providers.json"),
                                                                         Charsets.UTF_8));
       int count = 0;
@@ -145,7 +145,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                                                         Files.toString(FileUtils.getResourceAsFile("/mergerecv/empty_batch.json"),
                                                                         Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
index 7ea96e9..c29d99e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
@@ -70,7 +70,7 @@ public class TestOrderedPartitionExchange extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/sender/ordered_exchange.json"),
               Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index 587e562..42fe703 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -61,7 +61,7 @@ public class TestSimpleExternalSort extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/xsort/one_key_sort_descending.json"),
                       Charsets.UTF_8));
       int count = 0;
@@ -114,7 +114,7 @@ public class TestSimpleExternalSort extends PopUnitTestBase {
       bit1.run();
       bit2.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/xsort/one_key_sort_descending.json"),
                       Charsets.UTF_8));
       int count = 0;
@@ -164,7 +164,7 @@ public class TestSimpleExternalSort extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/xsort/oom_sort_test.json"),
                       Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
index 661cf67..adae024 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
@@ -80,7 +80,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_date.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
@@ -114,7 +114,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_sort_date.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
@@ -148,7 +148,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_timestamp.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
@@ -182,7 +182,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_interval.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_interval.json"));
 
@@ -240,7 +240,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_all_date_literals.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 
@@ -280,7 +280,7 @@ public class TestDateTypes extends PopUnitTestBase {
             // run query.
             bit.run();
             client.connect();
-            List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+            List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
                     Files.toString(FileUtils.getResourceAsFile("/record/vector/test_date_add.json"), Charsets.UTF_8)
                             .replace("#{TEST_FILE}", "/test_simple_date.json"));
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 5d2c859..d9ba32d 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -37,7 +37,6 @@ import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
@@ -45,14 +44,11 @@ import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.BitControl;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.UserServer;
-import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.CachedSingleFileSystem;
 import org.apache.drill.exec.store.TestOutputMutator;
 import org.apache.drill.exec.vector.ValueVector;
@@ -188,7 +184,7 @@ public class ParquetRecordReaderTest extends BaseTestQuery{
     ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
     TestFileGenerator.populateFieldInfoMap(props);
     ParquetResultListener resultListener = new ParquetResultListener(getAllocator(), props, numberOfTimesRead, true);
-    testWithListener(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+    testWithListener(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
     resultListener.getResults();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
index c569932..797c63c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
@@ -58,7 +58,7 @@ public class TestParquetPhysicalPlan extends ExecTest {
     try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
       RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
       int count = 0;
       for (QueryResultBatch b : results) {
@@ -124,7 +124,7 @@ public class TestParquetPhysicalPlan extends ExecTest {
       ParquetResultsListener listener = new ParquetResultsListener();
       Stopwatch watch = new Stopwatch();
       watch.start();
-      client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8), listener);
+      client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8), listener);
       System.out.println(String.format("Got %d total records in %d seconds", listener.await(), watch.elapsed(TimeUnit.SECONDS)));
       client.close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
index 3abd6a9..3091914 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TextRecordReaderTest.java
@@ -68,7 +68,7 @@ public class TextRecordReaderTest extends PopUnitTestBase {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(
                       FileUtils.getResourceAsFile("/store/text/test.json"), Charsets.UTF_8)
                       .replace("#{DATA_FILE}", FileUtils.getResourceAsFile("/store/text/data/regions.csv").toURI().toString()));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
index e2fc3f7..28d8e39 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
@@ -49,7 +49,7 @@ public class TestSpoolingBuffer extends ExecTest {
 
       bit1.run();
       client.connect();
-      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+      List<QueryResultBatch> results = client.runQuery(org.apache.drill.exec.proto.UserBitShared.QueryType.PHYSICAL,
               Files.toString(FileUtils.getResourceAsFile("/work/batch/multiple_exchange.json"),
                       Charsets.UTF_8));
       int count = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e6121d05/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/logback.xml b/exec/java-exec/src/test/resources/logback.xml
index b13bc58..cd3d971 100644
--- a/exec/java-exec/src/test/resources/logback.xml
+++ b/exec/java-exec/src/test/resources/logback.xml
@@ -42,7 +42,7 @@
 <!--   </logger> -->
 
 	<root>
-		<level value="debug" />
+		<level value="info" />
 		<appender-ref ref="STDOUT" />
 	</root>
 


Re: [10/24] ispan

Posted by Jacques Nadeau <ja...@apache.org>.
I always wanted to use it but initially it had an incompatible license.
 They switched to APL2 with v6.

Infinispan backs the JBoss distributed cache and isn't defeatured the way
Hazelcast is because it isn't RedHat's only product.  I'm experimenting
with it for comparison purposes.  The other nice about Infinispan is that
is built on Jgroups, which is very solid and has real world uses in larger
clusters.  It also could ultimately replace our drillbit control
communication backplane, making things like broadcasts much cleaner.


On Wed, May 21, 2014 at 7:56 PM, Timothy Chen <tn...@gmail.com> wrote:

> Any motivation for using inifinispan instead of hazel cast?
>
> Tim
>
> Sent from my iPhone
>
> > On May 21, 2014, at 6:14 PM, jacques@apache.org wrote:
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> > new file mode 100644
> > index 0000000..46d4eca
> > --- /dev/null
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> > @@ -0,0 +1,66 @@
> > +/**
> > + * 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.drill.exec.cache.infinispan;
> > +
> > +import org.infinispan.marshall.core.MarshalledEntry;
> > +import org.infinispan.persistence.spi.ExternalStore;
> > +import org.infinispan.persistence.spi.InitializationContext;
> > +
> > +/**
> > + * Stores the cached objects in zookeeper.  Objects are stored in
> /start/cache_name/key_name = data
> > + * @param <K>
> > + * @param <V>
> > + */
> > +public class ZookeeperCacheStore<K, V> implements ExternalStore<K, V>{
> > +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ZookeeperCacheStore.class);
> > +
> > +  private String cacheName;
> > +
> > +  @Override
> > +  public void init(InitializationContext ctx) {
> > +    ctx.getConfiguration();
> > +
> > +  }
> > +
> > +  @Override
> > +  public MarshalledEntry<K, V> load(K key) {
> > +    return null;
> > +  }
> > +
> > +  @Override
> > +  public boolean contains(K key) {
> > +    return false;
> > +  }
> > +
> > +  @Override
> > +  public void start() {
> > +  }
> > +
> > +  @Override
> > +  public void stop() {
> > +  }
> > +
> > +  @Override
> > +  public void write(MarshalledEntry<K, V> entry) {
> > +  }
> > +
> > +  @Override
> > +  public boolean delete(K key) {
> > +    return false;
> > +  }
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> > new file mode 100644
> > index 0000000..e66cc90
> > --- /dev/null
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> > @@ -0,0 +1,309 @@
> > +/**
> > + * 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.drill.exec.cache.local;
> > +
> > +import java.io.IOException;
> > +import java.io.InputStream;
> > +import java.io.OutputStream;
> > +import java.lang.reflect.InvocationTargetException;
> > +import java.util.Collection;
> > +import java.util.Iterator;
> > +import java.util.List;
> > +import java.util.Map;
> > +import java.util.Map.Entry;
> > +import java.util.concurrent.ConcurrentMap;
> > +import java.util.concurrent.TimeUnit;
> > +import java.util.concurrent.atomic.AtomicLong;
> > +
> > +import org.apache.drill.common.util.DataInputInputStream;
> > +import org.apache.drill.common.util.DataOutputOutputStream;
> > +import org.apache.drill.exec.cache.Counter;
> > +import org.apache.drill.exec.cache.DistributedCache;
> > +import org.apache.drill.exec.cache.DistributedMap;
> > +import org.apache.drill.exec.cache.DistributedMultiMap;
> > +import org.apache.drill.exec.cache.DrillSerializable;
> > +import org.apache.drill.exec.exception.DrillbitStartupException;
> > +import org.apache.drill.exec.memory.BufferAllocator;
> > +import org.apache.drill.exec.memory.TopLevelAllocator;
> > +import org.apache.drill.exec.proto.BitControl.PlanFragment;
> > +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> > +
> > +import com.fasterxml.jackson.databind.ObjectMapper;
> > +import com.google.common.collect.ArrayListMultimap;
> > +import com.google.common.collect.Lists;
> > +import com.google.common.collect.Maps;
> > +import com.google.common.io.ByteArrayDataInput;
> > +import com.google.common.io.ByteArrayDataOutput;
> > +import com.google.common.io.ByteStreams;
> > +
> > +public class LocalCache implements DistributedCache {
> > +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(LocalCache.class);
> > +
> > +  private volatile Map<FragmentHandle, PlanFragment> handles;
> > +  private volatile ConcurrentMap<String, DistributedMap<?>> namedMaps;
> > +  private volatile ConcurrentMap<Class<?>, DistributedMap<?>> maps;
> > +  private volatile ConcurrentMap<Class<?>, DistributedMultiMap<?>>
> multiMaps;
> > +  private volatile ConcurrentMap<String, Counter> counters;
> > +  private static final BufferAllocator allocator = new
> TopLevelAllocator();
> > +
> > +  private static final ObjectMapper mapper =
> DrillConfig.create().getMapper();
> > +
> > +  @Override
> > +  public void close() throws IOException {
> > +    handles = null;
> > +  }
> > +
> > +  @Override
> > +  public void run() throws DrillbitStartupException {
> > +    handles = Maps.newConcurrentMap();
> > +    maps = Maps.newConcurrentMap();
> > +    multiMaps = Maps.newConcurrentMap();
> > +    counters = Maps.newConcurrentMap();
> > +    namedMaps = Maps.newConcurrentMap();
> > +  }
> > +
> > +  @Override
> > +  public PlanFragment getFragment(FragmentHandle handle) {
> > +//    logger.debug("looking for fragment with handle: {}", handle);
> > +    return handles.get(handle);
> > +  }
> > +
> > +  @Override
> > +  public void storeFragment(PlanFragment fragment) {
> > +//    logger.debug("Storing fragment: {}", fragment);
> > +    handles.put(fragment.getHandle(), fragment);
> > +  }
> > +
> > +  @Override
> > +  public <V extends DrillSerializable> DistributedMultiMap<V>
> getMultiMap(Class<V> clazz) {
> > +    DistributedMultiMap<V> mmap = (DistributedMultiMap<V>)
> multiMaps.get(clazz);
> > +    if (mmap == null) {
> > +      multiMaps.putIfAbsent(clazz, new
> LocalDistributedMultiMapImpl<V>(clazz));
> > +      return (DistributedMultiMap<V>) multiMaps.get(clazz);
> > +    } else {
> > +      return mmap;
> > +    }
> > +  }
> > +
> > +  @Override
> > +  public <V extends DrillSerializable> DistributedMap<V>
> getMap(Class<V> clazz) {
> > +    DistributedMap m = maps.get(clazz);
> > +    if (m == null) {
> > +      maps.putIfAbsent(clazz, new LocalDistributedMapImpl<V>(clazz));
> > +      return (DistributedMap<V>) maps.get(clazz);
> > +    } else {
> > +      return m;
> > +    }
> > +  }
> > +
> > +
> > +  @Override
> > +  public <V extends DrillSerializable> DistributedMap<V>
> getNamedMap(String name, Class<V> clazz) {
> > +    DistributedMap m = namedMaps.get(clazz);
> > +    if (m == null) {
> > +      namedMaps.putIfAbsent(name, new
> LocalDistributedMapImpl<V>(clazz));
> > +      return (DistributedMap<V>) namedMaps.get(name);
> > +    } else {
> > +      return m;
> > +    }
> > +  }
> > +
> > +  @Override
> > +  public Counter getCounter(String name) {
> > +    Counter c = counters.get(name);
> > +    if (c == null) {
> > +      counters.putIfAbsent(name, new LocalCounterImpl());
> > +      return counters.get(name);
> > +    } else {
> > +      return c;
> > +    }
> > +  }
> > +
> > +  public static ByteArrayDataOutput serialize(DrillSerializable obj) {
> > +    if(obj instanceof JacksonSerializable){
> > +      try{
> > +        ByteArrayDataOutput out = ByteStreams.newDataOutput();
> > +        out.write(mapper.writeValueAsBytes(obj));
> > +        return out;
> > +      }catch(Exception e){
> > +        throw new RuntimeException(e);
> > +      }
> > +    }
> > +
> > +    ByteArrayDataOutput out = ByteStreams.newDataOutput();
> > +    OutputStream outputStream =
> DataOutputOutputStream.constructOutputStream(out);
> > +    try {
> > +      obj.writeToStream(outputStream);
> > +    } catch (IOException e) {
> > +      throw new RuntimeException(e);
> > +    }
> > +    try {
> > +      outputStream.flush();
> > +    } catch (IOException e) {
> > +      throw new RuntimeException(e);
> > +    }
> > +    return out;
> > +  }
> > +
> > +  public static <V extends DrillSerializable> V deserialize(byte[]
> bytes, Class<V> clazz) {
> > +    if(JacksonSerializable.class.isAssignableFrom(clazz)){
> > +      try{
> > +        return (V) mapper.readValue(bytes, clazz);
> > +      }catch(Exception e){
> > +        throw new RuntimeException(e);
> > +      }
> > +    }
> > +
> > +    ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
> > +    InputStream inputStream =
> DataInputInputStream.constructInputStream(in);
> > +    try {
> > +      V obj =
> clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
> > +      obj.readFromStream(inputStream);
> > +      return obj;
> > +    } catch (InstantiationException | IllegalAccessException |
> IOException | NoSuchMethodException | InvocationTargetException e) {
> > +      throw new RuntimeException(e);
> > +    }
> > +  }
> > +
> > +  public static class LocalDistributedMultiMapImpl<V extends
> DrillSerializable> implements DistributedMultiMap<V> {
> > +    private ArrayListMultimap<String, ByteArrayDataOutput> mmap;
> > +    private Class<V> clazz;
> > +
> > +    public LocalDistributedMultiMapImpl(Class<V> clazz) {
> > +      mmap = ArrayListMultimap.create();
> > +      this.clazz = clazz;
> > +    }
> > +
> > +    @Override
> > +    public Collection<V> get(String key) {
> > +      List<V> list = Lists.newArrayList();
> > +      for (ByteArrayDataOutput o : mmap.get(key)) {
> > +        list.add(deserialize(o.toByteArray(), this.clazz));
> > +      }
> > +      return list;
> > +    }
> > +
> > +    @Override
> > +    public void put(String key, DrillSerializable value) {
> > +      mmap.put(key, serialize(value));
> > +    }
> > +  }
> > +
> > +  public static class LocalDistributedMapImpl<V extends
> DrillSerializable> implements DistributedMap<V> {
> > +    protected ConcurrentMap<String, ByteArrayDataOutput> m;
> > +    protected Class<V> clazz;
> > +
> > +    public LocalDistributedMapImpl(Class<V> clazz) {
> > +      m = Maps.newConcurrentMap();
> > +      this.clazz = clazz;
> > +    }
> > +
> > +    @Override
> > +    public V get(String key) {
> > +      if (m.get(key) == null) return null;
> > +      ByteArrayDataOutput b = m.get(key);
> > +      byte[] bytes = b.toByteArray();
> > +      return (V) deserialize(m.get(key).toByteArray(), this.clazz);
> > +    }
> > +
> > +    @Override
> > +    public void put(String key, V value) {
> > +      m.put(key, serialize(value));
> > +    }
> > +
> > +    @Override
> > +    public void putIfAbsent(String key, V value) {
> > +      m.putIfAbsent(key, serialize(value));
> > +    }
> > +
> > +    @Override
> > +    public void putIfAbsent(String key, V value, long ttl, TimeUnit
> timeUnit) {
> > +      m.putIfAbsent(key, serialize(value));
> > +      logger.warn("Expiration not implemented in local map cache");
> > +    }
> > +
> > +    private class DeserializingTransformer implements
> Iterator<Map.Entry<String, V> >{
> > +      private Iterator<Map.Entry<String, ByteArrayDataOutput>> inner;
> > +
> > +      public DeserializingTransformer(Iterator<Entry<String,
> ByteArrayDataOutput>> inner) {
> > +        super();
> > +        this.inner = inner;
> > +      }
> > +
> > +      @Override
> > +      public boolean hasNext() {
> > +        return inner.hasNext();
> > +      }
> > +
> > +      @Override
> > +      public Entry<String, V> next() {
> > +        return newEntry(inner.next());
> > +      }
> > +
> > +      @Override
> > +      public void remove() {
> > +        throw new UnsupportedOperationException();
> > +      }
> > +
> > +      public Entry<String, V> newEntry(final Entry<String,
> ByteArrayDataOutput> input) {
> > +        return new Map.Entry<String, V>(){
> > +
> > +          @Override
> > +          public String getKey() {
> > +            return input.getKey();
> > +          }
> > +
> > +          @Override
> > +          public V getValue() {
> > +            return deserialize(input.getValue().toByteArray(), clazz);
> > +          }
> > +
> > +          @Override
> > +          public V setValue(V value) {
> > +            throw new UnsupportedOperationException();
> > +          }
> > +
> > +        };
> > +      }
> > +
> > +    }
> > +    @Override
> > +    public Iterator<Entry<String, V>> iterator() {
> > +      return new DeserializingTransformer(m.entrySet().iterator());
> > +    }
> > +  }
> > +
> > +  public static class LocalCounterImpl implements Counter {
> > +    private AtomicLong al = new AtomicLong();
> > +
> > +    @Override
> > +    public long get() {
> > +      return al.get();
> > +    }
> > +
> > +    @Override
> > +    public long incrementAndGet() {
> > +      return al.incrementAndGet();
> > +    }
> > +
> > +    @Override
> > +    public long decrementAndGet() {
> > +      return al.decrementAndGet();
> > +    }
> > +  }
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> > index f105363..a0c439e 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> > @@ -25,13 +25,13 @@ import java.util.concurrent.TimeUnit;
> >
> > import org.apache.drill.common.expression.ErrorCollector;
> > import org.apache.drill.common.expression.ErrorCollectorImpl;
> > -import org.apache.drill.common.expression.ExpressionPosition;
> > import org.apache.drill.common.expression.FieldReference;
> > import org.apache.drill.common.expression.LogicalExpression;
> > import org.apache.drill.common.expression.SchemaPath;
> > import org.apache.drill.common.logical.data.Order.Ordering;
> > import org.apache.drill.common.types.TypeProtos;
> > import org.apache.drill.common.types.Types;
> > +import org.apache.drill.exec.cache.CachedVectorContainer;
> > import org.apache.drill.exec.cache.Counter;
> > import org.apache.drill.exec.cache.DistributedCache;
> > import org.apache.drill.exec.cache.DistributedMap;
> > @@ -115,9 +115,9 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >   private int recordCount;
> >
> >   private final IntVector partitionKeyVector;
> > -  private final DistributedMap<VectorAccessibleSerializable> tableMap;
> > +  private final DistributedMap<CachedVectorContainer> tableMap;
> >   private final Counter minorFragmentSampleCount;
> > -  private final DistributedMultiMap<VectorAccessibleSerializable> mmap;
> > +  private final DistributedMultiMap<CachedVectorContainer> mmap;
> >   private final String mapKey;
> >   private List<VectorContainer> sampledIncomingBatches;
> >
> > @@ -131,8 +131,8 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >     this.completionFactor = pop.getCompletionFactor();
> >
> >     DistributedCache cache = context.getDrillbitContext().getCache();
> > -    this.mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
> > -    this.tableMap = cache.getMap(VectorAccessibleSerializable.class);
> > +    this.mmap = cache.getMultiMap(CachedVectorContainer.class);
> > +    this.tableMap = cache.getMap(CachedVectorContainer.class);
> >     Preconditions.checkNotNull(tableMap);
> >
> >     this.mapKey = String.format("%s_%d",
> context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
> > @@ -220,7 +220,7 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >     // into a serializable wrapper object, and then add to distributed
> map
> >
> >     WritableBatch batch =
> WritableBatch.getBatchNoHVWrap(containerToCache.getRecordCount(),
> containerToCache, false);
> > -    VectorAccessibleSerializable sampleToSave = new
> VectorAccessibleSerializable(batch, oContext.getAllocator());
> > +    CachedVectorContainer sampleToSave = new
> CachedVectorContainer(batch, context.getAllocator());
> >
> >     mmap.put(mapKey, sampleToSave);
> >     this.sampledIncomingBatches = builder.getHeldRecordBatches();
> > @@ -251,7 +251,7 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >         return false;
> >       }
> >
> > -      VectorAccessibleSerializable finalTable = null;
> > +      CachedVectorContainer finalTable = null;
> >
> >       long val = minorFragmentSampleCount.incrementAndGet();
> >       logger.debug("Incremented mfsc, got {}", val);
> > @@ -301,8 +301,8 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >
> >     // Get all samples from distributed map
> >
> > -    SortRecordBatchBuilder containerBuilder = new
> SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES);
> > -    for (VectorAccessibleSerializable w : mmap.get(mapKey)) {
> > +    SortRecordBatchBuilder containerBuilder = new
> SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
> > +    for (CachedVectorContainer w : mmap.get(mapKey)) {
> >       containerBuilder.add(w.get());
> >     }
> >     VectorContainer allSamplesContainer = new VectorContainer();
> > @@ -346,7 +346,7 @@ public class OrderedPartitionRecordBatch extends
> AbstractRecordBatch<OrderedPart
> >     }
> >     candidatePartitionTable.setRecordCount(copier.getOutputRecords());
> >     WritableBatch batch =
> WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(),
> candidatePartitionTable, false);
> > -    VectorAccessibleSerializable wrap = new
> VectorAccessibleSerializable(batch,
> context.getDrillbitContext().getAllocator());
> > +    CachedVectorContainer wrap = new CachedVectorContainer(batch,
> context.getDrillbitContext().getAllocator());
> >     tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
> >
> >     candidatePartitionTable.clear();
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> > index 7297dc3..0e3181d 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> > @@ -22,7 +22,7 @@ import java.io.Closeable;
> > import org.apache.drill.common.config.DrillConfig;
> > import org.apache.drill.exec.ExecConstants;
> > import org.apache.drill.exec.cache.DistributedCache;
> > -import org.apache.drill.exec.cache.HazelCache;
> > +import org.apache.drill.exec.cache.hazel.HazelCache;
> > import org.apache.drill.exec.coord.ClusterCoordinator;
> > import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
> > import org.apache.drill.exec.coord.ZKClusterCoordinator;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> > index c0b82bd..2078107 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> > @@ -21,17 +21,17 @@ import java.io.Closeable;
> > import java.io.IOException;
> >
> > import org.apache.drill.exec.cache.DistributedCache;
> > -import org.apache.drill.exec.cache.LocalCache;
> > +import org.apache.drill.exec.cache.local.LocalCache;
> > import org.apache.drill.exec.coord.ClusterCoordinator;
> > import org.apache.drill.exec.coord.LocalClusterCoordinator;
> > import org.apache.drill.exec.exception.DrillbitStartupException;
> >
> > public class RemoteServiceSet implements Closeable{
> >   static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
> > -
> > +
> >   private final DistributedCache cache;
> >   private final ClusterCoordinator coordinator;
> > -
> > +
> >   public RemoteServiceSet(DistributedCache cache, ClusterCoordinator
> coordinator) {
> >     super();
> >     this.cache = cache;
> > @@ -46,16 +46,21 @@ public class RemoteServiceSet implements Closeable{
> >   public ClusterCoordinator getCoordinator() {
> >     return coordinator;
> >   }
> > -
> > -
> > +
> > +
> >   @Override
> >   public void close() throws IOException {
> > +    try{
> >     cache.close();
> > +    }catch(Exception e){
> > +      if(e instanceof IOException) throw (IOException) e;
> > +      throw new IOException("Failure while closing cache", e);
> > +    }
> >     coordinator.close();
> >   }
> >
> >   public static RemoteServiceSet getLocalServiceSet(){
> >     return new RemoteServiceSet(new LocalCache(), new
> LocalClusterCoordinator());
> >   }
> > -
> > +
> > }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> > index 99e712b..20722d9 100644
> > --- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> > +++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> > @@ -29,7 +29,7 @@ import org.apache.drill.common.config.DrillConfig;
> > import org.apache.drill.common.util.TestTools;
> > import org.apache.drill.exec.ExecTest;
> > import org.apache.drill.exec.cache.DistributedCache;
> > -import org.apache.drill.exec.cache.LocalCache;
> > +import org.apache.drill.exec.cache.local.LocalCache;
> > import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
> > import org.apache.drill.exec.memory.TopLevelAllocator;
> > import org.apache.drill.exec.ops.QueryContext;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> > new file mode 100644
> > index 0000000..13322f1
> > --- /dev/null
> > +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> > @@ -0,0 +1,94 @@
> > +/**
> > + * 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.drill.exec.cache;
> > +
> > +import java.io.DataInput;
> > +import java.io.DataInputStream;
> > +import java.io.DataOutput;
> > +import java.io.DataOutputStream;
> > +import java.io.Externalizable;
> > +import java.io.IOException;
> > +import java.io.InputStream;
> > +import java.io.ObjectInput;
> > +import java.io.ObjectOutput;
> > +import java.io.OutputStream;
> > +import java.util.List;
> > +
> > +import org.infinispan.Cache;
> > +import org.infinispan.configuration.cache.CacheMode;
> > +import org.infinispan.configuration.cache.Configuration;
> > +import org.infinispan.configuration.cache.ConfigurationBuilder;
> > +import org.infinispan.configuration.global.GlobalConfiguration;
> > +import org.infinispan.configuration.global.GlobalConfigurationBuilder;
> > +import org.infinispan.manager.DefaultCacheManager;
> > +import org.infinispan.manager.EmbeddedCacheManager;
> > +
> > +import com.google.hive12.common.collect.Lists;
> > +
> > +public class ISpan {
> > +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ISpan.class);
> > +
> > +
> > +  public static void main(String[] args) throws Exception{
> > +    GlobalConfiguration gc = new
> GlobalConfigurationBuilder().transport().defaultTransport().build();
> > +    Configuration c = new ConfigurationBuilder() //
> > +    .clustering().cacheMode(CacheMode.DIST_ASYNC) //
> > +    .storeAsBinary()
> > +    .build();
> > +    EmbeddedCacheManager ecm = new DefaultCacheManager(gc, c);
> > +
> > +    Cache<String, List<XT>> cache = ecm.getCache();
> > +    List<XT> items = Lists.newArrayList();
> > +    items.add(new XT(1));
> > +    items.add(new XT(2));
> > +
> > +    cache.put("items", items);
> > +    for(XT x : cache.get("items")){
> > +      System.out.println(x.i);
> > +    }
> > +
> > +
> > +  }
> > +
> > +  private static class XT extends AbstractDataSerializable{
> > +
> > +    int i =0;
> > +
> > +
> > +    public XT(int i) {
> > +      super();
> > +      this.i = i;
> > +    }
> > +
> > +    @Override
> > +    public void read(DataInput input) throws IOException {
> > +      i = input.readInt();
> > +    }
> > +
> > +    @Override
> > +    public void write(DataOutput output) throws IOException {
> > +      output.writeInt(i);
> > +    }
> > +
> > +    @Override
> > +    public String toString() {
> > +      return "XT [i=" + i + "]";
> > +    }
> > +
> > +  }
> > +}
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > index a3d39a3..7686614 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > @@ -17,7 +17,7 @@
> >  */
> > package org.apache.drill.exec.cache;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import java.util.List;
> >
> > import org.apache.drill.common.config.DrillConfig;
> > import org.apache.drill.common.expression.ExpressionPosition;
> > @@ -25,67 +25,105 @@ import
> org.apache.drill.common.expression.SchemaPath;
> > import org.apache.drill.common.types.TypeProtos;
> > import org.apache.drill.common.types.Types;
> > import org.apache.drill.exec.ExecTest;
> > +import org.apache.drill.exec.cache.hazel.HazelCache;
> > +import org.apache.drill.exec.cache.infinispan.ICache;
> > import org.apache.drill.exec.expr.TypeHelper;
> > -import org.apache.drill.exec.record.*;
> > +import org.apache.drill.exec.memory.TopLevelAllocator;
> > +import org.apache.drill.exec.record.MaterializedField;
> > +import org.apache.drill.exec.record.VectorAccessible;
> > +import org.apache.drill.exec.record.VectorContainer;
> > +import org.apache.drill.exec.record.VectorWrapper;
> > +import org.apache.drill.exec.record.WritableBatch;
> > import org.apache.drill.exec.server.Drillbit;
> > import org.apache.drill.exec.server.DrillbitContext;
> > import org.apache.drill.exec.server.RemoteServiceSet;
> > -import org.apache.drill.exec.vector.*;
> > +import org.apache.drill.exec.vector.AllocationHelper;
> > +import org.apache.drill.exec.vector.IntVector;
> > +import org.apache.drill.exec.vector.ValueVector;
> > +import org.apache.drill.exec.vector.VarBinaryVector;
> > import org.junit.Test;
> >
> > -import java.util.List;
> > +import com.beust.jcommander.internal.Lists;
> >
> > -public class TestVectorCache  extends ExecTest{
> > +public class TestVectorCache extends ExecTest{
> >
> > -  @Test
> > -  public void testVectorCache() throws Exception {
> > +  private void testCache(DrillConfig config, DistributedCache dcache)
> throws Exception {
> >     List<ValueVector> vectorList = Lists.newArrayList();
> >     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
> > -    DrillConfig config = DrillConfig.create();
> > -    Drillbit bit = new Drillbit(config, serviceSet);
> > -    bit.run();
> > -    DrillbitContext context = bit.getContext();
> > -    HazelCache cache = new HazelCache(config, context.getAllocator());
> > -    cache.run();
> >
> > -    MaterializedField intField =
> MaterializedField.create(SchemaPath.getSimplePath("int"), Types.required(
> TypeProtos.MinorType.INT));
> > -    IntVector intVector = (IntVector)TypeHelper.getNewVector(intField,
> context.getAllocator());
> > -    MaterializedField binField = MaterializedField.create(new
> SchemaPath("binary", ExpressionPosition.UNKNOWN),
> Types.required(TypeProtos.MinorType.VARBINARY));
> > -    VarBinaryVector binVector =
> (VarBinaryVector)TypeHelper.getNewVector(binField, context.getAllocator());
> > -    AllocationHelper.allocate(intVector, 4, 4);
> > -    AllocationHelper.allocate(binVector, 4, 5);
> > -    vectorList.add(intVector);
> > -    vectorList.add(binVector);
> > -
> > -    intVector.getMutator().setSafe(0, 0);
> binVector.getMutator().setSafe(0, "ZERO".getBytes());
> > -    intVector.getMutator().setSafe(1, 1);
> binVector.getMutator().setSafe(1, "ONE".getBytes());
> > -    intVector.getMutator().setSafe(2, 2);
> binVector.getMutator().setSafe(2, "TWO".getBytes());
> > -    intVector.getMutator().setSafe(3, 3);
> binVector.getMutator().setSafe(3, "THREE".getBytes());
> > -    intVector.getMutator().setValueCount(4);
> > -    binVector.getMutator().setValueCount(4);
> > -
> > -    VectorContainer container = new VectorContainer();
> > -    container.addCollection(vectorList);
> > -    container.setRecordCount(4);
> > -    WritableBatch batch =
> WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container,
> false);
> > -    VectorAccessibleSerializable wrap = new
> VectorAccessibleSerializable(batch, context.getAllocator());
> > -
> > -    DistributedMultiMap<VectorAccessibleSerializable> mmap =
> cache.getMultiMap(VectorAccessibleSerializable.class);
> > -    mmap.put("vectors", wrap);
> > -    VectorAccessibleSerializable newWrap =
> (VectorAccessibleSerializable)mmap.get("vectors").iterator().next();
> > -
> > -    VectorAccessible newContainer = newWrap.get();
> > -    for (VectorWrapper w : newContainer) {
> > -      ValueVector vv = w.getValueVector();
> > -      int values = vv.getAccessor().getValueCount();
> > -      for (int i = 0; i < values; i++) {
> > -        Object o = vv.getAccessor().getObject(i);
> > -        if (o instanceof byte[]) {
> > -          System.out.println(new String((byte[])o));
> > -        } else {
> > -          System.out.println(o);
> > +    try (Drillbit bit = new Drillbit(config, serviceSet);
> DistributedCache cache = dcache) {
> > +      bit.run();
> > +      cache.run();
> > +
> > +      DrillbitContext context = bit.getContext();
> > +
> > +
> > +      MaterializedField intField = MaterializedField.create(new
> SchemaPath("int", ExpressionPosition.UNKNOWN),
> > +          Types.required(TypeProtos.MinorType.INT));
> > +      IntVector intVector = (IntVector)
> TypeHelper.getNewVector(intField, context.getAllocator());
> > +      MaterializedField binField = MaterializedField.create(new
> SchemaPath("binary", ExpressionPosition.UNKNOWN),
> > +          Types.required(TypeProtos.MinorType.VARBINARY));
> > +      VarBinaryVector binVector = (VarBinaryVector)
> TypeHelper.getNewVector(binField, context.getAllocator());
> > +      AllocationHelper.allocate(intVector, 4, 4);
> > +      AllocationHelper.allocate(binVector, 4, 5);
> > +      vectorList.add(intVector);
> > +      vectorList.add(binVector);
> > +
> > +      intVector.getMutator().set(0, 0);
> > +      binVector.getMutator().set(0, "ZERO".getBytes());
> > +      intVector.getMutator().set(1, 1);
> > +      binVector.getMutator().set(1, "ONE".getBytes());
> > +      intVector.getMutator().set(2, 2);
> > +      binVector.getMutator().set(2, "TWO".getBytes());
> > +      intVector.getMutator().set(3, 3);
> > +      binVector.getMutator().set(3, "THREE".getBytes());
> > +      intVector.getMutator().setValueCount(4);
> > +      binVector.getMutator().setValueCount(4);
> > +
> > +      VectorContainer container = new VectorContainer();
> > +      container.addCollection(vectorList);
> > +      container.setRecordCount(4);
> > +      WritableBatch batch =
> WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container,
> false);
> > +      CachedVectorContainer wrap = new CachedVectorContainer(batch,
> context.getAllocator());
> > +
> > +      DistributedMultiMap<CachedVectorContainer> mmap =
> cache.getMultiMap(CachedVectorContainer.class);
> > +      mmap.put("vectors", wrap);
> > +
> > +      CachedVectorContainer newWrap = (CachedVectorContainer)
> mmap.get("vectors").iterator().next();
> > +
> > +      VectorAccessible newContainer = newWrap.get();
> > +      for (VectorWrapper<?> w : newContainer) {
> > +        ValueVector vv = w.getValueVector();
> > +        int values = vv.getAccessor().getValueCount();
> > +        for (int i = 0; i < values; i++) {
> > +          Object o = vv.getAccessor().getObject(i);
> > +          if (o instanceof byte[]) {
> > +            System.out.println(new String((byte[]) o));
> > +          } else {
> > +            System.out.println(o);
> > +          }
> >         }
> >       }
> > +
> > +      newWrap.clear();
> >     }
> > +
> > +  }
> > +
> > +  @Test
> > +  public void testHazelVectorCache() throws Exception {
> > +    DrillConfig c = DrillConfig.create();
> > +    HazelCache cache = new HazelCache(c, new TopLevelAllocator());
> > +    cache.run();
> > +    testCache(c, cache);
> > +    cache.close();
> > +  }
> > +
> > +  @Test
> > +  public void testICache() throws Exception {
> > +    DrillConfig c = DrillConfig.create();
> > +    ICache cache = new ICache(c, new TopLevelAllocator());
> > +    testCache(c, cache);
> > +
> >   }
> > }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> > index 63bc0a9..a5dbfe5 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> > @@ -23,7 +23,7 @@ import net.hydromatic.optiq.tools.Frameworks;
> >
> > import org.apache.drill.common.config.DrillConfig;
> > import org.apache.drill.exec.ExecTest;
> > -import org.apache.drill.exec.cache.LocalCache;
> > +import org.apache.drill.exec.cache.local.LocalCache;
> > import org.apache.drill.exec.memory.TopLevelAllocator;
> > import org.apache.drill.exec.rpc.user.UserSession;
> > import org.apache.drill.exec.server.DrillbitContext;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> > index acb5929..3ccb96b 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> > @@ -18,12 +18,13 @@
> > package org.apache.drill.exec.store.ischema;
> >
> >
> > -import static org.mockito.Mockito.*;
> > +import static org.mockito.Mockito.mock;
> > +import static org.mockito.Mockito.when;
> > import net.hydromatic.optiq.SchemaPlus;
> > import net.hydromatic.optiq.tools.Frameworks;
> >
> > import org.apache.drill.common.config.DrillConfig;
> > -import org.apache.drill.exec.cache.LocalCache;
> > +import org.apache.drill.exec.cache.local.LocalCache;
> > import org.apache.drill.exec.memory.TopLevelAllocator;
> > import org.apache.drill.exec.rpc.user.UserSession;
> > import org.apache.drill.exec.server.DrillbitContext;
> > @@ -40,7 +41,7 @@ public class OrphanSchema {
> >    * @return root node of the created schema.
> >    */
> >   public static SchemaPlus create() throws Exception {
> > -
> > +
> >     final DrillConfig c = DrillConfig.create();
> >
> >     // Mock up a context which will allow us to create a schema.
> > @@ -51,7 +52,7 @@ public class OrphanSchema {
> >     when(bitContext.getCache()).thenReturn(new LocalCache());
> >
> >     bitContext.getCache().run();
> > -
> > +
> >     // Using the mock context, get the orphan schema.
> >     StoragePluginRegistry r = new StoragePluginRegistry(bitContext);
> >     r.init();
> >
>

Re: [10/24] ispan

Posted by Timothy Chen <tn...@gmail.com>.
Any motivation for using inifinispan instead of hazel cast?

Tim

Sent from my iPhone

> On May 21, 2014, at 6:14 PM, jacques@apache.org wrote:
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> new file mode 100644
> index 0000000..46d4eca
> --- /dev/null
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
> @@ -0,0 +1,66 @@
> +/**
> + * 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.drill.exec.cache.infinispan;
> +
> +import org.infinispan.marshall.core.MarshalledEntry;
> +import org.infinispan.persistence.spi.ExternalStore;
> +import org.infinispan.persistence.spi.InitializationContext;
> +
> +/**
> + * Stores the cached objects in zookeeper.  Objects are stored in /start/cache_name/key_name = data
> + * @param <K>
> + * @param <V>
> + */
> +public class ZookeeperCacheStore<K, V> implements ExternalStore<K, V>{
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZookeeperCacheStore.class);
> +
> +  private String cacheName;
> +
> +  @Override
> +  public void init(InitializationContext ctx) {
> +    ctx.getConfiguration();
> +
> +  }
> +
> +  @Override
> +  public MarshalledEntry<K, V> load(K key) {
> +    return null;
> +  }
> +
> +  @Override
> +  public boolean contains(K key) {
> +    return false;
> +  }
> +
> +  @Override
> +  public void start() {
> +  }
> +
> +  @Override
> +  public void stop() {
> +  }
> +
> +  @Override
> +  public void write(MarshalledEntry<K, V> entry) {
> +  }
> +
> +  @Override
> +  public boolean delete(K key) {
> +    return false;
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> new file mode 100644
> index 0000000..e66cc90
> --- /dev/null
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
> @@ -0,0 +1,309 @@
> +/**
> + * 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.drill.exec.cache.local;
> +
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.OutputStream;
> +import java.lang.reflect.InvocationTargetException;
> +import java.util.Collection;
> +import java.util.Iterator;
> +import java.util.List;
> +import java.util.Map;
> +import java.util.Map.Entry;
> +import java.util.concurrent.ConcurrentMap;
> +import java.util.concurrent.TimeUnit;
> +import java.util.concurrent.atomic.AtomicLong;
> +
> +import org.apache.drill.common.util.DataInputInputStream;
> +import org.apache.drill.common.util.DataOutputOutputStream;
> +import org.apache.drill.exec.cache.Counter;
> +import org.apache.drill.exec.cache.DistributedCache;
> +import org.apache.drill.exec.cache.DistributedMap;
> +import org.apache.drill.exec.cache.DistributedMultiMap;
> +import org.apache.drill.exec.cache.DrillSerializable;
> +import org.apache.drill.exec.exception.DrillbitStartupException;
> +import org.apache.drill.exec.memory.BufferAllocator;
> +import org.apache.drill.exec.memory.TopLevelAllocator;
> +import org.apache.drill.exec.proto.BitControl.PlanFragment;
> +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> +
> +import com.fasterxml.jackson.databind.ObjectMapper;
> +import com.google.common.collect.ArrayListMultimap;
> +import com.google.common.collect.Lists;
> +import com.google.common.collect.Maps;
> +import com.google.common.io.ByteArrayDataInput;
> +import com.google.common.io.ByteArrayDataOutput;
> +import com.google.common.io.ByteStreams;
> +
> +public class LocalCache implements DistributedCache {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
> +
> +  private volatile Map<FragmentHandle, PlanFragment> handles;
> +  private volatile ConcurrentMap<String, DistributedMap<?>> namedMaps;
> +  private volatile ConcurrentMap<Class<?>, DistributedMap<?>> maps;
> +  private volatile ConcurrentMap<Class<?>, DistributedMultiMap<?>> multiMaps;
> +  private volatile ConcurrentMap<String, Counter> counters;
> +  private static final BufferAllocator allocator = new TopLevelAllocator();
> +
> +  private static final ObjectMapper mapper = DrillConfig.create().getMapper();
> +
> +  @Override
> +  public void close() throws IOException {
> +    handles = null;
> +  }
> +
> +  @Override
> +  public void run() throws DrillbitStartupException {
> +    handles = Maps.newConcurrentMap();
> +    maps = Maps.newConcurrentMap();
> +    multiMaps = Maps.newConcurrentMap();
> +    counters = Maps.newConcurrentMap();
> +    namedMaps = Maps.newConcurrentMap();
> +  }
> +
> +  @Override
> +  public PlanFragment getFragment(FragmentHandle handle) {
> +//    logger.debug("looking for fragment with handle: {}", handle);
> +    return handles.get(handle);
> +  }
> +
> +  @Override
> +  public void storeFragment(PlanFragment fragment) {
> +//    logger.debug("Storing fragment: {}", fragment);
> +    handles.put(fragment.getHandle(), fragment);
> +  }
> +
> +  @Override
> +  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
> +    DistributedMultiMap<V> mmap = (DistributedMultiMap<V>) multiMaps.get(clazz);
> +    if (mmap == null) {
> +      multiMaps.putIfAbsent(clazz, new LocalDistributedMultiMapImpl<V>(clazz));
> +      return (DistributedMultiMap<V>) multiMaps.get(clazz);
> +    } else {
> +      return mmap;
> +    }
> +  }
> +
> +  @Override
> +  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
> +    DistributedMap m = maps.get(clazz);
> +    if (m == null) {
> +      maps.putIfAbsent(clazz, new LocalDistributedMapImpl<V>(clazz));
> +      return (DistributedMap<V>) maps.get(clazz);
> +    } else {
> +      return m;
> +    }
> +  }
> +
> +
> +  @Override
> +  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
> +    DistributedMap m = namedMaps.get(clazz);
> +    if (m == null) {
> +      namedMaps.putIfAbsent(name, new LocalDistributedMapImpl<V>(clazz));
> +      return (DistributedMap<V>) namedMaps.get(name);
> +    } else {
> +      return m;
> +    }
> +  }
> +
> +  @Override
> +  public Counter getCounter(String name) {
> +    Counter c = counters.get(name);
> +    if (c == null) {
> +      counters.putIfAbsent(name, new LocalCounterImpl());
> +      return counters.get(name);
> +    } else {
> +      return c;
> +    }
> +  }
> +
> +  public static ByteArrayDataOutput serialize(DrillSerializable obj) {
> +    if(obj instanceof JacksonSerializable){
> +      try{
> +        ByteArrayDataOutput out = ByteStreams.newDataOutput();
> +        out.write(mapper.writeValueAsBytes(obj));
> +        return out;
> +      }catch(Exception e){
> +        throw new RuntimeException(e);
> +      }
> +    }
> +
> +    ByteArrayDataOutput out = ByteStreams.newDataOutput();
> +    OutputStream outputStream = DataOutputOutputStream.constructOutputStream(out);
> +    try {
> +      obj.writeToStream(outputStream);
> +    } catch (IOException e) {
> +      throw new RuntimeException(e);
> +    }
> +    try {
> +      outputStream.flush();
> +    } catch (IOException e) {
> +      throw new RuntimeException(e);
> +    }
> +    return out;
> +  }
> +
> +  public static <V extends DrillSerializable> V deserialize(byte[] bytes, Class<V> clazz) {
> +    if(JacksonSerializable.class.isAssignableFrom(clazz)){
> +      try{
> +        return (V) mapper.readValue(bytes, clazz);
> +      }catch(Exception e){
> +        throw new RuntimeException(e);
> +      }
> +    }
> +
> +    ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
> +    InputStream inputStream = DataInputInputStream.constructInputStream(in);
> +    try {
> +      V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
> +      obj.readFromStream(inputStream);
> +      return obj;
> +    } catch (InstantiationException | IllegalAccessException | IOException | NoSuchMethodException | InvocationTargetException e) {
> +      throw new RuntimeException(e);
> +    }
> +  }
> +
> +  public static class LocalDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
> +    private ArrayListMultimap<String, ByteArrayDataOutput> mmap;
> +    private Class<V> clazz;
> +
> +    public LocalDistributedMultiMapImpl(Class<V> clazz) {
> +      mmap = ArrayListMultimap.create();
> +      this.clazz = clazz;
> +    }
> +
> +    @Override
> +    public Collection<V> get(String key) {
> +      List<V> list = Lists.newArrayList();
> +      for (ByteArrayDataOutput o : mmap.get(key)) {
> +        list.add(deserialize(o.toByteArray(), this.clazz));
> +      }
> +      return list;
> +    }
> +
> +    @Override
> +    public void put(String key, DrillSerializable value) {
> +      mmap.put(key, serialize(value));
> +    }
> +  }
> +
> +  public static class LocalDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
> +    protected ConcurrentMap<String, ByteArrayDataOutput> m;
> +    protected Class<V> clazz;
> +
> +    public LocalDistributedMapImpl(Class<V> clazz) {
> +      m = Maps.newConcurrentMap();
> +      this.clazz = clazz;
> +    }
> +
> +    @Override
> +    public V get(String key) {
> +      if (m.get(key) == null) return null;
> +      ByteArrayDataOutput b = m.get(key);
> +      byte[] bytes = b.toByteArray();
> +      return (V) deserialize(m.get(key).toByteArray(), this.clazz);
> +    }
> +
> +    @Override
> +    public void put(String key, V value) {
> +      m.put(key, serialize(value));
> +    }
> +
> +    @Override
> +    public void putIfAbsent(String key, V value) {
> +      m.putIfAbsent(key, serialize(value));
> +    }
> +
> +    @Override
> +    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit) {
> +      m.putIfAbsent(key, serialize(value));
> +      logger.warn("Expiration not implemented in local map cache");
> +    }
> +
> +    private class DeserializingTransformer implements Iterator<Map.Entry<String, V> >{
> +      private Iterator<Map.Entry<String, ByteArrayDataOutput>> inner;
> +
> +      public DeserializingTransformer(Iterator<Entry<String, ByteArrayDataOutput>> inner) {
> +        super();
> +        this.inner = inner;
> +      }
> +
> +      @Override
> +      public boolean hasNext() {
> +        return inner.hasNext();
> +      }
> +
> +      @Override
> +      public Entry<String, V> next() {
> +        return newEntry(inner.next());
> +      }
> +
> +      @Override
> +      public void remove() {
> +        throw new UnsupportedOperationException();
> +      }
> +
> +      public Entry<String, V> newEntry(final Entry<String, ByteArrayDataOutput> input) {
> +        return new Map.Entry<String, V>(){
> +
> +          @Override
> +          public String getKey() {
> +            return input.getKey();
> +          }
> +
> +          @Override
> +          public V getValue() {
> +            return deserialize(input.getValue().toByteArray(), clazz);
> +          }
> +
> +          @Override
> +          public V setValue(V value) {
> +            throw new UnsupportedOperationException();
> +          }
> +
> +        };
> +      }
> +
> +    }
> +    @Override
> +    public Iterator<Entry<String, V>> iterator() {
> +      return new DeserializingTransformer(m.entrySet().iterator());
> +    }
> +  }
> +
> +  public static class LocalCounterImpl implements Counter {
> +    private AtomicLong al = new AtomicLong();
> +
> +    @Override
> +    public long get() {
> +      return al.get();
> +    }
> +
> +    @Override
> +    public long incrementAndGet() {
> +      return al.incrementAndGet();
> +    }
> +
> +    @Override
> +    public long decrementAndGet() {
> +      return al.decrementAndGet();
> +    }
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> index f105363..a0c439e 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
> @@ -25,13 +25,13 @@ import java.util.concurrent.TimeUnit;
> 
> import org.apache.drill.common.expression.ErrorCollector;
> import org.apache.drill.common.expression.ErrorCollectorImpl;
> -import org.apache.drill.common.expression.ExpressionPosition;
> import org.apache.drill.common.expression.FieldReference;
> import org.apache.drill.common.expression.LogicalExpression;
> import org.apache.drill.common.expression.SchemaPath;
> import org.apache.drill.common.logical.data.Order.Ordering;
> import org.apache.drill.common.types.TypeProtos;
> import org.apache.drill.common.types.Types;
> +import org.apache.drill.exec.cache.CachedVectorContainer;
> import org.apache.drill.exec.cache.Counter;
> import org.apache.drill.exec.cache.DistributedCache;
> import org.apache.drill.exec.cache.DistributedMap;
> @@ -115,9 +115,9 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
>   private int recordCount;
> 
>   private final IntVector partitionKeyVector;
> -  private final DistributedMap<VectorAccessibleSerializable> tableMap;
> +  private final DistributedMap<CachedVectorContainer> tableMap;
>   private final Counter minorFragmentSampleCount;
> -  private final DistributedMultiMap<VectorAccessibleSerializable> mmap;
> +  private final DistributedMultiMap<CachedVectorContainer> mmap;
>   private final String mapKey;
>   private List<VectorContainer> sampledIncomingBatches;
> 
> @@ -131,8 +131,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
>     this.completionFactor = pop.getCompletionFactor();
> 
>     DistributedCache cache = context.getDrillbitContext().getCache();
> -    this.mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
> -    this.tableMap = cache.getMap(VectorAccessibleSerializable.class);
> +    this.mmap = cache.getMultiMap(CachedVectorContainer.class);
> +    this.tableMap = cache.getMap(CachedVectorContainer.class);
>     Preconditions.checkNotNull(tableMap);
> 
>     this.mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
> @@ -220,7 +220,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
>     // into a serializable wrapper object, and then add to distributed map
> 
>     WritableBatch batch = WritableBatch.getBatchNoHVWrap(containerToCache.getRecordCount(), containerToCache, false);
> -    VectorAccessibleSerializable sampleToSave = new VectorAccessibleSerializable(batch, oContext.getAllocator());
> +    CachedVectorContainer sampleToSave = new CachedVectorContainer(batch, context.getAllocator());
> 
>     mmap.put(mapKey, sampleToSave);
>     this.sampledIncomingBatches = builder.getHeldRecordBatches();
> @@ -251,7 +251,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
>         return false;
>       }
> 
> -      VectorAccessibleSerializable finalTable = null;
> +      CachedVectorContainer finalTable = null;
> 
>       long val = minorFragmentSampleCount.incrementAndGet();
>       logger.debug("Incremented mfsc, got {}", val);
> @@ -301,8 +301,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
> 
>     // Get all samples from distributed map
> 
> -    SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES);
> -    for (VectorAccessibleSerializable w : mmap.get(mapKey)) {
> +    SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
> +    for (CachedVectorContainer w : mmap.get(mapKey)) {
>       containerBuilder.add(w.get());
>     }
>     VectorContainer allSamplesContainer = new VectorContainer();
> @@ -346,7 +346,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
>     }
>     candidatePartitionTable.setRecordCount(copier.getOutputRecords());
>     WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false);
> -    VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getDrillbitContext().getAllocator());
> +    CachedVectorContainer wrap = new CachedVectorContainer(batch, context.getDrillbitContext().getAllocator());
>     tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
> 
>     candidatePartitionTable.clear();
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> index 7297dc3..0e3181d 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
> @@ -22,7 +22,7 @@ import java.io.Closeable;
> import org.apache.drill.common.config.DrillConfig;
> import org.apache.drill.exec.ExecConstants;
> import org.apache.drill.exec.cache.DistributedCache;
> -import org.apache.drill.exec.cache.HazelCache;
> +import org.apache.drill.exec.cache.hazel.HazelCache;
> import org.apache.drill.exec.coord.ClusterCoordinator;
> import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
> import org.apache.drill.exec.coord.ZKClusterCoordinator;
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> index c0b82bd..2078107 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
> @@ -21,17 +21,17 @@ import java.io.Closeable;
> import java.io.IOException;
> 
> import org.apache.drill.exec.cache.DistributedCache;
> -import org.apache.drill.exec.cache.LocalCache;
> +import org.apache.drill.exec.cache.local.LocalCache;
> import org.apache.drill.exec.coord.ClusterCoordinator;
> import org.apache.drill.exec.coord.LocalClusterCoordinator;
> import org.apache.drill.exec.exception.DrillbitStartupException;
> 
> public class RemoteServiceSet implements Closeable{
>   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
> -  
> +
>   private final DistributedCache cache;
>   private final ClusterCoordinator coordinator;
> -  
> +
>   public RemoteServiceSet(DistributedCache cache, ClusterCoordinator coordinator) {
>     super();
>     this.cache = cache;
> @@ -46,16 +46,21 @@ public class RemoteServiceSet implements Closeable{
>   public ClusterCoordinator getCoordinator() {
>     return coordinator;
>   }
> -  
> -  
> +
> +
>   @Override
>   public void close() throws IOException {
> +    try{
>     cache.close();
> +    }catch(Exception e){
> +      if(e instanceof IOException) throw (IOException) e;
> +      throw new IOException("Failure while closing cache", e);
> +    }
>     coordinator.close();
>   }
> 
>   public static RemoteServiceSet getLocalServiceSet(){
>     return new RemoteServiceSet(new LocalCache(), new LocalClusterCoordinator());
>   }
> -  
> +
> }
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> index 99e712b..20722d9 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
> @@ -29,7 +29,7 @@ import org.apache.drill.common.config.DrillConfig;
> import org.apache.drill.common.util.TestTools;
> import org.apache.drill.exec.ExecTest;
> import org.apache.drill.exec.cache.DistributedCache;
> -import org.apache.drill.exec.cache.LocalCache;
> +import org.apache.drill.exec.cache.local.LocalCache;
> import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
> import org.apache.drill.exec.memory.TopLevelAllocator;
> import org.apache.drill.exec.ops.QueryContext;
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> new file mode 100644
> index 0000000..13322f1
> --- /dev/null
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
> @@ -0,0 +1,94 @@
> +/**
> + * 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.drill.exec.cache;
> +
> +import java.io.DataInput;
> +import java.io.DataInputStream;
> +import java.io.DataOutput;
> +import java.io.DataOutputStream;
> +import java.io.Externalizable;
> +import java.io.IOException;
> +import java.io.InputStream;
> +import java.io.ObjectInput;
> +import java.io.ObjectOutput;
> +import java.io.OutputStream;
> +import java.util.List;
> +
> +import org.infinispan.Cache;
> +import org.infinispan.configuration.cache.CacheMode;
> +import org.infinispan.configuration.cache.Configuration;
> +import org.infinispan.configuration.cache.ConfigurationBuilder;
> +import org.infinispan.configuration.global.GlobalConfiguration;
> +import org.infinispan.configuration.global.GlobalConfigurationBuilder;
> +import org.infinispan.manager.DefaultCacheManager;
> +import org.infinispan.manager.EmbeddedCacheManager;
> +
> +import com.google.hive12.common.collect.Lists;
> +
> +public class ISpan {
> +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ISpan.class);
> +
> +
> +  public static void main(String[] args) throws Exception{
> +    GlobalConfiguration gc = new GlobalConfigurationBuilder().transport().defaultTransport().build();
> +    Configuration c = new ConfigurationBuilder() //
> +    .clustering().cacheMode(CacheMode.DIST_ASYNC) //
> +    .storeAsBinary()
> +    .build();
> +    EmbeddedCacheManager ecm = new DefaultCacheManager(gc, c);
> +
> +    Cache<String, List<XT>> cache = ecm.getCache();
> +    List<XT> items = Lists.newArrayList();
> +    items.add(new XT(1));
> +    items.add(new XT(2));
> +
> +    cache.put("items", items);
> +    for(XT x : cache.get("items")){
> +      System.out.println(x.i);
> +    }
> +
> +
> +  }
> +
> +  private static class XT extends AbstractDataSerializable{
> +
> +    int i =0;
> +
> +
> +    public XT(int i) {
> +      super();
> +      this.i = i;
> +    }
> +
> +    @Override
> +    public void read(DataInput input) throws IOException {
> +      i = input.readInt();
> +    }
> +
> +    @Override
> +    public void write(DataOutput output) throws IOException {
> +      output.writeInt(i);
> +    }
> +
> +    @Override
> +    public String toString() {
> +      return "XT [i=" + i + "]";
> +    }
> +
> +  }
> +}
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> index a3d39a3..7686614 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> @@ -17,7 +17,7 @@
>  */
> package org.apache.drill.exec.cache;
> 
> -import com.beust.jcommander.internal.Lists;
> +import java.util.List;
> 
> import org.apache.drill.common.config.DrillConfig;
> import org.apache.drill.common.expression.ExpressionPosition;
> @@ -25,67 +25,105 @@ import org.apache.drill.common.expression.SchemaPath;
> import org.apache.drill.common.types.TypeProtos;
> import org.apache.drill.common.types.Types;
> import org.apache.drill.exec.ExecTest;
> +import org.apache.drill.exec.cache.hazel.HazelCache;
> +import org.apache.drill.exec.cache.infinispan.ICache;
> import org.apache.drill.exec.expr.TypeHelper;
> -import org.apache.drill.exec.record.*;
> +import org.apache.drill.exec.memory.TopLevelAllocator;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.VectorAccessible;
> +import org.apache.drill.exec.record.VectorContainer;
> +import org.apache.drill.exec.record.VectorWrapper;
> +import org.apache.drill.exec.record.WritableBatch;
> import org.apache.drill.exec.server.Drillbit;
> import org.apache.drill.exec.server.DrillbitContext;
> import org.apache.drill.exec.server.RemoteServiceSet;
> -import org.apache.drill.exec.vector.*;
> +import org.apache.drill.exec.vector.AllocationHelper;
> +import org.apache.drill.exec.vector.IntVector;
> +import org.apache.drill.exec.vector.ValueVector;
> +import org.apache.drill.exec.vector.VarBinaryVector;
> import org.junit.Test;
> 
> -import java.util.List;
> +import com.beust.jcommander.internal.Lists;
> 
> -public class TestVectorCache  extends ExecTest{
> +public class TestVectorCache extends ExecTest{
> 
> -  @Test
> -  public void testVectorCache() throws Exception {
> +  private void testCache(DrillConfig config, DistributedCache dcache) throws Exception {
>     List<ValueVector> vectorList = Lists.newArrayList();
>     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
> -    DrillConfig config = DrillConfig.create();
> -    Drillbit bit = new Drillbit(config, serviceSet);
> -    bit.run();
> -    DrillbitContext context = bit.getContext();
> -    HazelCache cache = new HazelCache(config, context.getAllocator());
> -    cache.run();
> 
> -    MaterializedField intField = MaterializedField.create(SchemaPath.getSimplePath("int"), Types.required(TypeProtos.MinorType.INT));
> -    IntVector intVector = (IntVector)TypeHelper.getNewVector(intField, context.getAllocator());
> -    MaterializedField binField = MaterializedField.create(new SchemaPath("binary", ExpressionPosition.UNKNOWN), Types.required(TypeProtos.MinorType.VARBINARY));
> -    VarBinaryVector binVector = (VarBinaryVector)TypeHelper.getNewVector(binField, context.getAllocator());
> -    AllocationHelper.allocate(intVector, 4, 4);
> -    AllocationHelper.allocate(binVector, 4, 5);
> -    vectorList.add(intVector);
> -    vectorList.add(binVector);
> -
> -    intVector.getMutator().setSafe(0, 0); binVector.getMutator().setSafe(0, "ZERO".getBytes());
> -    intVector.getMutator().setSafe(1, 1); binVector.getMutator().setSafe(1, "ONE".getBytes());
> -    intVector.getMutator().setSafe(2, 2); binVector.getMutator().setSafe(2, "TWO".getBytes());
> -    intVector.getMutator().setSafe(3, 3); binVector.getMutator().setSafe(3, "THREE".getBytes());
> -    intVector.getMutator().setValueCount(4);
> -    binVector.getMutator().setValueCount(4);
> -
> -    VectorContainer container = new VectorContainer();
> -    container.addCollection(vectorList);
> -    container.setRecordCount(4);
> -    WritableBatch batch = WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container, false);
> -    VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getAllocator());
> -
> -    DistributedMultiMap<VectorAccessibleSerializable> mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
> -    mmap.put("vectors", wrap);
> -    VectorAccessibleSerializable newWrap = (VectorAccessibleSerializable)mmap.get("vectors").iterator().next();
> -
> -    VectorAccessible newContainer = newWrap.get();
> -    for (VectorWrapper w : newContainer) {
> -      ValueVector vv = w.getValueVector();
> -      int values = vv.getAccessor().getValueCount();
> -      for (int i = 0; i < values; i++) {
> -        Object o = vv.getAccessor().getObject(i);
> -        if (o instanceof byte[]) {
> -          System.out.println(new String((byte[])o));
> -        } else {
> -          System.out.println(o);
> +    try (Drillbit bit = new Drillbit(config, serviceSet); DistributedCache cache = dcache) {
> +      bit.run();
> +      cache.run();
> +
> +      DrillbitContext context = bit.getContext();
> +
> +
> +      MaterializedField intField = MaterializedField.create(new SchemaPath("int", ExpressionPosition.UNKNOWN),
> +          Types.required(TypeProtos.MinorType.INT));
> +      IntVector intVector = (IntVector) TypeHelper.getNewVector(intField, context.getAllocator());
> +      MaterializedField binField = MaterializedField.create(new SchemaPath("binary", ExpressionPosition.UNKNOWN),
> +          Types.required(TypeProtos.MinorType.VARBINARY));
> +      VarBinaryVector binVector = (VarBinaryVector) TypeHelper.getNewVector(binField, context.getAllocator());
> +      AllocationHelper.allocate(intVector, 4, 4);
> +      AllocationHelper.allocate(binVector, 4, 5);
> +      vectorList.add(intVector);
> +      vectorList.add(binVector);
> +
> +      intVector.getMutator().set(0, 0);
> +      binVector.getMutator().set(0, "ZERO".getBytes());
> +      intVector.getMutator().set(1, 1);
> +      binVector.getMutator().set(1, "ONE".getBytes());
> +      intVector.getMutator().set(2, 2);
> +      binVector.getMutator().set(2, "TWO".getBytes());
> +      intVector.getMutator().set(3, 3);
> +      binVector.getMutator().set(3, "THREE".getBytes());
> +      intVector.getMutator().setValueCount(4);
> +      binVector.getMutator().setValueCount(4);
> +
> +      VectorContainer container = new VectorContainer();
> +      container.addCollection(vectorList);
> +      container.setRecordCount(4);
> +      WritableBatch batch = WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container, false);
> +      CachedVectorContainer wrap = new CachedVectorContainer(batch, context.getAllocator());
> +
> +      DistributedMultiMap<CachedVectorContainer> mmap = cache.getMultiMap(CachedVectorContainer.class);
> +      mmap.put("vectors", wrap);
> +
> +      CachedVectorContainer newWrap = (CachedVectorContainer) mmap.get("vectors").iterator().next();
> +
> +      VectorAccessible newContainer = newWrap.get();
> +      for (VectorWrapper<?> w : newContainer) {
> +        ValueVector vv = w.getValueVector();
> +        int values = vv.getAccessor().getValueCount();
> +        for (int i = 0; i < values; i++) {
> +          Object o = vv.getAccessor().getObject(i);
> +          if (o instanceof byte[]) {
> +            System.out.println(new String((byte[]) o));
> +          } else {
> +            System.out.println(o);
> +          }
>         }
>       }
> +
> +      newWrap.clear();
>     }
> +
> +  }
> +
> +  @Test
> +  public void testHazelVectorCache() throws Exception {
> +    DrillConfig c = DrillConfig.create();
> +    HazelCache cache = new HazelCache(c, new TopLevelAllocator());
> +    cache.run();
> +    testCache(c, cache);
> +    cache.close();
> +  }
> +
> +  @Test
> +  public void testICache() throws Exception {
> +    DrillConfig c = DrillConfig.create();
> +    ICache cache = new ICache(c, new TopLevelAllocator());
> +    testCache(c, cache);
> +
>   }
> }
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> index 63bc0a9..a5dbfe5 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
> @@ -23,7 +23,7 @@ import net.hydromatic.optiq.tools.Frameworks;
> 
> import org.apache.drill.common.config.DrillConfig;
> import org.apache.drill.exec.ExecTest;
> -import org.apache.drill.exec.cache.LocalCache;
> +import org.apache.drill.exec.cache.local.LocalCache;
> import org.apache.drill.exec.memory.TopLevelAllocator;
> import org.apache.drill.exec.rpc.user.UserSession;
> import org.apache.drill.exec.server.DrillbitContext;
> 
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> index acb5929..3ccb96b 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
> @@ -18,12 +18,13 @@
> package org.apache.drill.exec.store.ischema;
> 
> 
> -import static org.mockito.Mockito.*;
> +import static org.mockito.Mockito.mock;
> +import static org.mockito.Mockito.when;
> import net.hydromatic.optiq.SchemaPlus;
> import net.hydromatic.optiq.tools.Frameworks;
> 
> import org.apache.drill.common.config.DrillConfig;
> -import org.apache.drill.exec.cache.LocalCache;
> +import org.apache.drill.exec.cache.local.LocalCache;
> import org.apache.drill.exec.memory.TopLevelAllocator;
> import org.apache.drill.exec.rpc.user.UserSession;
> import org.apache.drill.exec.server.DrillbitContext;
> @@ -40,7 +41,7 @@ public class OrphanSchema {
>    * @return root node of the created schema.
>    */
>   public static SchemaPlus create() throws Exception {
> -    
> +
>     final DrillConfig c = DrillConfig.create();
> 
>     // Mock up a context which will allow us to create a schema.
> @@ -51,7 +52,7 @@ public class OrphanSchema {
>     when(bitContext.getCache()).thenReturn(new LocalCache());
> 
>     bitContext.getCache().run();
> -    
> +
>     // Using the mock context, get the orphan schema.
>     StoragePluginRegistry r = new StoragePluginRegistry(bitContext);
>     r.init();
> 

[10/24] ispan

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
new file mode 100644
index 0000000..46d4eca
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ZookeeperCacheStore.java
@@ -0,0 +1,66 @@
+/**
+ * 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.drill.exec.cache.infinispan;
+
+import org.infinispan.marshall.core.MarshalledEntry;
+import org.infinispan.persistence.spi.ExternalStore;
+import org.infinispan.persistence.spi.InitializationContext;
+
+/**
+ * Stores the cached objects in zookeeper.  Objects are stored in /start/cache_name/key_name = data
+ * @param <K>
+ * @param <V>
+ */
+public class ZookeeperCacheStore<K, V> implements ExternalStore<K, V>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZookeeperCacheStore.class);
+
+  private String cacheName;
+
+  @Override
+  public void init(InitializationContext ctx) {
+    ctx.getConfiguration();
+
+  }
+
+  @Override
+  public MarshalledEntry<K, V> load(K key) {
+    return null;
+  }
+
+  @Override
+  public boolean contains(K key) {
+    return false;
+  }
+
+  @Override
+  public void start() {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  @Override
+  public void write(MarshalledEntry<K, V> entry) {
+  }
+
+  @Override
+  public boolean delete(K key) {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
new file mode 100644
index 0000000..e66cc90
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
@@ -0,0 +1,309 @@
+/**
+ * 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.drill.exec.cache.local;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.drill.common.util.DataInputInputStream;
+import org.apache.drill.common.util.DataOutputOutputStream;
+import org.apache.drill.exec.cache.Counter;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.DistributedMap;
+import org.apache.drill.exec.cache.DistributedMultiMap;
+import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.ByteArrayDataInput;
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
+
+public class LocalCache implements DistributedCache {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
+
+  private volatile Map<FragmentHandle, PlanFragment> handles;
+  private volatile ConcurrentMap<String, DistributedMap<?>> namedMaps;
+  private volatile ConcurrentMap<Class<?>, DistributedMap<?>> maps;
+  private volatile ConcurrentMap<Class<?>, DistributedMultiMap<?>> multiMaps;
+  private volatile ConcurrentMap<String, Counter> counters;
+  private static final BufferAllocator allocator = new TopLevelAllocator();
+
+  private static final ObjectMapper mapper = DrillConfig.create().getMapper();
+
+  @Override
+  public void close() throws IOException {
+    handles = null;
+  }
+
+  @Override
+  public void run() throws DrillbitStartupException {
+    handles = Maps.newConcurrentMap();
+    maps = Maps.newConcurrentMap();
+    multiMaps = Maps.newConcurrentMap();
+    counters = Maps.newConcurrentMap();
+    namedMaps = Maps.newConcurrentMap();
+  }
+
+  @Override
+  public PlanFragment getFragment(FragmentHandle handle) {
+//    logger.debug("looking for fragment with handle: {}", handle);
+    return handles.get(handle);
+  }
+
+  @Override
+  public void storeFragment(PlanFragment fragment) {
+//    logger.debug("Storing fragment: {}", fragment);
+    handles.put(fragment.getHandle(), fragment);
+  }
+
+  @Override
+  public <V extends DrillSerializable> DistributedMultiMap<V> getMultiMap(Class<V> clazz) {
+    DistributedMultiMap<V> mmap = (DistributedMultiMap<V>) multiMaps.get(clazz);
+    if (mmap == null) {
+      multiMaps.putIfAbsent(clazz, new LocalDistributedMultiMapImpl<V>(clazz));
+      return (DistributedMultiMap<V>) multiMaps.get(clazz);
+    } else {
+      return mmap;
+    }
+  }
+
+  @Override
+  public <V extends DrillSerializable> DistributedMap<V> getMap(Class<V> clazz) {
+    DistributedMap m = maps.get(clazz);
+    if (m == null) {
+      maps.putIfAbsent(clazz, new LocalDistributedMapImpl<V>(clazz));
+      return (DistributedMap<V>) maps.get(clazz);
+    } else {
+      return m;
+    }
+  }
+
+
+  @Override
+  public <V extends DrillSerializable> DistributedMap<V> getNamedMap(String name, Class<V> clazz) {
+    DistributedMap m = namedMaps.get(clazz);
+    if (m == null) {
+      namedMaps.putIfAbsent(name, new LocalDistributedMapImpl<V>(clazz));
+      return (DistributedMap<V>) namedMaps.get(name);
+    } else {
+      return m;
+    }
+  }
+
+  @Override
+  public Counter getCounter(String name) {
+    Counter c = counters.get(name);
+    if (c == null) {
+      counters.putIfAbsent(name, new LocalCounterImpl());
+      return counters.get(name);
+    } else {
+      return c;
+    }
+  }
+
+  public static ByteArrayDataOutput serialize(DrillSerializable obj) {
+    if(obj instanceof JacksonSerializable){
+      try{
+        ByteArrayDataOutput out = ByteStreams.newDataOutput();
+        out.write(mapper.writeValueAsBytes(obj));
+        return out;
+      }catch(Exception e){
+        throw new RuntimeException(e);
+      }
+    }
+
+    ByteArrayDataOutput out = ByteStreams.newDataOutput();
+    OutputStream outputStream = DataOutputOutputStream.constructOutputStream(out);
+    try {
+      obj.writeToStream(outputStream);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    try {
+      outputStream.flush();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return out;
+  }
+
+  public static <V extends DrillSerializable> V deserialize(byte[] bytes, Class<V> clazz) {
+    if(JacksonSerializable.class.isAssignableFrom(clazz)){
+      try{
+        return (V) mapper.readValue(bytes, clazz);
+      }catch(Exception e){
+        throw new RuntimeException(e);
+      }
+    }
+
+    ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
+    InputStream inputStream = DataInputInputStream.constructInputStream(in);
+    try {
+      V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
+      obj.readFromStream(inputStream);
+      return obj;
+    } catch (InstantiationException | IllegalAccessException | IOException | NoSuchMethodException | InvocationTargetException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public static class LocalDistributedMultiMapImpl<V extends DrillSerializable> implements DistributedMultiMap<V> {
+    private ArrayListMultimap<String, ByteArrayDataOutput> mmap;
+    private Class<V> clazz;
+
+    public LocalDistributedMultiMapImpl(Class<V> clazz) {
+      mmap = ArrayListMultimap.create();
+      this.clazz = clazz;
+    }
+
+    @Override
+    public Collection<V> get(String key) {
+      List<V> list = Lists.newArrayList();
+      for (ByteArrayDataOutput o : mmap.get(key)) {
+        list.add(deserialize(o.toByteArray(), this.clazz));
+      }
+      return list;
+    }
+
+    @Override
+    public void put(String key, DrillSerializable value) {
+      mmap.put(key, serialize(value));
+    }
+  }
+
+  public static class LocalDistributedMapImpl<V extends DrillSerializable> implements DistributedMap<V> {
+    protected ConcurrentMap<String, ByteArrayDataOutput> m;
+    protected Class<V> clazz;
+
+    public LocalDistributedMapImpl(Class<V> clazz) {
+      m = Maps.newConcurrentMap();
+      this.clazz = clazz;
+    }
+
+    @Override
+    public V get(String key) {
+      if (m.get(key) == null) return null;
+      ByteArrayDataOutput b = m.get(key);
+      byte[] bytes = b.toByteArray();
+      return (V) deserialize(m.get(key).toByteArray(), this.clazz);
+    }
+
+    @Override
+    public void put(String key, V value) {
+      m.put(key, serialize(value));
+    }
+
+    @Override
+    public void putIfAbsent(String key, V value) {
+      m.putIfAbsent(key, serialize(value));
+    }
+
+    @Override
+    public void putIfAbsent(String key, V value, long ttl, TimeUnit timeUnit) {
+      m.putIfAbsent(key, serialize(value));
+      logger.warn("Expiration not implemented in local map cache");
+    }
+
+    private class DeserializingTransformer implements Iterator<Map.Entry<String, V> >{
+      private Iterator<Map.Entry<String, ByteArrayDataOutput>> inner;
+
+      public DeserializingTransformer(Iterator<Entry<String, ByteArrayDataOutput>> inner) {
+        super();
+        this.inner = inner;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return inner.hasNext();
+      }
+
+      @Override
+      public Entry<String, V> next() {
+        return newEntry(inner.next());
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+
+      public Entry<String, V> newEntry(final Entry<String, ByteArrayDataOutput> input) {
+        return new Map.Entry<String, V>(){
+
+          @Override
+          public String getKey() {
+            return input.getKey();
+          }
+
+          @Override
+          public V getValue() {
+            return deserialize(input.getValue().toByteArray(), clazz);
+          }
+
+          @Override
+          public V setValue(V value) {
+            throw new UnsupportedOperationException();
+          }
+
+        };
+      }
+
+    }
+    @Override
+    public Iterator<Entry<String, V>> iterator() {
+      return new DeserializingTransformer(m.entrySet().iterator());
+    }
+  }
+
+  public static class LocalCounterImpl implements Counter {
+    private AtomicLong al = new AtomicLong();
+
+    @Override
+    public long get() {
+      return al.get();
+    }
+
+    @Override
+    public long incrementAndGet() {
+      return al.incrementAndGet();
+    }
+
+    @Override
+    public long decrementAndGet() {
+      return al.decrementAndGet();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index f105363..a0c439e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -25,13 +25,13 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.cache.CachedVectorContainer;
 import org.apache.drill.exec.cache.Counter;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.DistributedMap;
@@ -115,9 +115,9 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   private int recordCount;
 
   private final IntVector partitionKeyVector;
-  private final DistributedMap<VectorAccessibleSerializable> tableMap;
+  private final DistributedMap<CachedVectorContainer> tableMap;
   private final Counter minorFragmentSampleCount;
-  private final DistributedMultiMap<VectorAccessibleSerializable> mmap;
+  private final DistributedMultiMap<CachedVectorContainer> mmap;
   private final String mapKey;
   private List<VectorContainer> sampledIncomingBatches;
 
@@ -131,8 +131,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     this.completionFactor = pop.getCompletionFactor();
 
     DistributedCache cache = context.getDrillbitContext().getCache();
-    this.mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
-    this.tableMap = cache.getMap(VectorAccessibleSerializable.class);
+    this.mmap = cache.getMultiMap(CachedVectorContainer.class);
+    this.tableMap = cache.getMap(CachedVectorContainer.class);
     Preconditions.checkNotNull(tableMap);
 
     this.mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
@@ -220,7 +220,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     // into a serializable wrapper object, and then add to distributed map
 
     WritableBatch batch = WritableBatch.getBatchNoHVWrap(containerToCache.getRecordCount(), containerToCache, false);
-    VectorAccessibleSerializable sampleToSave = new VectorAccessibleSerializable(batch, oContext.getAllocator());
+    CachedVectorContainer sampleToSave = new CachedVectorContainer(batch, context.getAllocator());
 
     mmap.put(mapKey, sampleToSave);
     this.sampledIncomingBatches = builder.getHeldRecordBatches();
@@ -251,7 +251,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
         return false;
       }
 
-      VectorAccessibleSerializable finalTable = null;
+      CachedVectorContainer finalTable = null;
 
       long val = minorFragmentSampleCount.incrementAndGet();
       logger.debug("Incremented mfsc, got {}", val);
@@ -301,8 +301,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
 
     // Get all samples from distributed map
 
-    SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(oContext.getAllocator(), MAX_SORT_BYTES);
-    for (VectorAccessibleSerializable w : mmap.get(mapKey)) {
+    SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
+    for (CachedVectorContainer w : mmap.get(mapKey)) {
       containerBuilder.add(w.get());
     }
     VectorContainer allSamplesContainer = new VectorContainer();
@@ -346,7 +346,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     }
     candidatePartitionTable.setRecordCount(copier.getOutputRecords());
     WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false);
-    VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getDrillbitContext().getAllocator());
+    CachedVectorContainer wrap = new CachedVectorContainer(batch, context.getDrillbitContext().getAllocator());
     tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
 
     candidatePartitionTable.clear();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index 7297dc3..0e3181d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -22,7 +22,7 @@ import java.io.Closeable;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
-import org.apache.drill.exec.cache.HazelCache;
+import org.apache.drill.exec.cache.hazel.HazelCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
index c0b82bd..2078107 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
@@ -21,17 +21,17 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.drill.exec.cache.DistributedCache;
-import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.cache.local.LocalCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.LocalClusterCoordinator;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 
 public class RemoteServiceSet implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
-  
+
   private final DistributedCache cache;
   private final ClusterCoordinator coordinator;
-  
+
   public RemoteServiceSet(DistributedCache cache, ClusterCoordinator coordinator) {
     super();
     this.cache = cache;
@@ -46,16 +46,21 @@ public class RemoteServiceSet implements Closeable{
   public ClusterCoordinator getCoordinator() {
     return coordinator;
   }
-  
-  
+
+
   @Override
   public void close() throws IOException {
+    try{
     cache.close();
+    }catch(Exception e){
+      if(e instanceof IOException) throw (IOException) e;
+      throw new IOException("Failure while closing cache", e);
+    }
     coordinator.close();
   }
 
   public static RemoteServiceSet getLocalServiceSet(){
     return new RemoteServiceSet(new LocalCache(), new LocalClusterCoordinator());
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
index 99e712b..20722d9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -29,7 +29,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecTest;
 import org.apache.drill.exec.cache.DistributedCache;
-import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.cache.local.LocalCache;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.ops.QueryContext;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
new file mode 100644
index 0000000..13322f1
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
@@ -0,0 +1,94 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.OutputStream;
+import java.util.List;
+
+import org.infinispan.Cache;
+import org.infinispan.configuration.cache.CacheMode;
+import org.infinispan.configuration.cache.Configuration;
+import org.infinispan.configuration.cache.ConfigurationBuilder;
+import org.infinispan.configuration.global.GlobalConfiguration;
+import org.infinispan.configuration.global.GlobalConfigurationBuilder;
+import org.infinispan.manager.DefaultCacheManager;
+import org.infinispan.manager.EmbeddedCacheManager;
+
+import com.google.hive12.common.collect.Lists;
+
+public class ISpan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ISpan.class);
+
+
+  public static void main(String[] args) throws Exception{
+    GlobalConfiguration gc = new GlobalConfigurationBuilder().transport().defaultTransport().build();
+    Configuration c = new ConfigurationBuilder() //
+    .clustering().cacheMode(CacheMode.DIST_ASYNC) //
+    .storeAsBinary()
+    .build();
+    EmbeddedCacheManager ecm = new DefaultCacheManager(gc, c);
+
+    Cache<String, List<XT>> cache = ecm.getCache();
+    List<XT> items = Lists.newArrayList();
+    items.add(new XT(1));
+    items.add(new XT(2));
+
+    cache.put("items", items);
+    for(XT x : cache.get("items")){
+      System.out.println(x.i);
+    }
+
+
+  }
+
+  private static class XT extends AbstractDataSerializable{
+
+    int i =0;
+
+
+    public XT(int i) {
+      super();
+      this.i = i;
+    }
+
+    @Override
+    public void read(DataInput input) throws IOException {
+      i = input.readInt();
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+      output.writeInt(i);
+    }
+
+    @Override
+    public String toString() {
+      return "XT [i=" + i + "]";
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
index a3d39a3..7686614 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.cache;
 
-import com.beust.jcommander.internal.Lists;
+import java.util.List;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
@@ -25,67 +25,105 @@ import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecTest;
+import org.apache.drill.exec.cache.hazel.HazelCache;
+import org.apache.drill.exec.cache.infinispan.ICache;
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.record.*;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.VectorAccessible;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
-import org.apache.drill.exec.vector.*;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarBinaryVector;
 import org.junit.Test;
 
-import java.util.List;
+import com.beust.jcommander.internal.Lists;
 
-public class TestVectorCache  extends ExecTest{
+public class TestVectorCache extends ExecTest{
 
-  @Test
-  public void testVectorCache() throws Exception {
+  private void testCache(DrillConfig config, DistributedCache dcache) throws Exception {
     List<ValueVector> vectorList = Lists.newArrayList();
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    DrillConfig config = DrillConfig.create();
-    Drillbit bit = new Drillbit(config, serviceSet);
-    bit.run();
-    DrillbitContext context = bit.getContext();
-    HazelCache cache = new HazelCache(config, context.getAllocator());
-    cache.run();
 
-    MaterializedField intField = MaterializedField.create(SchemaPath.getSimplePath("int"), Types.required(TypeProtos.MinorType.INT));
-    IntVector intVector = (IntVector)TypeHelper.getNewVector(intField, context.getAllocator());
-    MaterializedField binField = MaterializedField.create(new SchemaPath("binary", ExpressionPosition.UNKNOWN), Types.required(TypeProtos.MinorType.VARBINARY));
-    VarBinaryVector binVector = (VarBinaryVector)TypeHelper.getNewVector(binField, context.getAllocator());
-    AllocationHelper.allocate(intVector, 4, 4);
-    AllocationHelper.allocate(binVector, 4, 5);
-    vectorList.add(intVector);
-    vectorList.add(binVector);
-
-    intVector.getMutator().setSafe(0, 0); binVector.getMutator().setSafe(0, "ZERO".getBytes());
-    intVector.getMutator().setSafe(1, 1); binVector.getMutator().setSafe(1, "ONE".getBytes());
-    intVector.getMutator().setSafe(2, 2); binVector.getMutator().setSafe(2, "TWO".getBytes());
-    intVector.getMutator().setSafe(3, 3); binVector.getMutator().setSafe(3, "THREE".getBytes());
-    intVector.getMutator().setValueCount(4);
-    binVector.getMutator().setValueCount(4);
-
-    VectorContainer container = new VectorContainer();
-    container.addCollection(vectorList);
-    container.setRecordCount(4);
-    WritableBatch batch = WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container, false);
-    VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getAllocator());
-
-    DistributedMultiMap<VectorAccessibleSerializable> mmap = cache.getMultiMap(VectorAccessibleSerializable.class);
-    mmap.put("vectors", wrap);
-    VectorAccessibleSerializable newWrap = (VectorAccessibleSerializable)mmap.get("vectors").iterator().next();
-
-    VectorAccessible newContainer = newWrap.get();
-    for (VectorWrapper w : newContainer) {
-      ValueVector vv = w.getValueVector();
-      int values = vv.getAccessor().getValueCount();
-      for (int i = 0; i < values; i++) {
-        Object o = vv.getAccessor().getObject(i);
-        if (o instanceof byte[]) {
-          System.out.println(new String((byte[])o));
-        } else {
-          System.out.println(o);
+    try (Drillbit bit = new Drillbit(config, serviceSet); DistributedCache cache = dcache) {
+      bit.run();
+      cache.run();
+
+      DrillbitContext context = bit.getContext();
+
+
+      MaterializedField intField = MaterializedField.create(new SchemaPath("int", ExpressionPosition.UNKNOWN),
+          Types.required(TypeProtos.MinorType.INT));
+      IntVector intVector = (IntVector) TypeHelper.getNewVector(intField, context.getAllocator());
+      MaterializedField binField = MaterializedField.create(new SchemaPath("binary", ExpressionPosition.UNKNOWN),
+          Types.required(TypeProtos.MinorType.VARBINARY));
+      VarBinaryVector binVector = (VarBinaryVector) TypeHelper.getNewVector(binField, context.getAllocator());
+      AllocationHelper.allocate(intVector, 4, 4);
+      AllocationHelper.allocate(binVector, 4, 5);
+      vectorList.add(intVector);
+      vectorList.add(binVector);
+
+      intVector.getMutator().set(0, 0);
+      binVector.getMutator().set(0, "ZERO".getBytes());
+      intVector.getMutator().set(1, 1);
+      binVector.getMutator().set(1, "ONE".getBytes());
+      intVector.getMutator().set(2, 2);
+      binVector.getMutator().set(2, "TWO".getBytes());
+      intVector.getMutator().set(3, 3);
+      binVector.getMutator().set(3, "THREE".getBytes());
+      intVector.getMutator().setValueCount(4);
+      binVector.getMutator().setValueCount(4);
+
+      VectorContainer container = new VectorContainer();
+      container.addCollection(vectorList);
+      container.setRecordCount(4);
+      WritableBatch batch = WritableBatch.getBatchNoHVWrap(container.getRecordCount(), container, false);
+      CachedVectorContainer wrap = new CachedVectorContainer(batch, context.getAllocator());
+
+      DistributedMultiMap<CachedVectorContainer> mmap = cache.getMultiMap(CachedVectorContainer.class);
+      mmap.put("vectors", wrap);
+
+      CachedVectorContainer newWrap = (CachedVectorContainer) mmap.get("vectors").iterator().next();
+
+      VectorAccessible newContainer = newWrap.get();
+      for (VectorWrapper<?> w : newContainer) {
+        ValueVector vv = w.getValueVector();
+        int values = vv.getAccessor().getValueCount();
+        for (int i = 0; i < values; i++) {
+          Object o = vv.getAccessor().getObject(i);
+          if (o instanceof byte[]) {
+            System.out.println(new String((byte[]) o));
+          } else {
+            System.out.println(o);
+          }
         }
       }
+
+      newWrap.clear();
     }
+
+  }
+
+  @Test
+  public void testHazelVectorCache() throws Exception {
+    DrillConfig c = DrillConfig.create();
+    HazelCache cache = new HazelCache(c, new TopLevelAllocator());
+    cache.run();
+    testCache(c, cache);
+    cache.close();
+  }
+
+  @Test
+  public void testICache() throws Exception {
+    DrillConfig c = DrillConfig.create();
+    ICache cache = new ICache(c, new TopLevelAllocator());
+    testCache(c, cache);
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
index 63bc0a9..a5dbfe5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
@@ -23,7 +23,7 @@ import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ExecTest;
-import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.cache.local.LocalCache;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8621b682/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
index acb5929..3ccb96b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
@@ -18,12 +18,13 @@
 package org.apache.drill.exec.store.ischema;
 
 
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 import net.hydromatic.optiq.SchemaPlus;
 import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.cache.local.LocalCache;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -40,7 +41,7 @@ public class OrphanSchema {
    * @return root node of the created schema.
    */
   public static SchemaPlus create() throws Exception {
-    
+
     final DrillConfig c = DrillConfig.create();
 
     // Mock up a context which will allow us to create a schema.
@@ -51,7 +52,7 @@ public class OrphanSchema {
     when(bitContext.getCache()).thenReturn(new LocalCache());
 
     bitContext.getCache().run();
-    
+
     // Using the mock context, get the orphan schema.
     StoragePluginRegistry r = new StoragePluginRegistry(bitContext);
     r.init();


[02/24] git commit: wip to support op numbering throughout exec.

Posted by ja...@apache.org.
wip to support op numbering throughout exec.


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

Branch: refs/heads/diagnostics2
Commit: ebf3d340497afeceb93d7e7c8211c5eebfce9ebf
Parents: e14a38c
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri May 16 10:52:15 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:11:22 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/fragment/Materializer.java     | 27 +++++++-----
 .../planner/physical/BroadcastExchangePrel.java |  2 +
 .../drill/exec/planner/physical/FilterPrel.java |  2 +-
 .../exec/planner/physical/HashAggPrel.java      |  2 +
 .../exec/planner/physical/HashJoinPrel.java     |  1 +
 .../physical/HashToMergeExchangePrel.java       |  1 +
 .../physical/HashToRandomExchangePrel.java      |  2 +
 .../drill/exec/planner/physical/LimitPrel.java  |  1 +
 .../exec/planner/physical/MergeJoinPrel.java    |  1 +
 .../planner/physical/PhysicalPlanCreator.java   | 14 +++++-
 .../exec/planner/physical/ProjectPrel.java      |  1 +
 .../drill/exec/planner/physical/ScanPrel.java   |  1 +
 .../drill/exec/planner/physical/ScreenPrel.java |  2 +
 .../physical/SelectionVectorRemoverPrel.java    |  5 ++-
 .../physical/SingleMergeExchangePrel.java       |  2 +
 .../drill/exec/planner/physical/SortPrel.java   |  1 +
 .../exec/planner/physical/StreamAggPrel.java    |  2 +-
 .../drill/exec/planner/physical/TopNPrel.java   |  1 +
 .../planner/physical/UnionExchangePrel.java     |  1 +
 .../drill/exec/planner/physical/WriterPrel.java |  5 ++-
 .../planner/physical/explain/PrelSequencer.java | 46 +++++++++++++++-----
 .../planner/sql/handlers/DefaultSqlHandler.java |  3 +-
 .../apache/drill/exec/TestOpSerialization.java  | 10 +++--
 23 files changed, 102 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
index 87078a2..ef9146a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -30,21 +30,23 @@ import com.google.common.collect.Lists;
 public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Materializer.IndexedFragmentNode, ExecutionSetupException>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Materializer.class);
 
-  
+
   @Override
   public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws ExecutionSetupException {
     iNode.addAllocation(exchange);
     if(exchange == iNode.getNode().getSendingExchange()){
-      
+
       // this is a sending exchange.
       PhysicalOperator child = exchange.getChild().accept(this, iNode);
       PhysicalOperator materializedSender = exchange.getSender(iNode.getMinorFragmentId(), child);
+      materializedSender.setOperatorId(0);
 //      logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child);
       return materializedSender;
-      
+
     }else{
       // receiving exchange.
       PhysicalOperator materializedReceiver = exchange.getReceiver(iNode.getMinorFragmentId());
+      materializedReceiver.setOperatorId(Short.MAX_VALUE & exchange.getOperatorId());
 //      logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver);
       return materializedReceiver;
     }
@@ -52,7 +54,9 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
 
   @Override
   public PhysicalOperator visitGroupScan(GroupScan groupScan, IndexedFragmentNode iNode) throws ExecutionSetupException {
-    return groupScan.getSpecificScan(iNode.getMinorFragmentId());
+    PhysicalOperator child = groupScan.getSpecificScan(iNode.getMinorFragmentId());
+    child.setOperatorId(Short.MAX_VALUE & groupScan.getOperatorId());
+    return child;
   }
 
   @Override
@@ -67,9 +71,10 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
     PhysicalOperator child = store.getChild().accept(this, iNode);
 
     iNode.addAllocation(store);
-    
+
     try {
       PhysicalOperator o = store.getSpecificStore(child, iNode.getMinorFragmentId());
+      o.setOperatorId(Short.MAX_VALUE & store.getOperatorId());
 //      logger.debug("New materialized store node {} with child {}", o, child);
       return o;
     } catch (PhysicalOperatorSetupException e) {
@@ -85,13 +90,15 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
     for(PhysicalOperator child : op){
       children.add(child.accept(this, iNode));
     }
-    return op.getNewWithChildren(children);
+    PhysicalOperator newOp = op.getNewWithChildren(children);
+    newOp.setOperatorId(Short.MAX_VALUE & op.getOperatorId());
+    return newOp;
   }
-  
+
   public static class IndexedFragmentNode{
     final Wrapper info;
     final int minorFragmentId;
-    
+
     public IndexedFragmentNode(int minorFragmentId, Wrapper info) {
       super();
       this.info = info;
@@ -113,7 +120,7 @@ public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Mate
     public void addAllocation(PhysicalOperator pop) {
       info.addAllocation(pop);
     }
-    
+
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
index e0f3ee1..8b1c720 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/BroadcastExchangePrel.java
@@ -89,6 +89,8 @@ public class BroadcastExchangePrel extends ExchangePrel{
     }
 
     BroadcastExchange g = new BroadcastExchange(childPOP);
+    g.setOperatorId(creator.getOperatorId(this));
+
     return g;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
index 8420e08..9632911 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
@@ -56,7 +56,7 @@ public class FilterPrel extends DrillFilterRelBase implements Prel {
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
 
     Filter p = new Filter(childPOP, getFilterExpression(new DrillParseContext()), 1.0f);
-
+    p.setOperatorId(creator.getOperatorId(this));
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
index b2378be..6377e35 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
@@ -110,6 +110,8 @@ public class HashAggPrel extends AggregateRelBase implements Prel{
         exprs.toArray(new NamedExpression[exprs.size()]),
         1.0f);
 
+    g.setOperatorId(creator.getOperatorId(this));
+
     return g;
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
index 1a528d5..87da31e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
@@ -106,6 +106,7 @@ public class HashJoinPrel  extends DrillJoinRelBase implements Prel {
     }
 
     HashJoinPOP hjoin = new HashJoinPOP(leftPop, rightPop, conditions, jtype);
+    hjoin.setOperatorId(creator.getOperatorId(this));
 
     return hjoin;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
index 262fd8c..0539a33 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToMergeExchangePrel.java
@@ -90,6 +90,7 @@ public class HashToMergeExchangePrel extends ExchangePrel {
     HashToMergeExchange g = new HashToMergeExchange(childPOP,
         PrelUtil.getHashExpression(this.distFields, getChild().getRowType()),
         PrelUtil.getOrdering(this.collation, getChild().getRowType()));
+    g.setOperatorId(creator.getOperatorId(this));
     return g;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
index ec9ed79..a5699cd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -94,6 +94,8 @@ public class HashToRandomExchangePrel extends ExchangePrel {
     if(PrelUtil.getSettings(getCluster()).isSingleMode()) return childPOP;
 
     HashToRandomExchange g = new HashToRandomExchange(childPOP, PrelUtil.getHashExpression(this.fields, getChild().getRowType()));
+    g.setOperatorId(creator.getOperatorId(this));
+
     return g;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
index 5986fde..794593a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
@@ -65,6 +65,7 @@ public class LimitPrel extends DrillLimitRelBase implements Prel {
     Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
 
     Limit limit = new Limit(childPOP, first, last);
+    limit.setOperatorId(creator.getOperatorId(this));
 
     return limit;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index fe03c40..400c6a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -110,6 +110,7 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     }
 
     MergeJoinPOP mjoin = new MergeJoinPOP(leftPop, rightPop, conditions, jtype);
+    mjoin.setOperatorId(creator.getOperatorId(this));
 
     return mjoin;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
index 9ac07d3..f4189e4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.drill.common.logical.PlanProperties;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
@@ -27,20 +28,24 @@ import org.apache.drill.common.logical.PlanProperties.PlanType;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.physical.explain.PrelSequencer.OpId;
 
 import com.google.common.collect.Lists;
+import com.google.hive12.common.collect.Maps;
 
 
 public class PhysicalPlanCreator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanCreator.class);
 
+  private final Map<Prel, OpId> opIdMap;
+
   private List<PhysicalOperator> popList;
   private final QueryContext context;
   PhysicalPlan plan = null;
 
-  public PhysicalPlanCreator(QueryContext context) {
+  public PhysicalPlanCreator(QueryContext context, Map<Prel, OpId> opIdMap) {
     this.context = context;
+    this.opIdMap = opIdMap;
     popList = Lists.newArrayList();
   }
 
@@ -48,6 +53,11 @@ public class PhysicalPlanCreator {
     return context;
   }
 
+  public int getOperatorId(Prel prel){
+    OpId id = opIdMap.get(prel);
+    return id.getAsSingleInt();
+  }
+
   public PhysicalPlan build(Prel rootPrel, boolean forceRebuild) {
 
     if (plan != null && !forceRebuild) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
index 1aa34d3..70dca25 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -57,6 +57,7 @@ public class ProjectPrel extends DrillProjectRelBase implements Prel{
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
 
     Project p = new Project(this.getProjectExpressions(new DrillParseContext()),  childPOP);
+    p.setOperatorId(creator.getOperatorId(this));
 
     return p;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index 74cd7a9..8461e24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -73,6 +73,7 @@ public class ScanPrel extends AbstractRelNode implements DrillScanPrel {
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator)
       throws IOException {
+    groupScan.setOperatorId(creator.getOperatorId(this));
     return groupScan;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
index 36bf796..d02ed44 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
@@ -51,6 +51,8 @@ public class ScreenPrel extends DrillScreenRelBase implements Prel {
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
 
     Screen s = new Screen(childPOP, creator.getContext().getCurrentEndpoint());
+    s.setOperatorId(creator.getOperatorId(this));
+
     return s;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
index 63cdcaa..fd07749 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SelectionVectorRemoverPrel.java
@@ -32,7 +32,10 @@ public class SelectionVectorRemoverPrel extends SinglePrel{
 
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
-    return new SelectionVectorRemover( ((Prel)getChild()).getPhysicalOperator(creator));
+    SelectionVectorRemover r =  new SelectionVectorRemover( ((Prel)getChild()).getPhysicalOperator(creator));
+    r.setOperatorId(creator.getOperatorId(this));
+    return r;
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
index 05d6e89..99d99a7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
@@ -89,6 +89,8 @@ public class SingleMergeExchangePrel extends ExchangePrel {
     if(PrelUtil.getSettings(getCluster()).isSingleMode()) return childPOP;
 
     SingleMergeExchange g = new SingleMergeExchange(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()));
+    g.setOperatorId(creator.getOperatorId(this));
+
     return g;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index d582bc6..fa5e900 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -73,6 +73,7 @@ public class SortPrel extends SortRel implements Prel {
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
 
     Sort g = new ExternalSort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
+    g.setOperatorId(creator.getOperatorId(this));
 
     return g;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index 5fb758a..a95d926 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -109,7 +109,7 @@ public class StreamAggPrel extends AggregateRelBase implements Prel{
 
     Prel child = (Prel) this.getChild();
     StreamingAggregate g = new StreamingAggregate(child.getPhysicalOperator(creator), keys.toArray(new NamedExpression[keys.size()]), exprs.toArray(new NamedExpression[exprs.size()]), 1.0f);
-
+    g.setOperatorId(creator.getOperatorId(this));
     return g;
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
index 3c8cfe0..0067926 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
@@ -61,6 +61,7 @@ public class TopNPrel extends SinglePrel {
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
 
     TopN topN = new TopN(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false, this.limit);
+    topN.setOperatorId(creator.getOperatorId(this));
 
     return topN;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
index 5d6b85d..f14df72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
@@ -81,6 +81,7 @@ public class UnionExchangePrel extends ExchangePrel {
     if(PrelUtil.getSettings(getCluster()).isSingleMode()) return childPOP;
 
     UnionExchange g = new UnionExchange(childPOP);
+    g.setOperatorId(creator.getOperatorId(this));
     return g;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
index 4cefeb5..e948125 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
@@ -43,7 +43,10 @@ public class WriterPrel extends DrillWriterRelBase implements Prel {
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    return getCreateTableEntry().getWriter(child.getPhysicalOperator(creator));
+    PhysicalOperator g = getCreateTableEntry().getWriter(child.getPhysicalOperator(creator));
+    g.setOperatorId(creator.getOperatorId(this));
+
+    return g;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
index 169deca..2ab6c74 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/PrelSequencer.java
@@ -42,14 +42,17 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
       if (rel == null) {
         return null;
       }
-      PrelSequencer s = new PrelSequencer();
       final StringWriter sw = new StringWriter();
-      final RelWriter planWriter = new NumberingRelWriter(s.go(rel), new PrintWriter(sw), explainlevel);
+      final RelWriter planWriter = new NumberingRelWriter(getIdMap(rel), new PrintWriter(sw), explainlevel);
       rel.explain(planWriter);
       return sw.toString();
 
   }
 
+  public static Map<Prel, OpId> getIdMap(Prel rel){
+    PrelSequencer s = new PrelSequencer();
+    return s.go(rel);
+  }
 
 
   static class Frag implements Iterable<Frag>{
@@ -110,7 +113,7 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
 
   }
 
-  static class OpId{
+  public static class OpId{
     int fragmentId;
     int opId;
     public OpId(int fragmentId, int opId) {
@@ -118,6 +121,21 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
       this.fragmentId = fragmentId;
       this.opId = opId;
     }
+
+
+    public int getFragmentId() {
+      return fragmentId;
+    }
+
+
+    public int getOpId() {
+      return opId;
+    }
+
+    public int getAsSingleInt(){
+      return (fragmentId << 16) + opId;
+    }
+
     @Override
     public int hashCode() {
       final int prime = 31;
@@ -172,19 +190,27 @@ public class PrelSequencer implements PrelVisitor<Void, PrelSequencer.Frag, Runt
     }
 
     // for each fragment, do a dfs of operators to assign operator ids.
-    Map<Prel, OpId> ids = Maps.newHashMap();
+    Map<Prel, OpId> ids = Maps.newIdentityHashMap();
+
+    ids.put(rootFrag.root, new OpId(0, 0));
     for(Frag f : frags){
-      int id = 0;
+      int id = 1;
       Queue<Prel> ops = Lists.newLinkedList();
       ops.add(f.root);
       while(!ops.isEmpty()){
         Prel p = ops.remove();
-        if(p instanceof ExchangePrel && p != f.root) continue;
-        ids.put(p, new OpId(f.majorFragmentId, id++) );
+        boolean isExchange = p instanceof ExchangePrel;
+
+        if(p != f.root){      // we account for exchanges as receviers to guarantee unique identifiers.
+          ids.put(p, new OpId(f.majorFragmentId, id++) );
+        }
+
 
-        List<Prel> children = Lists.reverse(Lists.newArrayList(p.iterator()));
-        for(Prel child : children){
-          ops.add(child);
+        if(!isExchange || p == f.root){
+          List<Prel> children = Lists.reverse(Lists.newArrayList(p.iterator()));
+          for(Prel child : children){
+            ops.add(child);
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 1cb3cfb..b06432a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -132,8 +132,7 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
   }
 
   protected PhysicalOperator convertToPop(Prel prel) throws IOException {
-
-    PhysicalPlanCreator creator = new PhysicalPlanCreator(context);
+    PhysicalPlanCreator creator = new PhysicalPlanCreator(context, PrelSequencer.getIdMap(prel));
     PhysicalOperator op = prel.getPhysicalOperator(creator);
     return op;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ebf3d340/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
index 3040de2..ad1d6b6 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
@@ -12,6 +12,7 @@ import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.Filter;
 import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.UnionExchange;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.store.mock.MockSubScanPOP;
@@ -27,10 +28,12 @@ public class TestOpSerialization {
     DrillConfig c = DrillConfig.create();
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     MockSubScanPOP s = new MockSubScanPOP("abc", null);
-    s.setOperatorId(2);
+    s.setOperatorId(3);
     Filter f = new Filter(s, new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN), 0.1f);
-    f.setOperatorId(1);
-    Screen screen = new Screen(f, CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    f.setOperatorId(2);
+    UnionExchange e = new UnionExchange(f);
+    e.setOperatorId(1);
+    Screen screen = new Screen(e, CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     screen.setOperatorId(0);
 
     boolean reversed = false;
@@ -38,6 +41,7 @@ public class TestOpSerialization {
 
       List<PhysicalOperator> pops = Lists.newArrayList();
       pops.add(s);
+      pops.add(e);
       pops.add(f);
       pops.add(screen);
 


[13/24] diag wip

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/protocol/src/main/protobuf/BitControl.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
index d9fa4b4..386f692 100644
--- a/protocol/src/main/protobuf/BitControl.proto
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -40,27 +40,8 @@ message BitStatus {
 }
 
 message FragmentStatus {
-  
-  enum FragmentState {
-    SENDING = 0;
-    AWAITING_ALLOCATION = 1;
-    RUNNING = 2;
-    FINISHED = 3;
-    CANCELLED = 4;
-    FAILED = 5;
-  }
-  
-  optional int64 memory_use = 1;
-  optional int64 batches_completed = 2;
-  optional int64 records_completed = 3;
-  optional int32 estimated_completion_percentage = 4;
-  optional FragmentState state = 5;
-  optional int64 data_processed = 6;
-  
-  optional FragmentHandle handle = 7;
-  optional exec.shared.DrillPBError error = 8;
-  optional int64 running_time = 9;
-  optional exec.shared.MinorFragmentProfile profile = 10;
+  optional exec.shared.MinorFragmentProfile profile = 1;
+  optional FragmentHandle handle = 2;
 }
 
 message PlanFragment {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index c379112..9a3dd2a 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -82,7 +82,8 @@ message QueryProfile {
   optional int64 start = 3;
   optional int64 end = 4;
   optional string query = 5;
-  repeated MajorFragmentProfile fragment_profile = 6;
+  optional string plan = 6;
+  repeated MajorFragmentProfile fragment_profile = 7;
 }
 
 message MajorFragmentProfile {
@@ -91,15 +92,19 @@ message MajorFragmentProfile {
 } 
 
 message MinorFragmentProfile {
-  optional int32 minor_fragment_id = 1;
-  repeated OperatorProfile operator_profile = 2; 
-  optional int64 start_time = 3;
-  optional int64 end_time = 4;  
+  optional FragmentState state = 1;
+  optional DrillPBError error = 2; 
+  optional int32 minor_fragment_id = 3;
+  repeated OperatorProfile operator_profile = 4; 
+  optional int64 start_time = 5;
+  optional int64 end_time = 6;
+  optional int64 memory_used = 7;
+  optional int64 max_memory_used = 8;
+  optional DrillbitEndpoint endpoint = 9;
 }
 
 message OperatorProfile {
   repeated StreamProfile input_profile = 1;
-  optional StreamProfile output_profile = 2;
   optional int32 operator_id = 3;
   optional int32 operator_type = 4;
   optional int64 setup_nanos = 5;
@@ -120,6 +125,14 @@ message MetricValue {
   optional double double_value = 3;
 }
 
+enum FragmentState {
+  SENDING = 0;
+  AWAITING_ALLOCATION = 1;
+  RUNNING = 2;
+  FINISHED = 3;
+  CANCELLED = 4;
+  FAILED = 5;
+}
 
 enum CoreOperatorType {
   SINGLE_SENDER = 0;


[03/24] git commit: Enhance PhysicalOperator to use OperatorIds for references rather than random ids.

Posted by ja...@apache.org.
Enhance PhysicalOperator to use OperatorIds for references rather than random ids.


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

Branch: refs/heads/diagnostics2
Commit: e14a38c58157af7f379378cdd97602ceea6fa9d2
Parents: 1bc276d
Author: Jacques Nadeau <ja...@apache.org>
Authored: Fri May 16 08:47:51 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:11:22 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/physical/base/AbstractBase.java  | 21 ++++--
 .../exec/physical/base/PhysicalOperator.java    | 21 ++++--
 .../apache/drill/exec/TestOpSerialization.java  | 67 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e14a38c5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
index a79cbc3..a028252 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
@@ -18,9 +18,9 @@
 package org.apache.drill.exec.physical.base;
 
 import org.apache.drill.common.graph.GraphVisitor;
-import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 
 public abstract class AbstractBase implements PhysicalOperator{
@@ -28,7 +28,7 @@ public abstract class AbstractBase implements PhysicalOperator{
 
   protected long initialAllocation = 1000000L;
   protected long maxAllocation = 10000000000L;
-
+  private int id;
 
   @Override
   public void accept(GraphVisitor<PhysicalOperator> visitor) {
@@ -36,16 +36,25 @@ public abstract class AbstractBase implements PhysicalOperator{
     if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
     for(PhysicalOperator o : this){
       Preconditions.checkNotNull(o, String.format("Null in iterator for pop %s.", this));
-      o.accept(visitor);  
+      o.accept(visitor);
     }
     visitor.leave(this);
   }
-  
+
   @Override
   public boolean isExecutable() {
     return true;
   }
-  
+
+  public final void setOperatorId(int id){
+    this.id = id;
+  }
+
+  @Override
+  public int getOperatorId() {
+    return id;
+  }
+
   @Override
   public SelectionVectorMode getSVMode() {
     return SelectionVectorMode.NONE;
@@ -60,5 +69,5 @@ public abstract class AbstractBase implements PhysicalOperator{
   public long getMaxAllocation() {
     return maxAllocation;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e14a38c5/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
index db57922..483c364 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -22,42 +22,44 @@ import java.util.List;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.graph.GraphValue;
 import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonIdentityInfo;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.annotation.ObjectIdGenerators;
 
 @JsonInclude(Include.NON_NULL)
 @JsonPropertyOrder({ "@id" })
-@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+@JsonIdentityInfo(generator = ObjectIdGenerators.PropertyGenerator.class, property = "@id")
 @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
 public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
 
   /**
    * Get the cost of execution of this particular operator.
-   * 
+   *
    * @return
    */
   @JsonIgnore
   public OperatorCost getCost();
-  
+
   /**
    * Get the estimated size of this particular operator.
    * @return
    */
   @JsonIgnore
   public Size getSize();
-  
+
   /**
    * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
    * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
    * Exec sub components.
-   * 
+   *
    * @return
    */
   @JsonIgnore
@@ -70,10 +72,10 @@ public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
    */
   @JsonIgnore
   public SelectionVectorMode getSVMode();
-  
+
   /**
    * Provides capability to build a set of output based on traversing a query graph tree.
-   * 
+   *
    * @param physicalVisitor
    * @return
    */
@@ -97,4 +99,9 @@ public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
    */
   public long getMaxAllocation();
 
+  @JsonProperty("@id")
+  public int getOperatorId();
+
+  @JsonProperty("@id")
+  public void setOperatorId(int id);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e14a38c5/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
new file mode 100644
index 0000000..3040de2
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
@@ -0,0 +1,67 @@
+package org.apache.drill.exec;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.logical.PlanProperties;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.store.mock.MockSubScanPOP;
+import org.junit.Test;
+
+import com.google.hive12.common.collect.Lists;
+
+public class TestOpSerialization {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOpSerialization.class);
+
+  @Test
+  public void testSerializedDeserialize() throws Throwable {
+    DrillConfig c = DrillConfig.create();
+    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    MockSubScanPOP s = new MockSubScanPOP("abc", null);
+    s.setOperatorId(2);
+    Filter f = new Filter(s, new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN), 0.1f);
+    f.setOperatorId(1);
+    Screen screen = new Screen(f, CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    screen.setOperatorId(0);
+
+    boolean reversed = false;
+    while(true){
+
+      List<PhysicalOperator> pops = Lists.newArrayList();
+      pops.add(s);
+      pops.add(f);
+      pops.add(screen);
+
+      if(reversed) pops = Lists.reverse(pops);
+      PhysicalPlan plan1 = new PhysicalPlan(PlanProperties.builder().build(), pops);
+      String json = plan1.unparse(c.getMapper().writer());
+      System.out.println(json);
+
+      PhysicalPlan plan2 = reader.readPhysicalPlan(json);
+      System.out.println("++++++++");
+      System.out.println(plan2.unparse(c.getMapper().writer()));
+
+      PhysicalOperator root = plan2.getSortedOperators(false).iterator().next();
+      assertEquals(0, root.getOperatorId());
+      PhysicalOperator o1 = root.iterator().next();
+      assertEquals(1, o1.getOperatorId());
+      PhysicalOperator o2 = o1.iterator().next();
+      assertEquals(2, o2.getOperatorId());
+      if(reversed) break;
+      reversed = !reversed;
+    }
+
+
+
+
+  }
+}


[24/24] git commit: fix to use delimited for protobuf serialization

Posted by ja...@apache.org.
fix to use delimited for protobuf serialization


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

Branch: refs/heads/diagnostics2
Commit: c34a190fc5547a31876a58eaceb3d11eae8b3ca5
Parents: e2740df
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 18:14:26 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 18:14:26 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/drill/exec/cache/ProtoSerializable.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/c34a190f/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
index 1538a85..0a2aa4a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoSerializable.java
@@ -45,12 +45,12 @@ public abstract class ProtoSerializable<V extends Message> extends AbstractStrea
 
   @Override
   public void readFromStream(InputStream input) throws IOException {
-    obj = protoParser.parseFrom(input);
+    obj = protoParser.parseDelimitedFrom(input);
   }
 
   @Override
   public void writeToStream(OutputStream output) throws IOException {
-    obj.writeTo(output);
+    obj.writeDelimitedTo(output);
   }
 
   public static class PlanFragmentSerializable extends ProtoSerializable<PlanFragment>{


[19/24] git commit: more diag fixes

Posted by ja...@apache.org.
more diag fixes


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/79054a85
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/79054a85
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/79054a85

Branch: refs/heads/diagnostics2
Commit: 79054a85a979e0d2640855edb0e7fd96b69397fb
Parents: f4c37bf
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 08:38:11 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 13:32:00 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |   3 +
 .../drill/exec/cache/CachedVectorContainer.java |  24 +++-
 .../drill/exec/cache/local/LocalCache.java      |   7 +-
 .../apache/drill/exec/client/DrillClient.java   |   2 +-
 .../drill/exec/expr/EvaluationVisitor.java      |   3 +-
 .../exec/physical/impl/join/MergeJoinBatch.java | 132 ++++++++++---------
 .../impl/project/ProjectRecordBatch.java        |   7 +-
 .../exec/planner/common/DrillWriterRelBase.java |  15 +++
 .../exec/planner/logical/DrillWriterRel.java    |   1 +
 .../drill/exec/planner/physical/WriterPrel.java |   7 +-
 .../physical/visitor/FinalColumnReorderer.java  |   5 +
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   2 +-
 .../org/apache/drill/exec/rpc/BasicServer.java  |   5 +-
 .../drill/exec/rpc/control/ControllerImpl.java  |   6 +-
 .../exec/rpc/data/DataConnectionCreator.java    |   6 +-
 .../org/apache/drill/exec/server/Drillbit.java  |  17 ++-
 .../drill/exec/service/ServiceEngine.java       |  18 +--
 .../drill/exec/store/sys/SystemTableScan.java   |   6 +
 .../src/main/resources/drill-module.conf        |   4 +
 .../apache/drill/exec/server/TestBitRpc.java    |   2 +-
 .../exec/store/json/JsonRecordReader2Test.java  |   4 +-
 exec/java-exec/src/test/resources/logback.xml   |   2 +-
 pom.xml                                         |   2 +-
 23 files changed, 179 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 238fae9..d9e0833 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -71,5 +71,8 @@ public interface ExecConstants {
   public static final OptionValidator OUTPUT_FORMAT_VALIDATOR = new StringValidator(OUTPUT_FORMAT_OPTION, "parquet");
   public static final String PARQUET_BLOCK_SIZE = "parquet.block.size";
   public static final OptionValidator PARQUET_BLOCK_SIZE_VALIDATOR = new LongValidator(PARQUET_BLOCK_SIZE, 512*1024*1024);
+  public static final String HTTP_ENABLE = "drill.exec.http.enabled";
+  public static final String HTTP_PORT = "drill.exec.http.port";
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
index 1447e28..da0b186 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/CachedVectorContainer.java
@@ -18,6 +18,8 @@
 package org.apache.drill.exec.cache;
 
 import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
@@ -29,7 +31,7 @@ import org.apache.drill.exec.record.WritableBatch;
 public class CachedVectorContainer extends LoopedAbstractDrillSerializable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CachedVectorContainer.class);
 
-  private final byte[] data;
+  private byte[] data;
   private final BufferAllocator allocator;
   private VectorContainer container;
 
@@ -42,6 +44,10 @@ public class CachedVectorContainer extends LoopedAbstractDrillSerializable {
     va.clear();
   }
 
+  public CachedVectorContainer(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
   public CachedVectorContainer(byte[] data, BufferAllocator allocator) {
     this.data = data;
     this.allocator = allocator;
@@ -58,6 +64,20 @@ public class CachedVectorContainer extends LoopedAbstractDrillSerializable {
 
   }
 
+
+  @Override
+  public void read(DataInput input) throws IOException {
+    int len = input.readInt();
+    this.data = new byte[len];
+    input.readFully(data);
+  }
+
+  @Override
+  public void write(DataOutput output) throws IOException {
+    output.writeInt(data.length);
+    output.write(data);
+  }
+
   public VectorAccessible get() {
     if (container == null) {
       construct();
@@ -66,7 +86,7 @@ public class CachedVectorContainer extends LoopedAbstractDrillSerializable {
   }
 
   public void clear() {
-    container.clear();
+    if(container != null) container.clear();
     container = null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
index 942e09e..1b44c6b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/local/LocalCache.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.cache.local;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -31,7 +32,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.DataInputInputStream;
 import org.apache.drill.common.util.DataOutputOutputStream;
 import org.apache.drill.exec.cache.Counter;
 import org.apache.drill.exec.cache.DistributedCache;
@@ -171,8 +171,7 @@ public class LocalCache implements DistributedCache {
       }
     }
 
-    ByteArrayDataInput in = ByteStreams.newDataInput(bytes);
-    InputStream inputStream = DataInputInputStream.constructInputStream(in);
+    InputStream inputStream = new ByteArrayInputStream(bytes);
     try {
       V obj = clazz.getConstructor(BufferAllocator.class).newInstance(allocator);
       obj.readFromStream(inputStream);
@@ -220,7 +219,7 @@ public class LocalCache implements DistributedCache {
       if (m.get(key) == null) return null;
       ByteArrayDataOutput b = m.get(key);
       byte[] bytes = b.toByteArray();
-      return (V) deserialize(m.get(key).toByteArray(), this.clazz);
+      return (V) deserialize(bytes, this.clazz);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 3b87dc4..92097e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -188,7 +188,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
    * Closes this client's connection to the server
    */
   public void close(){
-    this.client.close();
+    if(this.client != null) this.client.close();
     if(ownsZkConnection){
       try {
         this.clusterCoordinator.close();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index 731ab6b..ba846b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -383,7 +383,8 @@ public class EvaluationVisitor {
         PathSegment seg = e.getReadPath();
         int listNum = 0;
         boolean lastWasArray = false;
-        while(true){
+
+        while(seg != null){
           if(seg.isArray()){
             lastWasArray = true;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index 46dea64..121cfec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -130,77 +130,83 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
 
   @Override
   public IterOutcome next() {
+    stats.startProcessing();
+
+    try{
+      // we do this in the here instead of the constructor because don't necessary want to start consuming on construction.
+      status.ensureInitial();
+
+      // loop so we can start over again if we find a new batch was created.
+      while(true){
+
+        JoinOutcome outcome = status.getOutcome();
+        // if the previous outcome was a change in schema or we sent a batch, we have to set up a new batch.
+        if (outcome == JoinOutcome.BATCH_RETURNED ||
+            outcome == JoinOutcome.SCHEMA_CHANGED)
+          allocateBatch();
+
+        // reset the output position to zero after our parent iterates this RecordBatch
+        if (outcome == JoinOutcome.BATCH_RETURNED ||
+            outcome == JoinOutcome.SCHEMA_CHANGED ||
+            outcome == JoinOutcome.NO_MORE_DATA)
+          status.resetOutputPos();
+
+        if (outcome == JoinOutcome.NO_MORE_DATA) {
+          logger.debug("NO MORE DATA; returning {}  NONE");
+          return IterOutcome.NONE;
+        }
 
-    // we do this in the here instead of the constructor because don't necessary want to start consuming on construction.
-    status.ensureInitial();
-
-    // loop so we can start over again if we find a new batch was created.
-    while(true){
-
-      JoinOutcome outcome = status.getOutcome();
-      // if the previous outcome was a change in schema or we sent a batch, we have to set up a new batch.
-      if (outcome == JoinOutcome.BATCH_RETURNED ||
-          outcome == JoinOutcome.SCHEMA_CHANGED)
-        allocateBatch();
-
-      // reset the output position to zero after our parent iterates this RecordBatch
-      if (outcome == JoinOutcome.BATCH_RETURNED ||
-          outcome == JoinOutcome.SCHEMA_CHANGED ||
-          outcome == JoinOutcome.NO_MORE_DATA)
-        status.resetOutputPos();
+        boolean first = false;
+        if(worker == null){
+          try {
+            logger.debug("Creating New Worker");
+            stats.startSetup();
+            this.worker = generateNewWorker();
+            first = true;
+            stats.stopSetup();
+          } catch (ClassTransformationException | IOException | SchemaChangeException e) {
+            stats.stopSetup();
+            context.fail(new SchemaChangeException(e));
+            kill();
+            return IterOutcome.STOP;
+          }
+        }
 
-      if (outcome == JoinOutcome.NO_MORE_DATA) {
-        logger.debug("NO MORE DATA; returning {}  NONE");
-        return IterOutcome.NONE;
-      }
+        // join until we have a complete outgoing batch
+        if (!worker.doJoin(status))
+          worker = null;
 
-      boolean first = false;
-      if(worker == null){
-        try {
-          logger.debug("Creating New Worker");
-          stats.startSetup();
-          this.worker = generateNewWorker();
-          first = true;
-          stats.stopSetup();
-        } catch (ClassTransformationException | IOException | SchemaChangeException e) {
-          stats.stopSetup();
-          context.fail(new SchemaChangeException(e));
+        // get the outcome of the join.
+        switch(status.getOutcome()){
+        case BATCH_RETURNED:
+          // only return new schema if new worker has been setup.
+          logger.debug("BATCH RETURNED; returning {}", (first ? "OK_NEW_SCHEMA" : "OK"));
+          return first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK;
+        case FAILURE:
           kill();
           return IterOutcome.STOP;
+        case NO_MORE_DATA:
+          logger.debug("NO MORE DATA; returning {}", (status.getOutPosition() > 0 ? (first ? "OK_NEW_SCHEMA" : "OK") : "NONE"));
+          return status.getOutPosition() > 0 ? (first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK): IterOutcome.NONE;
+        case SCHEMA_CHANGED:
+          worker = null;
+          if(status.getOutPosition() > 0){
+            // if we have current data, let's return that.
+            logger.debug("SCHEMA CHANGED; returning {} ", (first ? "OK_NEW_SCHEMA" : "OK"));
+            return first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK;
+          }else{
+            // loop again to rebuild worker.
+            continue;
+          }
+        case WAITING:
+          return IterOutcome.NOT_YET;
+        default:
+          throw new IllegalStateException();
         }
       }
 
-      // join until we have a complete outgoing batch
-      if (!worker.doJoin(status))
-        worker = null;
-
-      // get the outcome of the join.
-      switch(status.getOutcome()){
-      case BATCH_RETURNED:
-        // only return new schema if new worker has been setup.
-        logger.debug("BATCH RETURNED; returning {}", (first ? "OK_NEW_SCHEMA" : "OK"));
-        return first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK;
-      case FAILURE:
-        kill();
-        return IterOutcome.STOP;
-      case NO_MORE_DATA:
-        logger.debug("NO MORE DATA; returning {}", (status.getOutPosition() > 0 ? (first ? "OK_NEW_SCHEMA" : "OK") : "NONE"));
-        return status.getOutPosition() > 0 ? (first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK): IterOutcome.NONE;
-      case SCHEMA_CHANGED:
-        worker = null;
-        if(status.getOutPosition() > 0){
-          // if we have current data, let's return that.
-          logger.debug("SCHEMA CHANGED; returning {} ", (first ? "OK_NEW_SCHEMA" : "OK"));
-          return first ? IterOutcome.OK_NEW_SCHEMA : IterOutcome.OK;
-        }else{
-          // loop again to rebuild worker.
-          continue;
-        }
-      case WAITING:
-        return IterOutcome.NOT_YET;
-      default:
-        throw new IllegalStateException();
-      }
+    }finally{
+      stats.stopProcessing();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index fe19797..96d3242 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -180,10 +180,15 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
     boolean isAnyWildcard = isAnyWildcard(exprs);
 
     if(isAnyWildcard){
+
+      // add this until we have sv2 project on wildcard working correctly.
+      if(incoming.getSchema().getSelectionVectorMode() != SelectionVectorMode.NONE){
+        throw new UnsupportedOperationException("Drill doesn't yet wildcard projects where there is a sv2, patch coming shortly.");
+      }
       for(VectorWrapper<?> wrapper : incoming){
         ValueVector vvIn = wrapper.getValueVector();
 
-        String name = vvIn.getField().getPath().getLastSegment().getNameSegment().getPath();
+        String name = vvIn.getField().getPath().getRootSegment().getPath();
         FieldReference ref = new FieldReference(name);
         TransferPair tp = wrapper.getValueVector().getTransferPair(ref);
         transfers.add(tp);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
index 357cb2e..03431d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
@@ -17,18 +17,33 @@
  */
 package org.apache.drill.exec.planner.common;
 
+import java.util.List;
+
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.Convention;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.sql.type.SqlTypeName;
+
+import com.google.common.collect.ImmutableList;
+import com.google.hive12.common.collect.Lists;
 
 /** Base class for logical and physical Writer implemented in Drill. */
 public abstract class DrillWriterRelBase extends SingleRel implements DrillRelNode {
 
+  private static final List<String> FIELD_NAMES = ImmutableList.of("Fragment", "Number of records written");
   private final CreateTableEntry createTableEntry;
 
+  protected void setRowType(){
+    List<RelDataType> fields = Lists.newArrayList();
+    fields.add(this.getCluster().getTypeFactory().createSqlType(SqlTypeName.VARCHAR, 255));
+    fields.add(this.getCluster().getTypeFactory().createSqlType(SqlTypeName.BIGINT));
+    this.rowType = this.getCluster().getTypeFactory().createStructType(fields, FIELD_NAMES);
+  }
+
   public DrillWriterRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
       CreateTableEntry createTableEntry) {
     super(cluster, traitSet, input);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWriterRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWriterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWriterRel.java
index f212026..04dd133 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWriterRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillWriterRel.java
@@ -30,6 +30,7 @@ public class DrillWriterRel extends DrillWriterRelBase implements DrillRel {
 
   public DrillWriterRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, CreateTableEntry createTableEntry) {
     super(DRILL_LOGICAL, cluster, traitSet, input, createTableEntry);
+    setRowType();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
index a7f611c..233b20b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
@@ -37,14 +37,11 @@ import com.google.hive12.common.collect.Lists;
 
 public class WriterPrel extends DrillWriterRelBase implements Prel {
 
-  private static final List<String> FIELD_NAMES = ImmutableList.of("Fragment", "Number of records written");
+
 
   public WriterPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, CreateTableEntry createTableEntry) {
     super(Prel.DRILL_PHYSICAL, cluster, traits, child, createTableEntry);
-    List<RelDataType> fields = Lists.newArrayList();
-    fields.add(cluster.getTypeFactory().createSqlType(SqlTypeName.VARCHAR, 255));
-    fields.add(cluster.getTypeFactory().createSqlType(SqlTypeName.BIGINT));
-    this.rowType = cluster.getTypeFactory().createStructType(fields, FIELD_NAMES);
+    setRowType();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
index 6ed3c1f..4ea82cd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/visitor/FinalColumnReorderer.java
@@ -48,9 +48,14 @@ public class FinalColumnReorderer extends BasePrelVisitor<Prel, Void, RuntimeExc
 
   private Prel addTrivialOrderedProjectPrel(Prel prel){
     RelDataType t = prel.getRowType();
+
     RexBuilder b = prel.getCluster().getRexBuilder();
     List<RexNode> projections = Lists.newArrayList();
     int projectCount = t.getFieldList().size();
+
+    // no point in reordering if we only have one column
+    if(projectCount < 2) return prel;
+
     for(int i =0; i < projectCount; i++){
       projections.add(b.makeInputRef(prel, i));
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index df66dcf..eb2c891 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -100,7 +100,7 @@ public class DrillSqlWorker {
   }
 
   public PhysicalPlan getPlan(String sql) throws SqlParseException, ValidationException, RelConversionException, IOException{
-    return getPlan(null);
+    return getPlan(sql, null);
   }
 
   public PhysicalPlan getPlan(String sql, Pointer<String> textPlan) throws SqlParseException, ValidationException, RelConversionException, IOException{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index a3307cf..a912778 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -142,15 +142,16 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
     return null;
   }
 
-  public int bind(final int initialPort) throws InterruptedException, DrillbitStartupException {
+  public int bind(final int initialPort, boolean allowPortHunting) throws InterruptedException, DrillbitStartupException {
     int port = initialPort - 1;
     while (true) {
       try {
         b.bind(++port).sync();
         break;
       } catch (Exception e) {
-        if (e instanceof BindException)
+        if (e instanceof BindException && allowPortHunting){
           continue;
+        }
         throw new DrillbitStartupException("Could not bind Drillbit", e);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
index de8caf6..1cacc4f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java
@@ -39,19 +39,21 @@ public class ControllerImpl implements Controller {
   private final ControlMessageHandler handler;
   private final BootStrapContext context;
   private final ConnectionManagerRegistry connectionRegistry;
+  private final boolean allowPortHunting;
 
-  public ControllerImpl(BootStrapContext context, ControlMessageHandler handler) {
+  public ControllerImpl(BootStrapContext context, ControlMessageHandler handler, boolean allowPortHunting) {
     super();
     this.handler = handler;
     this.context = context;
     this.connectionRegistry = new ConnectionManagerRegistry(handler, context);
+    this.allowPortHunting = allowPortHunting;
   }
 
   @Override
   public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException {
     server = new ControlServer(handler, context, connectionRegistry);
     int port = context.getConfig().getInt(ExecConstants.INITIAL_BIT_PORT);
-    port = server.bind(port);
+    port = server.bind(port, allowPortHunting);
     DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setControlPort(port).build();
     connectionRegistry.setEndpoint(completeEndpoint);
     return completeEndpoint;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
index f15494f..9c2ef5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
@@ -37,17 +37,19 @@ public class DataConnectionCreator implements Closeable {
   private final BootStrapContext context;
   private final WorkEventBus workBus;
   private final DataResponseHandler dataHandler;
+  private final boolean allowPortHunting;
 
-  public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler) {
+  public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting) {
     super();
     this.context = context;
     this.workBus = workBus;
     this.dataHandler = dataHandler;
+    this.allowPortHunting = allowPortHunting;
   }
 
   public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws InterruptedException, DrillbitStartupException {
     server = new DataServer(context, workBus, dataHandler);
-    int port = server.bind(partialEndpoint.getControlPort() + 1);
+    int port = server.bind(partialEndpoint.getControlPort() + 1, allowPortHunting);
     DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setDataPort(port).build();
     return completeEndpoint;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index eba6e92..fb499b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -81,11 +81,18 @@ public class Drillbit implements Closeable{
   private volatile RegistrationHandle handle;
 
   public Drillbit(DrillConfig config, RemoteServiceSet serviceSet) throws Exception {
-
+    boolean allowPortHunting = serviceSet != null;
+    boolean enableHttp = config.getBoolean(ExecConstants.HTTP_ENABLE);
     this.context = new BootStrapContext(config);
     this.manager = new WorkManager(context);
-    this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler());
-    this.embeddedJetty = new Server(8047);
+    this.engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context, manager.getWorkBus(), manager.getDataHandler(), allowPortHunting);
+
+    if(enableHttp){
+      this.embeddedJetty = new Server(config.getInt(ExecConstants.HTTP_PORT));
+    }else{
+      this.embeddedJetty = null;
+    }
+
 
     if(serviceSet != null){
       this.coord = serviceSet.getCoordinator();
@@ -99,6 +106,8 @@ public class Drillbit implements Closeable{
   }
 
   private void startJetty() throws Exception{
+    if(embeddedJetty == null) return;
+
     ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS);
     context.setContextPath("/");
     embeddedJetty.setHandler(context);
@@ -131,7 +140,7 @@ public class Drillbit implements Closeable{
       logger.warn("Interrupted while sleeping during coordination deregistration.");
     }
     try {
-      embeddedJetty.stop();
+      if(embeddedJetty != null) embeddedJetty.stop();
     } catch (Exception e) {
       logger.warn("Failure while shutting down embedded jetty server.");
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index cfbde73..bd745d1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -43,23 +43,25 @@ import com.google.common.io.Closeables;
 
 public class ServiceEngine implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ServiceEngine.class);
-  
+
   private final UserServer userServer;
   private final Controller controller;
   private final DataConnectionCreator dataPool;
   private final DrillConfig config;
   boolean useIP = false;
-  
-  public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler){
+  private final boolean allowPortHunting;
+
+  public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting){
     this.userServer = new UserServer(context.getAllocator(), new NioEventLoopGroup(context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS),
             new NamedThreadFactory("UserServer-")), userWorker);
-    this.controller = new ControllerImpl(context, controlMessageHandler);
-    this.dataPool = new DataConnectionCreator(context, workBus, dataHandler);
+    this.controller = new ControllerImpl(context, controlMessageHandler, allowPortHunting);
+    this.dataPool = new DataConnectionCreator(context, workBus, dataHandler, allowPortHunting);
     this.config = context.getConfig();
+    this.allowPortHunting = allowPortHunting;
   }
-  
+
   public DrillbitEndpoint start() throws DrillbitStartupException, InterruptedException, UnknownHostException{
-    int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT));
+    int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT), allowPortHunting);
     String address = useIP ?  InetAddress.getLocalHost().getHostAddress() : InetAddress.getLocalHost().getCanonicalHostName();
     DrillbitEndpoint partialEndpoint = DrillbitEndpoint.newBuilder()
         .setAddress(address)
@@ -74,7 +76,7 @@ public class ServiceEngine implements Closeable{
   public DataConnectionCreator getDataConnectionCreator(){
     return dataPool;
   }
-  
+
   public Controller getController() {
     return controller;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
index 9a745ac..b0133f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/SystemTableScan.java
@@ -32,6 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.physical.base.SubScan;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 
 import parquet.org.codehaus.jackson.annotate.JsonCreator;
@@ -112,6 +113,11 @@ public class SystemTableScan extends AbstractGroupScan implements SubScan{
 
 
   @Override
+  public int getOperatorType() {
+    return CoreOperatorType.SYSTEM_TABLE_SCAN_VALUE;
+  }
+
+  @Override
   public GroupScan clone(List<SchemaPath> columns) {
     return this;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/drill-module.conf b/exec/java-exec/src/main/resources/drill-module.conf
index 9a180fd..26205bd 100644
--- a/exec/java-exec/src/main/resources/drill-module.conf
+++ b/exec/java-exec/src/main/resources/drill-module.conf
@@ -65,6 +65,10 @@ drill.exec: {
   	  delay: 500
   	}    
   },
+  http: {
+    enabled: true,
+    port: 8047
+  },
   functions: ["org.apache.drill.expr.fn.impl"],
   network: {
     start: 35000

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
index 74f5ba9..f579448 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -74,7 +74,7 @@ public class TestBitRpc extends ExecTest {
     DataResponseHandler drp = new BitComTestHandler();
     DataServer server = new DataServer(c, workBus, drp);
 
-    port = server.bind(port);
+    port = server.bind(port, false);
     DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
     DataConnectionManager manager = new DataConnectionManager(FragmentHandle.getDefaultInstance(), ep, c2);
     DataTunnel tunnel = new DataTunnel(manager);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JsonRecordReader2Test.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JsonRecordReader2Test.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JsonRecordReader2Test.java
index 0abdbd3..34bcb5e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JsonRecordReader2Test.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JsonRecordReader2Test.java
@@ -35,14 +35,14 @@ public class JsonRecordReader2Test extends BaseTestQuery{
   }
 
   @Test
-  public void z() throws Exception{
+  public void testComplexMultipleTimes() throws Exception{
     for(int i =0 ; i < 5; i++){
     test("select * from cp.`join/merge_join.json`");
     }
   }
 
   @Test
-  public void y() throws Exception{
+  public void trySimpleQueryWithLimit() throws Exception{
     test("select * from cp.`limit/test1.json` limit 10");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/logback.xml b/exec/java-exec/src/test/resources/logback.xml
index cd3d971..b8e0ca2 100644
--- a/exec/java-exec/src/test/resources/logback.xml
+++ b/exec/java-exec/src/test/resources/logback.xml
@@ -42,7 +42,7 @@
 <!--   </logger> -->
 
 	<root>
-		<level value="info" />
+		<level value="error" />
 		<appender-ref ref="STDOUT" />
 	</root>
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/79054a85/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d4c077a..5d976e8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -260,7 +260,7 @@
           <artifactId>maven-surefire-plugin</artifactId>
           <version>2.17</version>
           <configuration>
-            <argLine>-Xms512m -Xmx1g -XX:MaxDirectMemorySize=2096M -XX:+CMSClassUnloadingEnabled</argLine>
+            <argLine>-Xms512m -Xmx1g -Ddrill.exec.http.enabled=false -XX:MaxPermSize=256M -XX:MaxDirectMemorySize=2096M -XX:+CMSClassUnloadingEnabled</argLine>
             <forkCount>4</forkCount>
             <reuseForks>true</reuseForks>
             <additionalClasspathElements>


[15/24] diag wip

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
index 9e041dc..7491617 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -1578,117 +1578,33 @@ public final class BitControl {
   public interface FragmentStatusOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
-    // optional int64 memory_use = 1;
+    // optional .exec.shared.MinorFragmentProfile profile = 1;
     /**
-     * <code>optional int64 memory_use = 1;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    boolean hasMemoryUse();
-    /**
-     * <code>optional int64 memory_use = 1;</code>
-     */
-    long getMemoryUse();
-
-    // optional int64 batches_completed = 2;
-    /**
-     * <code>optional int64 batches_completed = 2;</code>
-     */
-    boolean hasBatchesCompleted();
-    /**
-     * <code>optional int64 batches_completed = 2;</code>
-     */
-    long getBatchesCompleted();
-
-    // optional int64 records_completed = 3;
-    /**
-     * <code>optional int64 records_completed = 3;</code>
-     */
-    boolean hasRecordsCompleted();
-    /**
-     * <code>optional int64 records_completed = 3;</code>
-     */
-    long getRecordsCompleted();
-
-    // optional int32 estimated_completion_percentage = 4;
-    /**
-     * <code>optional int32 estimated_completion_percentage = 4;</code>
-     */
-    boolean hasEstimatedCompletionPercentage();
-    /**
-     * <code>optional int32 estimated_completion_percentage = 4;</code>
-     */
-    int getEstimatedCompletionPercentage();
-
-    // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
-    /**
-     * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-     */
-    boolean hasState();
-    /**
-     * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-     */
-    org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState();
-
-    // optional int64 data_processed = 6;
+    boolean hasProfile();
     /**
-     * <code>optional int64 data_processed = 6;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    boolean hasDataProcessed();
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile();
     /**
-     * <code>optional int64 data_processed = 6;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    long getDataProcessed();
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder();
 
-    // optional .exec.bit.FragmentHandle handle = 7;
+    // optional .exec.bit.FragmentHandle handle = 2;
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     boolean hasHandle();
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle();
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder();
-
-    // optional .exec.shared.DrillPBError error = 8;
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    boolean hasError();
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.DrillPBError getError();
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder();
-
-    // optional int64 running_time = 9;
-    /**
-     * <code>optional int64 running_time = 9;</code>
-     */
-    boolean hasRunningTime();
-    /**
-     * <code>optional int64 running_time = 9;</code>
-     */
-    long getRunningTime();
-
-    // optional .exec.shared.MinorFragmentProfile profile = 10;
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    boolean hasProfile();
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile();
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder();
   }
   /**
    * Protobuf type {@code exec.bit.control.FragmentStatus}
@@ -1741,45 +1657,22 @@ public final class BitControl {
               }
               break;
             }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              memoryUse_ = input.readInt64();
-              break;
-            }
-            case 16: {
-              bitField0_ |= 0x00000002;
-              batchesCompleted_ = input.readInt64();
-              break;
-            }
-            case 24: {
-              bitField0_ |= 0x00000004;
-              recordsCompleted_ = input.readInt64();
-              break;
-            }
-            case 32: {
-              bitField0_ |= 0x00000008;
-              estimatedCompletionPercentage_ = input.readInt32();
-              break;
-            }
-            case 40: {
-              int rawValue = input.readEnum();
-              org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState value = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.valueOf(rawValue);
-              if (value == null) {
-                unknownFields.mergeVarintField(5, rawValue);
-              } else {
-                bitField0_ |= 0x00000010;
-                state_ = value;
+            case 10: {
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = profile_.toBuilder();
               }
+              profile_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(profile_);
+                profile_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
               break;
             }
-            case 48: {
-              bitField0_ |= 0x00000020;
-              dataProcessed_ = input.readInt64();
-              break;
-            }
-            case 58: {
+            case 18: {
               org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000040) == 0x00000040)) {
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
                 subBuilder = handle_.toBuilder();
               }
               handle_ = input.readMessage(org.apache.drill.exec.proto.ExecProtos.FragmentHandle.PARSER, extensionRegistry);
@@ -1787,38 +1680,7 @@ public final class BitControl {
                 subBuilder.mergeFrom(handle_);
                 handle_ = subBuilder.buildPartial();
               }
-              bitField0_ |= 0x00000040;
-              break;
-            }
-            case 66: {
-              org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000080) == 0x00000080)) {
-                subBuilder = error_.toBuilder();
-              }
-              error_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.DrillPBError.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(error_);
-                error_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000080;
-              break;
-            }
-            case 72: {
-              bitField0_ |= 0x00000100;
-              runningTime_ = input.readInt64();
-              break;
-            }
-            case 82: {
-              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000200) == 0x00000200)) {
-                subBuilder = profile_.toBuilder();
-              }
-              profile_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(profile_);
-                profile_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000200;
+              bitField0_ |= 0x00000002;
               break;
             }
           }
@@ -1860,314 +1722,54 @@ public final class BitControl {
       return PARSER;
     }
 
-    /**
-     * Protobuf enum {@code exec.bit.control.FragmentStatus.FragmentState}
-     */
-    public enum FragmentState
-        implements com.google.protobuf.ProtocolMessageEnum {
-      /**
-       * <code>SENDING = 0;</code>
-       */
-      SENDING(0, 0),
-      /**
-       * <code>AWAITING_ALLOCATION = 1;</code>
-       */
-      AWAITING_ALLOCATION(1, 1),
-      /**
-       * <code>RUNNING = 2;</code>
-       */
-      RUNNING(2, 2),
-      /**
-       * <code>FINISHED = 3;</code>
-       */
-      FINISHED(3, 3),
-      /**
-       * <code>CANCELLED = 4;</code>
-       */
-      CANCELLED(4, 4),
-      /**
-       * <code>FAILED = 5;</code>
-       */
-      FAILED(5, 5),
-      ;
-
-      /**
-       * <code>SENDING = 0;</code>
-       */
-      public static final int SENDING_VALUE = 0;
-      /**
-       * <code>AWAITING_ALLOCATION = 1;</code>
-       */
-      public static final int AWAITING_ALLOCATION_VALUE = 1;
-      /**
-       * <code>RUNNING = 2;</code>
-       */
-      public static final int RUNNING_VALUE = 2;
-      /**
-       * <code>FINISHED = 3;</code>
-       */
-      public static final int FINISHED_VALUE = 3;
-      /**
-       * <code>CANCELLED = 4;</code>
-       */
-      public static final int CANCELLED_VALUE = 4;
-      /**
-       * <code>FAILED = 5;</code>
-       */
-      public static final int FAILED_VALUE = 5;
-
-
-      public final int getNumber() { return value; }
-
-      public static FragmentState valueOf(int value) {
-        switch (value) {
-          case 0: return SENDING;
-          case 1: return AWAITING_ALLOCATION;
-          case 2: return RUNNING;
-          case 3: return FINISHED;
-          case 4: return CANCELLED;
-          case 5: return FAILED;
-          default: return null;
-        }
-      }
-
-      public static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
-          internalGetValueMap() {
-        return internalValueMap;
-      }
-      private static com.google.protobuf.Internal.EnumLiteMap<FragmentState>
-          internalValueMap =
-            new com.google.protobuf.Internal.EnumLiteMap<FragmentState>() {
-              public FragmentState findValueByNumber(int number) {
-                return FragmentState.valueOf(number);
-              }
-            };
-
-      public final com.google.protobuf.Descriptors.EnumValueDescriptor
-          getValueDescriptor() {
-        return getDescriptor().getValues().get(index);
-      }
-      public final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptorForType() {
-        return getDescriptor();
-      }
-      public static final com.google.protobuf.Descriptors.EnumDescriptor
-          getDescriptor() {
-        return org.apache.drill.exec.proto.BitControl.FragmentStatus.getDescriptor().getEnumTypes().get(0);
-      }
-
-      private static final FragmentState[] VALUES = values();
-
-      public static FragmentState valueOf(
-          com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
-        if (desc.getType() != getDescriptor()) {
-          throw new java.lang.IllegalArgumentException(
-            "EnumValueDescriptor is not for this type.");
-        }
-        return VALUES[desc.getIndex()];
-      }
-
-      private final int index;
-      private final int value;
-
-      private FragmentState(int index, int value) {
-        this.index = index;
-        this.value = value;
-      }
-
-      // @@protoc_insertion_point(enum_scope:exec.bit.control.FragmentStatus.FragmentState)
-    }
-
     private int bitField0_;
-    // optional int64 memory_use = 1;
-    public static final int MEMORY_USE_FIELD_NUMBER = 1;
-    private long memoryUse_;
+    // optional .exec.shared.MinorFragmentProfile profile = 1;
+    public static final int PROFILE_FIELD_NUMBER = 1;
+    private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_;
     /**
-     * <code>optional int64 memory_use = 1;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    public boolean hasMemoryUse() {
+    public boolean hasProfile() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
-     * <code>optional int64 memory_use = 1;</code>
-     */
-    public long getMemoryUse() {
-      return memoryUse_;
-    }
-
-    // optional int64 batches_completed = 2;
-    public static final int BATCHES_COMPLETED_FIELD_NUMBER = 2;
-    private long batchesCompleted_;
-    /**
-     * <code>optional int64 batches_completed = 2;</code>
-     */
-    public boolean hasBatchesCompleted() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional int64 batches_completed = 2;</code>
-     */
-    public long getBatchesCompleted() {
-      return batchesCompleted_;
-    }
-
-    // optional int64 records_completed = 3;
-    public static final int RECORDS_COMPLETED_FIELD_NUMBER = 3;
-    private long recordsCompleted_;
-    /**
-     * <code>optional int64 records_completed = 3;</code>
-     */
-    public boolean hasRecordsCompleted() {
-      return ((bitField0_ & 0x00000004) == 0x00000004);
-    }
-    /**
-     * <code>optional int64 records_completed = 3;</code>
-     */
-    public long getRecordsCompleted() {
-      return recordsCompleted_;
-    }
-
-    // optional int32 estimated_completion_percentage = 4;
-    public static final int ESTIMATED_COMPLETION_PERCENTAGE_FIELD_NUMBER = 4;
-    private int estimatedCompletionPercentage_;
-    /**
-     * <code>optional int32 estimated_completion_percentage = 4;</code>
-     */
-    public boolean hasEstimatedCompletionPercentage() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional int32 estimated_completion_percentage = 4;</code>
-     */
-    public int getEstimatedCompletionPercentage() {
-      return estimatedCompletionPercentage_;
-    }
-
-    // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
-    public static final int STATE_FIELD_NUMBER = 5;
-    private org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState state_;
-    /**
-     * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-     */
-    public boolean hasState() {
-      return ((bitField0_ & 0x00000010) == 0x00000010);
-    }
-    /**
-     * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-     */
-    public org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState() {
-      return state_;
-    }
-
-    // optional int64 data_processed = 6;
-    public static final int DATA_PROCESSED_FIELD_NUMBER = 6;
-    private long dataProcessed_;
-    /**
-     * <code>optional int64 data_processed = 6;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    public boolean hasDataProcessed() {
-      return ((bitField0_ & 0x00000020) == 0x00000020);
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
+      return profile_;
     }
     /**
-     * <code>optional int64 data_processed = 6;</code>
+     * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
      */
-    public long getDataProcessed() {
-      return dataProcessed_;
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
+      return profile_;
     }
 
-    // optional .exec.bit.FragmentHandle handle = 7;
-    public static final int HANDLE_FIELD_NUMBER = 7;
+    // optional .exec.bit.FragmentHandle handle = 2;
+    public static final int HANDLE_FIELD_NUMBER = 2;
     private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_;
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     public boolean hasHandle() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
+      return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
       return handle_;
     }
     /**
-     * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+     * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
      */
     public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
       return handle_;
     }
 
-    // optional .exec.shared.DrillPBError error = 8;
-    public static final int ERROR_FIELD_NUMBER = 8;
-    private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_;
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    public boolean hasError() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
-      return error_;
-    }
-    /**
-     * <code>optional .exec.shared.DrillPBError error = 8;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
-      return error_;
-    }
-
-    // optional int64 running_time = 9;
-    public static final int RUNNING_TIME_FIELD_NUMBER = 9;
-    private long runningTime_;
-    /**
-     * <code>optional int64 running_time = 9;</code>
-     */
-    public boolean hasRunningTime() {
-      return ((bitField0_ & 0x00000100) == 0x00000100);
-    }
-    /**
-     * <code>optional int64 running_time = 9;</code>
-     */
-    public long getRunningTime() {
-      return runningTime_;
-    }
-
-    // optional .exec.shared.MinorFragmentProfile profile = 10;
-    public static final int PROFILE_FIELD_NUMBER = 10;
-    private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_;
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    public boolean hasProfile() {
-      return ((bitField0_ & 0x00000200) == 0x00000200);
-    }
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
-      return profile_;
-    }
-    /**
-     * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-     */
-    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
-      return profile_;
-    }
-
     private void initFields() {
-      memoryUse_ = 0L;
-      batchesCompleted_ = 0L;
-      recordsCompleted_ = 0L;
-      estimatedCompletionPercentage_ = 0;
-      state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
-      dataProcessed_ = 0L;
-      handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
-      error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
-      runningTime_ = 0L;
       profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+      handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2182,34 +1784,10 @@ public final class BitControl {
                         throws java.io.IOException {
       getSerializedSize();
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeInt64(1, memoryUse_);
+        output.writeMessage(1, profile_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeInt64(2, batchesCompleted_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        output.writeInt64(3, recordsCompleted_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeInt32(4, estimatedCompletionPercentage_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeEnum(5, state_.getNumber());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        output.writeInt64(6, dataProcessed_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeMessage(7, handle_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeMessage(8, error_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        output.writeInt64(9, runningTime_);
-      }
-      if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        output.writeMessage(10, profile_);
+        output.writeMessage(2, handle_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -2222,43 +1800,11 @@ public final class BitControl {
       size = 0;
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(1, memoryUse_);
+          .computeMessageSize(1, profile_);
       }
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(2, batchesCompleted_);
-      }
-      if (((bitField0_ & 0x00000004) == 0x00000004)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(3, recordsCompleted_);
-      }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(4, estimatedCompletionPercentage_);
-      }
-      if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeEnumSize(5, state_.getNumber());
-      }
-      if (((bitField0_ & 0x00000020) == 0x00000020)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(6, dataProcessed_);
-      }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(7, handle_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(8, error_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(9, runningTime_);
-      }
-      if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(10, profile_);
+          .computeMessageSize(2, handle_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -2368,9 +1914,8 @@ public final class BitControl {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getHandleFieldBuilder();
-          getErrorFieldBuilder();
           getProfileFieldBuilder();
+          getHandleFieldBuilder();
         }
       }
       private static Builder create() {
@@ -2379,38 +1924,18 @@ public final class BitControl {
 
       public Builder clear() {
         super.clear();
-        memoryUse_ = 0L;
+        if (profileBuilder_ == null) {
+          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+        } else {
+          profileBuilder_.clear();
+        }
         bitField0_ = (bitField0_ & ~0x00000001);
-        batchesCompleted_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        recordsCompleted_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000004);
-        estimatedCompletionPercentage_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000008);
-        state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
-        bitField0_ = (bitField0_ & ~0x00000010);
-        dataProcessed_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000020);
         if (handleBuilder_ == null) {
           handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
         } else {
           handleBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
-        if (errorBuilder_ == null) {
-          error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
-        } else {
-          errorBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000080);
-        runningTime_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000100);
-        if (profileBuilder_ == null) {
-          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
-        } else {
-          profileBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000200);
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
 
@@ -2442,55 +1967,19 @@ public final class BitControl {
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
         }
-        result.memoryUse_ = memoryUse_;
+        if (profileBuilder_ == null) {
+          result.profile_ = profile_;
+        } else {
+          result.profile_ = profileBuilder_.build();
+        }
         if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
           to_bitField0_ |= 0x00000002;
         }
-        result.batchesCompleted_ = batchesCompleted_;
-        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
-          to_bitField0_ |= 0x00000004;
-        }
-        result.recordsCompleted_ = recordsCompleted_;
-        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.estimatedCompletionPercentage_ = estimatedCompletionPercentage_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000010;
-        }
-        result.state_ = state_;
-        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
-          to_bitField0_ |= 0x00000020;
-        }
-        result.dataProcessed_ = dataProcessed_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000040;
-        }
         if (handleBuilder_ == null) {
           result.handle_ = handle_;
         } else {
           result.handle_ = handleBuilder_.build();
         }
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        if (errorBuilder_ == null) {
-          result.error_ = error_;
-        } else {
-          result.error_ = errorBuilder_.build();
-        }
-        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
-          to_bitField0_ |= 0x00000100;
-        }
-        result.runningTime_ = runningTime_;
-        if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
-          to_bitField0_ |= 0x00000200;
-        }
-        if (profileBuilder_ == null) {
-          result.profile_ = profile_;
-        } else {
-          result.profile_ = profileBuilder_.build();
-        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2507,36 +1996,12 @@ public final class BitControl {
 
       public Builder mergeFrom(org.apache.drill.exec.proto.BitControl.FragmentStatus other) {
         if (other == org.apache.drill.exec.proto.BitControl.FragmentStatus.getDefaultInstance()) return this;
-        if (other.hasMemoryUse()) {
-          setMemoryUse(other.getMemoryUse());
-        }
-        if (other.hasBatchesCompleted()) {
-          setBatchesCompleted(other.getBatchesCompleted());
-        }
-        if (other.hasRecordsCompleted()) {
-          setRecordsCompleted(other.getRecordsCompleted());
-        }
-        if (other.hasEstimatedCompletionPercentage()) {
-          setEstimatedCompletionPercentage(other.getEstimatedCompletionPercentage());
-        }
-        if (other.hasState()) {
-          setState(other.getState());
-        }
-        if (other.hasDataProcessed()) {
-          setDataProcessed(other.getDataProcessed());
+        if (other.hasProfile()) {
+          mergeProfile(other.getProfile());
         }
         if (other.hasHandle()) {
           mergeHandle(other.getHandle());
         }
-        if (other.hasError()) {
-          mergeError(other.getError());
-        }
-        if (other.hasRunningTime()) {
-          setRunningTime(other.getRunningTime());
-        }
-        if (other.hasProfile()) {
-          mergeProfile(other.getProfile());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -2564,219 +2029,135 @@ public final class BitControl {
       }
       private int bitField0_;
 
-      // optional int64 memory_use = 1;
-      private long memoryUse_ ;
+      // optional .exec.shared.MinorFragmentProfile profile = 1;
+      private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> profileBuilder_;
       /**
-       * <code>optional int64 memory_use = 1;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public boolean hasMemoryUse() {
+      public boolean hasProfile() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
-       * <code>optional int64 memory_use = 1;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public long getMemoryUse() {
-        return memoryUse_;
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
+        if (profileBuilder_ == null) {
+          return profile_;
+        } else {
+          return profileBuilder_.getMessage();
+        }
       }
       /**
-       * <code>optional int64 memory_use = 1;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public Builder setMemoryUse(long value) {
+      public Builder setProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (profileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          profile_ = value;
+          onChanged();
+        } else {
+          profileBuilder_.setMessage(value);
+        }
         bitField0_ |= 0x00000001;
-        memoryUse_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 memory_use = 1;</code>
-       */
-      public Builder clearMemoryUse() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        memoryUse_ = 0L;
-        onChanged();
         return this;
       }
-
-      // optional int64 batches_completed = 2;
-      private long batchesCompleted_ ;
-      /**
-       * <code>optional int64 batches_completed = 2;</code>
-       */
-      public boolean hasBatchesCompleted() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional int64 batches_completed = 2;</code>
-       */
-      public long getBatchesCompleted() {
-        return batchesCompleted_;
-      }
-      /**
-       * <code>optional int64 batches_completed = 2;</code>
-       */
-      public Builder setBatchesCompleted(long value) {
-        bitField0_ |= 0x00000002;
-        batchesCompleted_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 batches_completed = 2;</code>
-       */
-      public Builder clearBatchesCompleted() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        batchesCompleted_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int64 records_completed = 3;
-      private long recordsCompleted_ ;
       /**
-       * <code>optional int64 records_completed = 3;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public boolean hasRecordsCompleted() {
-        return ((bitField0_ & 0x00000004) == 0x00000004);
-      }
-      /**
-       * <code>optional int64 records_completed = 3;</code>
-       */
-      public long getRecordsCompleted() {
-        return recordsCompleted_;
-      }
-      /**
-       * <code>optional int64 records_completed = 3;</code>
-       */
-      public Builder setRecordsCompleted(long value) {
-        bitField0_ |= 0x00000004;
-        recordsCompleted_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 records_completed = 3;</code>
-       */
-      public Builder clearRecordsCompleted() {
-        bitField0_ = (bitField0_ & ~0x00000004);
-        recordsCompleted_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional int32 estimated_completion_percentage = 4;
-      private int estimatedCompletionPercentage_ ;
-      /**
-       * <code>optional int32 estimated_completion_percentage = 4;</code>
-       */
-      public boolean hasEstimatedCompletionPercentage() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int32 estimated_completion_percentage = 4;</code>
-       */
-      public int getEstimatedCompletionPercentage() {
-        return estimatedCompletionPercentage_;
-      }
-      /**
-       * <code>optional int32 estimated_completion_percentage = 4;</code>
-       */
-      public Builder setEstimatedCompletionPercentage(int value) {
-        bitField0_ |= 0x00000008;
-        estimatedCompletionPercentage_ = value;
-        onChanged();
+      public Builder setProfile(
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
+        if (profileBuilder_ == null) {
+          profile_ = builderForValue.build();
+          onChanged();
+        } else {
+          profileBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
         return this;
       }
       /**
-       * <code>optional int32 estimated_completion_percentage = 4;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public Builder clearEstimatedCompletionPercentage() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        estimatedCompletionPercentage_ = 0;
-        onChanged();
+      public Builder mergeProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (profileBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              profile_ != org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance()) {
+            profile_ =
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.newBuilder(profile_).mergeFrom(value).buildPartial();
+          } else {
+            profile_ = value;
+          }
+          onChanged();
+        } else {
+          profileBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
         return this;
       }
-
-      // optional .exec.bit.control.FragmentStatus.FragmentState state = 5;
-      private org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
       /**
-       * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public boolean hasState() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
-      }
-      /**
-       * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-       */
-      public org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState getState() {
-        return state_;
-      }
-      /**
-       * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
-       */
-      public Builder setState(org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState value) {
-        if (value == null) {
-          throw new NullPointerException();
+      public Builder clearProfile() {
+        if (profileBuilder_ == null) {
+          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
+          onChanged();
+        } else {
+          profileBuilder_.clear();
         }
-        bitField0_ |= 0x00000010;
-        state_ = value;
-        onChanged();
+        bitField0_ = (bitField0_ & ~0x00000001);
         return this;
       }
       /**
-       * <code>optional .exec.bit.control.FragmentStatus.FragmentState state = 5;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public Builder clearState() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        state_ = org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState.SENDING;
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder getProfileBuilder() {
+        bitField0_ |= 0x00000001;
         onChanged();
-        return this;
-      }
-
-      // optional int64 data_processed = 6;
-      private long dataProcessed_ ;
-      /**
-       * <code>optional int64 data_processed = 6;</code>
-       */
-      public boolean hasDataProcessed() {
-        return ((bitField0_ & 0x00000020) == 0x00000020);
-      }
-      /**
-       * <code>optional int64 data_processed = 6;</code>
-       */
-      public long getDataProcessed() {
-        return dataProcessed_;
+        return getProfileFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional int64 data_processed = 6;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public Builder setDataProcessed(long value) {
-        bitField0_ |= 0x00000020;
-        dataProcessed_ = value;
-        onChanged();
-        return this;
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
+        if (profileBuilder_ != null) {
+          return profileBuilder_.getMessageOrBuilder();
+        } else {
+          return profile_;
+        }
       }
       /**
-       * <code>optional int64 data_processed = 6;</code>
+       * <code>optional .exec.shared.MinorFragmentProfile profile = 1;</code>
        */
-      public Builder clearDataProcessed() {
-        bitField0_ = (bitField0_ & ~0x00000020);
-        dataProcessed_ = 0L;
-        onChanged();
-        return this;
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> 
+          getProfileFieldBuilder() {
+        if (profileBuilder_ == null) {
+          profileBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder>(
+                  profile_,
+                  getParentForChildren(),
+                  isClean());
+          profile_ = null;
+        }
+        return profileBuilder_;
       }
 
-      // optional .exec.bit.FragmentHandle handle = 7;
+      // optional .exec.bit.FragmentHandle handle = 2;
       private org.apache.drill.exec.proto.ExecProtos.FragmentHandle handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> handleBuilder_;
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public boolean hasHandle() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
+        return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public org.apache.drill.exec.proto.ExecProtos.FragmentHandle getHandle() {
         if (handleBuilder_ == null) {
@@ -2786,7 +2167,7 @@ public final class BitControl {
         }
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public Builder setHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
         if (handleBuilder_ == null) {
@@ -2798,11 +2179,11 @@ public final class BitControl {
         } else {
           handleBuilder_.setMessage(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public Builder setHandle(
           org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder builderForValue) {
@@ -2812,15 +2193,15 @@ public final class BitControl {
         } else {
           handleBuilder_.setMessage(builderForValue.build());
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public Builder mergeHandle(org.apache.drill.exec.proto.ExecProtos.FragmentHandle value) {
         if (handleBuilder_ == null) {
-          if (((bitField0_ & 0x00000040) == 0x00000040) &&
+          if (((bitField0_ & 0x00000002) == 0x00000002) &&
               handle_ != org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance()) {
             handle_ =
               org.apache.drill.exec.proto.ExecProtos.FragmentHandle.newBuilder(handle_).mergeFrom(value).buildPartial();
@@ -2831,11 +2212,11 @@ public final class BitControl {
         } else {
           handleBuilder_.mergeFrom(value);
         }
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000002;
         return this;
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public Builder clearHandle() {
         if (handleBuilder_ == null) {
@@ -2844,19 +2225,19 @@ public final class BitControl {
         } else {
           handleBuilder_.clear();
         }
-        bitField0_ = (bitField0_ & ~0x00000040);
+        bitField0_ = (bitField0_ & ~0x00000002);
         return this;
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder getHandleBuilder() {
-        bitField0_ |= 0x00000040;
+        bitField0_ |= 0x00000002;
         onChanged();
         return getHandleFieldBuilder().getBuilder();
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       public org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder getHandleOrBuilder() {
         if (handleBuilder_ != null) {
@@ -2866,7 +2247,7 @@ public final class BitControl {
         }
       }
       /**
-       * <code>optional .exec.bit.FragmentHandle handle = 7;</code>
+       * <code>optional .exec.bit.FragmentHandle handle = 2;</code>
        */
       private com.google.protobuf.SingleFieldBuilder<
           org.apache.drill.exec.proto.ExecProtos.FragmentHandle, org.apache.drill.exec.proto.ExecProtos.FragmentHandle.Builder, org.apache.drill.exec.proto.ExecProtos.FragmentHandleOrBuilder> 
@@ -2882,273 +2263,6 @@ public final class BitControl {
         return handleBuilder_;
       }
 
-      // optional .exec.shared.DrillPBError error = 8;
-      private org.apache.drill.exec.proto.UserBitShared.DrillPBError error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> errorBuilder_;
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public boolean hasError() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError getError() {
-        if (errorBuilder_ == null) {
-          return error_;
-        } else {
-          return errorBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public Builder setError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
-        if (errorBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          error_ = value;
-          onChanged();
-        } else {
-          errorBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000080;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public Builder setError(
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder builderForValue) {
-        if (errorBuilder_ == null) {
-          error_ = builderForValue.build();
-          onChanged();
-        } else {
-          errorBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000080;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public Builder mergeError(org.apache.drill.exec.proto.UserBitShared.DrillPBError value) {
-        if (errorBuilder_ == null) {
-          if (((bitField0_ & 0x00000080) == 0x00000080) &&
-              error_ != org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance()) {
-            error_ =
-              org.apache.drill.exec.proto.UserBitShared.DrillPBError.newBuilder(error_).mergeFrom(value).buildPartial();
-          } else {
-            error_ = value;
-          }
-          onChanged();
-        } else {
-          errorBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000080;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public Builder clearError() {
-        if (errorBuilder_ == null) {
-          error_ = org.apache.drill.exec.proto.UserBitShared.DrillPBError.getDefaultInstance();
-          onChanged();
-        } else {
-          errorBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000080);
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder getErrorBuilder() {
-        bitField0_ |= 0x00000080;
-        onChanged();
-        return getErrorFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder getErrorOrBuilder() {
-        if (errorBuilder_ != null) {
-          return errorBuilder_.getMessageOrBuilder();
-        } else {
-          return error_;
-        }
-      }
-      /**
-       * <code>optional .exec.shared.DrillPBError error = 8;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder> 
-          getErrorFieldBuilder() {
-        if (errorBuilder_ == null) {
-          errorBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.drill.exec.proto.UserBitShared.DrillPBError, org.apache.drill.exec.proto.UserBitShared.DrillPBError.Builder, org.apache.drill.exec.proto.UserBitShared.DrillPBErrorOrBuilder>(
-                  error_,
-                  getParentForChildren(),
-                  isClean());
-          error_ = null;
-        }
-        return errorBuilder_;
-      }
-
-      // optional int64 running_time = 9;
-      private long runningTime_ ;
-      /**
-       * <code>optional int64 running_time = 9;</code>
-       */
-      public boolean hasRunningTime() {
-        return ((bitField0_ & 0x00000100) == 0x00000100);
-      }
-      /**
-       * <code>optional int64 running_time = 9;</code>
-       */
-      public long getRunningTime() {
-        return runningTime_;
-      }
-      /**
-       * <code>optional int64 running_time = 9;</code>
-       */
-      public Builder setRunningTime(long value) {
-        bitField0_ |= 0x00000100;
-        runningTime_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int64 running_time = 9;</code>
-       */
-      public Builder clearRunningTime() {
-        bitField0_ = (bitField0_ & ~0x00000100);
-        runningTime_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // optional .exec.shared.MinorFragmentProfile profile = 10;
-      private org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> profileBuilder_;
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public boolean hasProfile() {
-        return ((bitField0_ & 0x00000200) == 0x00000200);
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getProfile() {
-        if (profileBuilder_ == null) {
-          return profile_;
-        } else {
-          return profileBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public Builder setProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
-        if (profileBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          profile_ = value;
-          onChanged();
-        } else {
-          profileBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000200;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public Builder setProfile(
-          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
-        if (profileBuilder_ == null) {
-          profile_ = builderForValue.build();
-          onChanged();
-        } else {
-          profileBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000200;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public Builder mergeProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
-        if (profileBuilder_ == null) {
-          if (((bitField0_ & 0x00000200) == 0x00000200) &&
-              profile_ != org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance()) {
-            profile_ =
-              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.newBuilder(profile_).mergeFrom(value).buildPartial();
-          } else {
-            profile_ = value;
-          }
-          onChanged();
-        } else {
-          profileBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000200;
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public Builder clearProfile() {
-        if (profileBuilder_ == null) {
-          profile_ = org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.getDefaultInstance();
-          onChanged();
-        } else {
-          profileBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000200);
-        return this;
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder getProfileBuilder() {
-        bitField0_ |= 0x00000200;
-        onChanged();
-        return getProfileFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getProfileOrBuilder() {
-        if (profileBuilder_ != null) {
-          return profileBuilder_.getMessageOrBuilder();
-        } else {
-          return profile_;
-        }
-      }
-      /**
-       * <code>optional .exec.shared.MinorFragmentProfile profile = 10;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> 
-          getProfileFieldBuilder() {
-        if (profileBuilder_ == null) {
-          profileBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder>(
-                  profile_,
-                  getParentForChildren(),
-                  isClean());
-          profile_ = null;
-        }
-        return profileBuilder_;
-      }
-
       // @@protoc_insertion_point(builder_scope:exec.bit.control.FragmentStatus)
     }
 
@@ -6090,40 +5204,31 @@ public final class BitControl {
       "(\0162\027.exec.shared.RpcChannel:\013BIT_CONTROL" +
       "\022(\n\010endpoint\030\003 \001(\0132\026.exec.DrillbitEndpoi" +
       "nt\"F\n\tBitStatus\0229\n\017fragment_status\030\001 \003(\013" +
-      "2 .exec.bit.control.FragmentStatus\"\345\003\n\016F" +
-      "ragmentStatus\022\022\n\nmemory_use\030\001 \001(\003\022\031\n\021bat" +
-      "ches_completed\030\002 \001(\003\022\031\n\021records_complete",
-      "d\030\003 \001(\003\022\'\n\037estimated_completion_percenta" +
-      "ge\030\004 \001(\005\022=\n\005state\030\005 \001(\0162..exec.bit.contr" +
-      "ol.FragmentStatus.FragmentState\022\026\n\016data_" +
-      "processed\030\006 \001(\003\022(\n\006handle\030\007 \001(\0132\030.exec.b" +
-      "it.FragmentHandle\022(\n\005error\030\010 \001(\0132\031.exec." +
-      "shared.DrillPBError\022\024\n\014running_time\030\t \001(" +
-      "\003\0222\n\007profile\030\n \001(\0132!.exec.shared.MinorFr" +
-      "agmentProfile\"k\n\rFragmentState\022\013\n\007SENDIN" +
-      "G\020\000\022\027\n\023AWAITING_ALLOCATION\020\001\022\013\n\007RUNNING\020" +
-      "\002\022\014\n\010FINISHED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED",
-      "\020\005\"\276\003\n\014PlanFragment\022(\n\006handle\030\001 \001(\0132\030.ex" +
-      "ec.bit.FragmentHandle\022\024\n\014network_cost\030\004 " +
-      "\001(\002\022\020\n\010cpu_cost\030\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002" +
-      "\022\023\n\013memory_cost\030\007 \001(\002\022\025\n\rfragment_json\030\010" +
-      " \001(\t\022*\n\nassignment\030\n \001(\0132\026.exec.Drillbit" +
-      "Endpoint\022\025\n\rleaf_fragment\030\t \001(\010\022\'\n\007forem" +
-      "an\030\013 \001(\0132\026.exec.DrillbitEndpoint\022\035\n\013mem_" +
-      "initial\030\014 \001(\003:\01020000000\022\034\n\007mem_max\030\r \001(\003" +
-      ":\01320000000000\022\030\n\020query_start_time\030\016 \001(\003\022" +
-      "1\n\013credentials\030\017 \001(\0132\034.exec.shared.UserC",
-      "redentials\022\021\n\ttime_zone\030\020 \001(\005\022\024\n\014options" +
-      "_json\030\021 \001(\t\"f\n\017WorkQueueStatus\022(\n\010endpoi" +
-      "nt\030\001 \001(\0132\026.exec.DrillbitEndpoint\022\024\n\014queu" +
-      "e_length\030\002 \001(\005\022\023\n\013report_time\030\003 \001(\003*\332\001\n\007" +
-      "RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODB" +
-      "YE\020\002\022\033\n\027REQ_INIATILIZE_FRAGMENT\020\003\022\027\n\023REQ" +
-      "_CANCEL_FRAGMENT\020\006\022\027\n\023REQ_FRAGMENT_STATU" +
-      "S\020\007\022\022\n\016REQ_BIT_STATUS\020\010\022\030\n\024RESP_FRAGMENT" +
-      "_HANDLE\020\t\022\030\n\024RESP_FRAGMENT_STATUS\020\n\022\023\n\017R" +
-      "ESP_BIT_STATUS\020\013B+\n\033org.apache.drill.exe",
-      "c.protoB\nBitControlH\001"
+      "2 .exec.bit.control.FragmentStatus\"n\n\016Fr" +
+      "agmentStatus\0222\n\007profile\030\001 \001(\0132!.exec.sha" +
+      "red.MinorFragmentProfile\022(\n\006handle\030\002 \001(\013",
+      "2\030.exec.bit.FragmentHandle\"\276\003\n\014PlanFragm" +
+      "ent\022(\n\006handle\030\001 \001(\0132\030.exec.bit.FragmentH" +
+      "andle\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_cost\030" +
+      "\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_cost\030" +
+      "\007 \001(\002\022\025\n\rfragment_json\030\010 \001(\t\022*\n\nassignme" +
+      "nt\030\n \001(\0132\026.exec.DrillbitEndpoint\022\025\n\rleaf" +
+      "_fragment\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.exec." +
+      "DrillbitEndpoint\022\035\n\013mem_initial\030\014 \001(\003:\0102" +
+      "0000000\022\034\n\007mem_max\030\r \001(\003:\01320000000000\022\030\n" +
+      "\020query_start_time\030\016 \001(\003\0221\n\013credentials\030\017",
+      " \001(\0132\034.exec.shared.UserCredentials\022\021\n\tti" +
+      "me_zone\030\020 \001(\005\022\024\n\014options_json\030\021 \001(\t\"f\n\017W" +
+      "orkQueueStatus\022(\n\010endpoint\030\001 \001(\0132\026.exec." +
+      "DrillbitEndpoint\022\024\n\014queue_length\030\002 \001(\005\022\023" +
+      "\n\013report_time\030\003 \001(\003*\332\001\n\007RpcType\022\r\n\tHANDS" +
+      "HAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027REQ_INIA" +
+      "TILIZE_FRAGMENT\020\003\022\027\n\023REQ_CANCEL_FRAGMENT" +
+      "\020\006\022\027\n\023REQ_FRAGMENT_STATUS\020\007\022\022\n\016REQ_BIT_S" +
+      "TATUS\020\010\022\030\n\024RESP_FRAGMENT_HANDLE\020\t\022\030\n\024RES" +
+      "P_FRAGMENT_STATUS\020\n\022\023\n\017RESP_BIT_STATUS\020\013",
+      "B+\n\033org.apache.drill.exec.protoB\nBitCont" +
+      "rolH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -6147,7 +5252,7 @@ public final class BitControl {
           internal_static_exec_bit_control_FragmentStatus_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_FragmentStatus_descriptor,
-              new java.lang.String[] { "MemoryUse", "BatchesCompleted", "RecordsCompleted", "EstimatedCompletionPercentage", "State", "DataProcessed", "Handle", "Error", "RunningTime", "Profile", });
+              new java.lang.String[] { "Profile", "Handle", });
           internal_static_exec_bit_control_PlanFragment_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new


Re: [21/24] git commit: Remove references to jcommander's copy of Guava's Lists class.

Posted by Timothy Chen <tn...@gmail.com>.
Cool :)

Thanks for doing this.

Tim

On Wed, May 21, 2014 at 7:48 PM, Jacques Nadeau <ja...@apache.org> wrote:
> Yeah, it is another one of the commits in my working branch.  :)
>
> There is a way to add a maven plugin that makes sure we don't reference
> these in the future.  I need to track it down again and create a JIRA for
> it.
>
>
> On Wed, May 21, 2014 at 6:17 PM, Timothy Chen <tn...@gmail.com> wrote:
>
>> May we also remove all the Hive12.Lists reference as well?
>>
>> +import com.google.hive12.common.collect.Lists;
>>
>> I see these cropping up everywhere now.
>>
>> Tim
>>
>> On Wed, May 21, 2014 at 6:14 PM,  <ja...@apache.org> wrote:
>> > Remove references to jcommander's copy of Guava's Lists class.
>> >
>> >
>> > Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
>> > Commit:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/864ebde0
>> > Tree:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/864ebde0
>> > Diff:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/864ebde0
>> >
>> > Branch: refs/heads/diagnostics2
>> > Commit: 864ebde0384030d5d553251dc879ea3c2b98164c
>> > Parents: fbdca8f
>> > Author: Jacques Nadeau <ja...@apache.org>
>> > Authored: Wed May 21 13:34:40 2014 -0700
>> > Committer: Jacques Nadeau <ja...@apache.org>
>> > Committed: Wed May 21 13:43:33 2014 -0700
>> >
>> > ----------------------------------------------------------------------
>> >  .../src/main/java/org/apache/drill/exec/client/DumpCat.java      | 2 +-
>> >  .../java/org/apache/drill/exec/compile/ClassTransformer.java     | 2 +-
>> >  .../drill/exec/compile/sig/ConstantExpressionIdentifier.java     | 2 +-
>> >  .../java/org/apache/drill/exec/compile/sig/SignatureHolder.java  | 2 +-
>> >  .../src/main/java/org/apache/drill/exec/expr/ClassGenerator.java | 2 +-
>> >  .../src/main/java/org/apache/drill/exec/ops/FragmentContext.java | 2 +-
>> >  .../src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java  | 2 +-
>> >  .../java/org/apache/drill/exec/physical/config/HashJoinPOP.java  | 4
>> ++--
>> >  .../java/org/apache/drill/exec/physical/config/MergeJoinPOP.java | 4
>> ++--
>> >  .../drill/exec/physical/config/OrderedPartitionSender.java       | 2 +-
>> >  .../apache/drill/exec/physical/impl/limit/LimitRecordBatch.java  | 2 +-
>> >  .../exec/physical/impl/xsort/PriorityQueueCopierTemplate.java    | 2 +-
>> >  .../org/apache/drill/exec/planner/logical/DrillJoinRule.java     | 2 +-
>> >  .../java/org/apache/drill/exec/planner/physical/HashAggPrel.java | 2 +-
>> >  .../org/apache/drill/exec/planner/physical/HashJoinPrel.java     | 2 +-
>> >  .../org/apache/drill/exec/planner/physical/MergeJoinPrel.java    | 2 +-
>> >  .../java/org/apache/drill/exec/planner/physical/PrelUtil.java    | 2 +-
>> >  .../org/apache/drill/exec/planner/physical/StreamAggPrel.java    | 2 +-
>> >  .../apache/drill/exec/planner/sql/parser/DrillParserUtil.java    | 2 +-
>> >  .../main/java/org/apache/drill/exec/record/VectorContainer.java  | 2 +-
>> >  .../java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java | 2 +-
>> >  .../java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java   | 2 +-
>> >  .../org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java   | 2 +-
>> >  .../java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java | 2 +-
>> >  .../org/apache/drill/exec/store/hive/HiveScanBatchCreator.java   | 2 +-
>> >  .../main/java/org/apache/drill/exec/store/hive/HiveTable.java    | 2 +-
>> >  .../org/apache/drill/exec/store/schedule/AffinityCreator.java    | 2 +-
>> >  .../org/apache/drill/exec/store/schedule/BlockMapBuilder.java    | 2 +-
>> >  .../src/main/java/org/apache/drill/exec/util/BatchPrinter.java   | 2 +-
>> >  .../src/main/java/org/apache/drill/exec/util/VectorUtil.java     | 2 +-
>> >  .../test/java/org/apache/drill/exec/cache/TestVectorCache.java   | 2 +-
>> >  .../test/java/org/apache/drill/exec/cache/TestWriteToDisk.java   | 2 +-
>> >  .../java/org/apache/drill/exec/physical/impl/SimpleRootExec.java | 2 +-
>> >  .../org/apache/drill/exec/physical/impl/join/TestMergeJoin.java  | 2 +-
>> >  .../exec/physical/impl/mergereceiver/TestMergingReceiver.java    | 2 +-
>> >  35 files changed, 37 insertions(+), 37 deletions(-)
>> > ----------------------------------------------------------------------
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
>> > index 7e8a4a2..2d65e91 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
>> > @@ -36,7 +36,7 @@ import com.beust.jcommander.IParameterValidator;
>> >  import com.beust.jcommander.JCommander;
>> >  import com.beust.jcommander.Parameter;
>> >  import com.beust.jcommander.ParameterException;
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class DumpCat {
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
>> > index 09fdfa9..3dfc4ea 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
>> > @@ -30,7 +30,7 @@ import org.codehaus.commons.compiler.CompileException;
>> >  import org.objectweb.asm.ClassReader;
>> >  import org.objectweb.asm.tree.ClassNode;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.beust.jcommander.internal.Maps;
>> >  import com.beust.jcommander.internal.Sets;
>> >  import com.google.common.base.Preconditions;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
>> > index c0c8484..8a1efdf 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
>> > @@ -45,7 +45,7 @@ import
>> org.apache.drill.common.expression.ValueExpressions.Decimal38Expression;
>> >  import org.apache.drill.common.expression.ValueExpressions.QuotedString;
>> >  import org.apache.drill.common.expression.visitors.ExprVisitor;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class ConstantExpressionIdentifier implements
>> ExprVisitor<Boolean, IdentityHashMap<LogicalExpression, Object>,
>> RuntimeException>{
>> >    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ConstantExpressionIdentifier.class);
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
>> > index 1eb6732..158570a 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
>> > @@ -24,7 +24,7 @@ import java.util.Iterator;
>> >  import java.util.List;
>> >  import java.util.Map;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.beust.jcommander.internal.Maps;
>> >  import com.google.common.collect.ImmutableMap;
>> >  import com.google.common.collect.Iterators;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
>> > index fc48552..51ef003 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
>> > @@ -37,7 +37,7 @@ import
>> org.apache.drill.exec.exception.SchemaChangeException;
>> >  import org.apache.drill.exec.expr.fn.DrillFuncHolder.WorkspaceReference;
>> >  import org.apache.drill.exec.record.TypedFieldId;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.beust.jcommander.internal.Maps;
>> >  import com.google.common.base.Preconditions;
>> >  import com.sun.codemodel.JArray;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
>> > index 3f9f181..7812d11 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
>> > @@ -48,7 +48,7 @@ import org.apache.drill.exec.server.options.OptionList;
>> >  import org.apache.drill.exec.server.options.OptionManager;
>> >  import org.apache.drill.exec.work.batch.IncomingBuffers;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.beust.jcommander.internal.Maps;
>> >
>> >  /**
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
>> > index 78c0e3c..24d9cfe 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
>> > @@ -60,7 +60,7 @@ import org.apache.drill.exec.store.StoragePlugin;
>> >  import org.eigenbase.rel.RelFieldCollation.Direction;
>> >  import org.eigenbase.rel.RelFieldCollation.NullDirection;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class BasicOptimizer extends Optimizer{
>> >    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(BasicOptimizer.class);
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
>> > index 1ef7e97..06c23f3 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
>> > @@ -32,7 +32,7 @@ import
>> org.apache.drill.exec.physical.base.PhysicalOperator;
>> >  import org.apache.drill.exec.physical.base.PhysicalVisitor;
>> >  import org.apache.drill.exec.physical.base.Size;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JsonCreator;
>> >  import com.fasterxml.jackson.annotation.JsonProperty;
>> >  import com.google.common.base.Preconditions;
>> > @@ -111,7 +111,7 @@ public class HashJoinPOP extends AbstractBase {
>> >
>> >      public HashJoinPOP flipIfRight(){
>> >          if(joinType == JoinRelType.RIGHT){
>> > -            List<JoinCondition> flippedConditions =
>> Lists.newArrayList(conditions.size());
>> > +            List<JoinCondition> flippedConditions =
>> Lists.newArrayList();
>> >              for(JoinCondition c : conditions){
>> >                  flippedConditions.add(c.flip());
>> >              }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
>> > index be9cf95..b79a07e 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
>> > @@ -30,7 +30,7 @@ import org.apache.drill.exec.physical.base.Size;
>> >  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
>> >  import org.eigenbase.rel.JoinRelType;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JsonCreator;
>> >  import com.fasterxml.jackson.annotation.JsonProperty;
>> >  import com.fasterxml.jackson.annotation.JsonTypeName;
>> > @@ -106,7 +106,7 @@ public class MergeJoinPOP extends AbstractBase{
>> >
>> >    public MergeJoinPOP flipIfRight(){
>> >      if(joinType == JoinRelType.RIGHT){
>> > -      List<JoinCondition> flippedConditions =
>> Lists.newArrayList(conditions.size());
>> > +      List<JoinCondition> flippedConditions = Lists.newArrayList();
>> >        for(JoinCondition c : conditions){
>> >          flippedConditions.add(c.flip());
>> >        }
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
>> > index 83076a4..d73926e 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
>> > @@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.base.Size;
>> >  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>> >  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JsonCreator;
>> >  import com.fasterxml.jackson.annotation.JsonProperty;
>> >  import com.fasterxml.jackson.annotation.JsonTypeName;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
>> > index ed56e79..648fd89 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.physical.impl.limit;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.base.Objects;
>> >  import org.apache.drill.exec.exception.SchemaChangeException;
>> >  import org.apache.drill.exec.memory.OutOfMemoryException;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
>> > index 4221ae2..686cc48 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.physical.impl.xsort;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.base.Stopwatch;
>> >  import org.apache.drill.common.expression.ExpressionPosition;
>> >  import org.apache.drill.common.expression.SchemaPath;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
>> > index 96854a7..e940b2e 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
>> > @@ -28,7 +28,7 @@ import org.eigenbase.rex.RexUtil;
>> >  import org.eigenbase.sql.fun.SqlStdOperatorTable;
>> >  import org.eigenbase.trace.EigenbaseTrace;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  import java.util.ArrayList;
>> >  import java.util.List;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
>> > index b33805c..b06c9e9 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
>> > @@ -48,7 +48,7 @@ import org.eigenbase.relopt.RelOptCost;
>> >  import org.eigenbase.relopt.RelOptPlanner;
>> >  import org.eigenbase.relopt.RelTraitSet;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class HashAggPrel extends AggregateRelBase implements Prel{
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
>> > index b03ef62..9ac0834 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
>> > @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
>> >  import org.eigenbase.rex.RexNode;
>> >  import org.eigenbase.util.Pair;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class HashJoinPrel  extends JoinPrel {
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
>> > index 1987e99..ee9480d 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
>> > @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
>> >  import org.eigenbase.rex.RexNode;
>> >  import org.eigenbase.util.Pair;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class MergeJoinPrel  extends JoinPrel {
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
>> > index 1cd480c..1de2db3 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
>> > @@ -49,7 +49,7 @@ import org.eigenbase.rex.RexLiteral;
>> >  import org.eigenbase.rex.RexNode;
>> >  import org.eigenbase.rex.RexVisitorImpl;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.collect.ImmutableList;
>> >  import com.google.common.collect.Sets;
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
>> > index b35d1bb..f7cda32 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
>> > @@ -49,7 +49,7 @@ import org.eigenbase.relopt.RelOptCost;
>> >  import org.eigenbase.relopt.RelOptPlanner;
>> >  import org.eigenbase.relopt.RelTraitSet;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class StreamAggPrel extends AggregateRelBase implements Prel{
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
>> > index 4b427be..ed36903 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
>> > @@ -18,7 +18,7 @@
>> >
>> >  package org.apache.drill.exec.planner.sql.parser;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import org.eigenbase.sql.SqlNode;
>> >  import org.eigenbase.sql.SqlOperator;
>> >  import org.eigenbase.sql.parser.SqlParserPos;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
>> > index 1c7714e..dd62c67 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
>> > @@ -27,7 +27,7 @@ import
>> org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >  import org.apache.drill.exec.vector.complex.AbstractMapVector;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.base.Preconditions;
>> >
>> >  public class VectorContainer extends AbstractMapVector implements
>> Iterable<VectorWrapper<?>>, VectorAccessible {
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
>> > index 232ec07..4ac79fd 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
>> > @@ -29,7 +29,7 @@ import
>> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>> >  import org.apache.hadoop.fs.FSDataInputStream;
>> >  import org.apache.hadoop.fs.FileStatus;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JsonIgnore;
>> >  import com.google.common.collect.ImmutableList;
>> >  import com.google.common.collect.Range;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
>> > index 39d71e9..a26ea1f 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
>> > @@ -39,7 +39,7 @@ import
>> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>> >  import org.apache.drill.exec.store.dfs.shim.FileSystemCreator;
>> >  import org.apache.hadoop.conf.Configuration;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.beust.jcommander.internal.Maps;
>> >
>> >  import org.apache.hadoop.fs.FileSystem;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
>> > index 718da23..e702c9c 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
>> > @@ -49,7 +49,7 @@ import
>> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>> >  import org.apache.hadoop.conf.Configuration;
>> >  import org.apache.hadoop.io.compress.CompressionCodecFactory;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.collect.ImmutableSet;
>> >
>> >  public abstract class EasyFormatPlugin<T extends FormatPluginConfig>
>> implements FormatPlugin {
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
>> > index 03e2095..d0cd8cc 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
>> > @@ -40,7 +40,7 @@ import
>> org.apache.drill.exec.store.schedule.BlockMapBuilder;
>> >  import org.apache.drill.exec.store.schedule.CompleteFileWork;
>> >  import
>> org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JacksonInject;
>> >  import com.fasterxml.jackson.annotation.JsonCreator;
>> >  import com.fasterxml.jackson.annotation.JsonIgnore;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
>> > index 3dc9ac4..a0837bc 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.store.hive;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
>> >  import org.apache.drill.exec.ops.FragmentContext;
>> >  import org.apache.drill.exec.physical.impl.BatchCreator;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
>> > index 3858804..df2e428 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.store.hive;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.fasterxml.jackson.annotation.JsonCreator;
>> >  import com.fasterxml.jackson.annotation.JsonIgnore;
>> >  import com.fasterxml.jackson.annotation.JsonProperty;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
>> > index d25abad..56d0485 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
>> > @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
>> >  import org.apache.drill.exec.physical.EndpointAffinity;
>> >  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
>> >  import com.carrotsearch.hppc.cursors.ObjectFloatCursor;
>> >  import com.carrotsearch.hppc.cursors.ObjectLongCursor;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
>> > index 1ad134e..f27e8e6 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
>> > @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
>> >  import org.apache.hadoop.fs.FileSystem;
>> >  import org.apache.hadoop.fs.Path;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.codahale.metrics.MetricRegistry;
>> >  import com.codahale.metrics.Timer;
>> >  import com.google.common.base.Stopwatch;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
>> > index e5b8ad4..6222514 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.util;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  import org.apache.commons.lang.StringUtils;
>> >  import org.apache.drill.common.expression.ExpressionStringBuilder;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
>> > index a43225e..a19be27 100644
>> > ---
>> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
>> > +++
>> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
>> > @@ -26,7 +26,7 @@ import org.apache.drill.exec.record.VectorAccessible;
>> >  import org.apache.drill.exec.record.VectorWrapper;
>> >  import org.apache.drill.exec.rpc.RpcException;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class VectorUtil {
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
>> > index 49f41e3..3e0be69 100644
>> > ---
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
>> > +++
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
>> > @@ -43,7 +43,7 @@ import org.apache.drill.exec.vector.ValueVector;
>> >  import org.apache.drill.exec.vector.VarBinaryVector;
>> >  import org.junit.Test;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  public class TestVectorCache extends ExecTest{
>> >
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
>> > index d3f6da5..2666233 100644
>> > ---
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
>> > +++
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
>> > @@ -17,7 +17,7 @@
>> >   */
>> >  package org.apache.drill.exec.cache;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  import org.apache.drill.common.config.DrillConfig;
>> >  import org.apache.drill.common.expression.ExpressionPosition;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
>> > index 62f9a21..7d04340 100644
>> > ---
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
>> > +++
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
>> > @@ -31,7 +31,7 @@ import
>> org.apache.drill.exec.record.selection.SelectionVector2;
>> >  import org.apache.drill.exec.record.selection.SelectionVector4;
>> >  import org.apache.drill.exec.vector.ValueVector;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >
>> >  @Deprecated
>> >  public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
>> > index 5ca7e72..4c1ce93 100644
>> > ---
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
>> > +++
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
>> > @@ -56,7 +56,7 @@ import org.junit.AfterClass;
>> >  import org.junit.Ignore;
>> >  import org.junit.Test;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import com.google.common.base.Charsets;
>> >  import com.google.common.io.Files;
>> >  import com.codahale.metrics.MetricRegistry;
>> >
>> >
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
>> > ----------------------------------------------------------------------
>> > diff --git
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
>> > index 14d6fe2..3fb04aa 100644
>> > ---
>> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
>> > +++
>> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
>> > @@ -20,7 +20,7 @@ package
>> org.apache.drill.exec.physical.impl.mergereceiver;
>> >
>> >  import java.util.List;
>> >
>> > -import com.beust.jcommander.internal.Lists;
>> > +import com.google.common.collect.Lists;
>> >  import org.apache.drill.common.util.FileUtils;
>> >  import org.apache.drill.exec.client.DrillClient;
>> >  import org.apache.drill.exec.pop.PopUnitTestBase;
>> >
>>

Re: [21/24] git commit: Remove references to jcommander's copy of Guava's Lists class.

Posted by Jacques Nadeau <ja...@apache.org>.
Yeah, it is another one of the commits in my working branch.  :)

There is a way to add a maven plugin that makes sure we don't reference
these in the future.  I need to track it down again and create a JIRA for
it.


On Wed, May 21, 2014 at 6:17 PM, Timothy Chen <tn...@gmail.com> wrote:

> May we also remove all the Hive12.Lists reference as well?
>
> +import com.google.hive12.common.collect.Lists;
>
> I see these cropping up everywhere now.
>
> Tim
>
> On Wed, May 21, 2014 at 6:14 PM,  <ja...@apache.org> wrote:
> > Remove references to jcommander's copy of Guava's Lists class.
> >
> >
> > Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> > Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/864ebde0
> > Tree:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/864ebde0
> > Diff:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/864ebde0
> >
> > Branch: refs/heads/diagnostics2
> > Commit: 864ebde0384030d5d553251dc879ea3c2b98164c
> > Parents: fbdca8f
> > Author: Jacques Nadeau <ja...@apache.org>
> > Authored: Wed May 21 13:34:40 2014 -0700
> > Committer: Jacques Nadeau <ja...@apache.org>
> > Committed: Wed May 21 13:43:33 2014 -0700
> >
> > ----------------------------------------------------------------------
> >  .../src/main/java/org/apache/drill/exec/client/DumpCat.java      | 2 +-
> >  .../java/org/apache/drill/exec/compile/ClassTransformer.java     | 2 +-
> >  .../drill/exec/compile/sig/ConstantExpressionIdentifier.java     | 2 +-
> >  .../java/org/apache/drill/exec/compile/sig/SignatureHolder.java  | 2 +-
> >  .../src/main/java/org/apache/drill/exec/expr/ClassGenerator.java | 2 +-
> >  .../src/main/java/org/apache/drill/exec/ops/FragmentContext.java | 2 +-
> >  .../src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java  | 2 +-
> >  .../java/org/apache/drill/exec/physical/config/HashJoinPOP.java  | 4
> ++--
> >  .../java/org/apache/drill/exec/physical/config/MergeJoinPOP.java | 4
> ++--
> >  .../drill/exec/physical/config/OrderedPartitionSender.java       | 2 +-
> >  .../apache/drill/exec/physical/impl/limit/LimitRecordBatch.java  | 2 +-
> >  .../exec/physical/impl/xsort/PriorityQueueCopierTemplate.java    | 2 +-
> >  .../org/apache/drill/exec/planner/logical/DrillJoinRule.java     | 2 +-
> >  .../java/org/apache/drill/exec/planner/physical/HashAggPrel.java | 2 +-
> >  .../org/apache/drill/exec/planner/physical/HashJoinPrel.java     | 2 +-
> >  .../org/apache/drill/exec/planner/physical/MergeJoinPrel.java    | 2 +-
> >  .../java/org/apache/drill/exec/planner/physical/PrelUtil.java    | 2 +-
> >  .../org/apache/drill/exec/planner/physical/StreamAggPrel.java    | 2 +-
> >  .../apache/drill/exec/planner/sql/parser/DrillParserUtil.java    | 2 +-
> >  .../main/java/org/apache/drill/exec/record/VectorContainer.java  | 2 +-
> >  .../java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java | 2 +-
> >  .../java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java   | 2 +-
> >  .../org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java   | 2 +-
> >  .../java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java | 2 +-
> >  .../org/apache/drill/exec/store/hive/HiveScanBatchCreator.java   | 2 +-
> >  .../main/java/org/apache/drill/exec/store/hive/HiveTable.java    | 2 +-
> >  .../org/apache/drill/exec/store/schedule/AffinityCreator.java    | 2 +-
> >  .../org/apache/drill/exec/store/schedule/BlockMapBuilder.java    | 2 +-
> >  .../src/main/java/org/apache/drill/exec/util/BatchPrinter.java   | 2 +-
> >  .../src/main/java/org/apache/drill/exec/util/VectorUtil.java     | 2 +-
> >  .../test/java/org/apache/drill/exec/cache/TestVectorCache.java   | 2 +-
> >  .../test/java/org/apache/drill/exec/cache/TestWriteToDisk.java   | 2 +-
> >  .../java/org/apache/drill/exec/physical/impl/SimpleRootExec.java | 2 +-
> >  .../org/apache/drill/exec/physical/impl/join/TestMergeJoin.java  | 2 +-
> >  .../exec/physical/impl/mergereceiver/TestMergingReceiver.java    | 2 +-
> >  35 files changed, 37 insertions(+), 37 deletions(-)
> > ----------------------------------------------------------------------
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> > index 7e8a4a2..2d65e91 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> > @@ -36,7 +36,7 @@ import com.beust.jcommander.IParameterValidator;
> >  import com.beust.jcommander.JCommander;
> >  import com.beust.jcommander.Parameter;
> >  import com.beust.jcommander.ParameterException;
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class DumpCat {
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> > index 09fdfa9..3dfc4ea 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> > @@ -30,7 +30,7 @@ import org.codehaus.commons.compiler.CompileException;
> >  import org.objectweb.asm.ClassReader;
> >  import org.objectweb.asm.tree.ClassNode;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.beust.jcommander.internal.Maps;
> >  import com.beust.jcommander.internal.Sets;
> >  import com.google.common.base.Preconditions;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> > index c0c8484..8a1efdf 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> > @@ -45,7 +45,7 @@ import
> org.apache.drill.common.expression.ValueExpressions.Decimal38Expression;
> >  import org.apache.drill.common.expression.ValueExpressions.QuotedString;
> >  import org.apache.drill.common.expression.visitors.ExprVisitor;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class ConstantExpressionIdentifier implements
> ExprVisitor<Boolean, IdentityHashMap<LogicalExpression, Object>,
> RuntimeException>{
> >    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ConstantExpressionIdentifier.class);
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> > index 1eb6732..158570a 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> > @@ -24,7 +24,7 @@ import java.util.Iterator;
> >  import java.util.List;
> >  import java.util.Map;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.beust.jcommander.internal.Maps;
> >  import com.google.common.collect.ImmutableMap;
> >  import com.google.common.collect.Iterators;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> > index fc48552..51ef003 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> > @@ -37,7 +37,7 @@ import
> org.apache.drill.exec.exception.SchemaChangeException;
> >  import org.apache.drill.exec.expr.fn.DrillFuncHolder.WorkspaceReference;
> >  import org.apache.drill.exec.record.TypedFieldId;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.beust.jcommander.internal.Maps;
> >  import com.google.common.base.Preconditions;
> >  import com.sun.codemodel.JArray;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> > index 3f9f181..7812d11 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> > @@ -48,7 +48,7 @@ import org.apache.drill.exec.server.options.OptionList;
> >  import org.apache.drill.exec.server.options.OptionManager;
> >  import org.apache.drill.exec.work.batch.IncomingBuffers;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.beust.jcommander.internal.Maps;
> >
> >  /**
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> > index 78c0e3c..24d9cfe 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> > @@ -60,7 +60,7 @@ import org.apache.drill.exec.store.StoragePlugin;
> >  import org.eigenbase.rel.RelFieldCollation.Direction;
> >  import org.eigenbase.rel.RelFieldCollation.NullDirection;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class BasicOptimizer extends Optimizer{
> >    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(BasicOptimizer.class);
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> > index 1ef7e97..06c23f3 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> > @@ -32,7 +32,7 @@ import
> org.apache.drill.exec.physical.base.PhysicalOperator;
> >  import org.apache.drill.exec.physical.base.PhysicalVisitor;
> >  import org.apache.drill.exec.physical.base.Size;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JsonCreator;
> >  import com.fasterxml.jackson.annotation.JsonProperty;
> >  import com.google.common.base.Preconditions;
> > @@ -111,7 +111,7 @@ public class HashJoinPOP extends AbstractBase {
> >
> >      public HashJoinPOP flipIfRight(){
> >          if(joinType == JoinRelType.RIGHT){
> > -            List<JoinCondition> flippedConditions =
> Lists.newArrayList(conditions.size());
> > +            List<JoinCondition> flippedConditions =
> Lists.newArrayList();
> >              for(JoinCondition c : conditions){
> >                  flippedConditions.add(c.flip());
> >              }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> > index be9cf95..b79a07e 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> > @@ -30,7 +30,7 @@ import org.apache.drill.exec.physical.base.Size;
> >  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
> >  import org.eigenbase.rel.JoinRelType;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JsonCreator;
> >  import com.fasterxml.jackson.annotation.JsonProperty;
> >  import com.fasterxml.jackson.annotation.JsonTypeName;
> > @@ -106,7 +106,7 @@ public class MergeJoinPOP extends AbstractBase{
> >
> >    public MergeJoinPOP flipIfRight(){
> >      if(joinType == JoinRelType.RIGHT){
> > -      List<JoinCondition> flippedConditions =
> Lists.newArrayList(conditions.size());
> > +      List<JoinCondition> flippedConditions = Lists.newArrayList();
> >        for(JoinCondition c : conditions){
> >          flippedConditions.add(c.flip());
> >        }
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> > index 83076a4..d73926e 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> > @@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.base.Size;
> >  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> >  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JsonCreator;
> >  import com.fasterxml.jackson.annotation.JsonProperty;
> >  import com.fasterxml.jackson.annotation.JsonTypeName;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> > index ed56e79..648fd89 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.physical.impl.limit;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.base.Objects;
> >  import org.apache.drill.exec.exception.SchemaChangeException;
> >  import org.apache.drill.exec.memory.OutOfMemoryException;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> > index 4221ae2..686cc48 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.physical.impl.xsort;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.base.Stopwatch;
> >  import org.apache.drill.common.expression.ExpressionPosition;
> >  import org.apache.drill.common.expression.SchemaPath;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> > index 96854a7..e940b2e 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> > @@ -28,7 +28,7 @@ import org.eigenbase.rex.RexUtil;
> >  import org.eigenbase.sql.fun.SqlStdOperatorTable;
> >  import org.eigenbase.trace.EigenbaseTrace;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  import java.util.ArrayList;
> >  import java.util.List;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> > index b33805c..b06c9e9 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> > @@ -48,7 +48,7 @@ import org.eigenbase.relopt.RelOptCost;
> >  import org.eigenbase.relopt.RelOptPlanner;
> >  import org.eigenbase.relopt.RelTraitSet;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class HashAggPrel extends AggregateRelBase implements Prel{
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> > index b03ef62..9ac0834 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> > @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
> >  import org.eigenbase.rex.RexNode;
> >  import org.eigenbase.util.Pair;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class HashJoinPrel  extends JoinPrel {
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> > index 1987e99..ee9480d 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> > @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
> >  import org.eigenbase.rex.RexNode;
> >  import org.eigenbase.util.Pair;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class MergeJoinPrel  extends JoinPrel {
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> > index 1cd480c..1de2db3 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> > @@ -49,7 +49,7 @@ import org.eigenbase.rex.RexLiteral;
> >  import org.eigenbase.rex.RexNode;
> >  import org.eigenbase.rex.RexVisitorImpl;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.collect.ImmutableList;
> >  import com.google.common.collect.Sets;
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> > index b35d1bb..f7cda32 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> > @@ -49,7 +49,7 @@ import org.eigenbase.relopt.RelOptCost;
> >  import org.eigenbase.relopt.RelOptPlanner;
> >  import org.eigenbase.relopt.RelTraitSet;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class StreamAggPrel extends AggregateRelBase implements Prel{
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> > index 4b427be..ed36903 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> > @@ -18,7 +18,7 @@
> >
> >  package org.apache.drill.exec.planner.sql.parser;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import org.eigenbase.sql.SqlNode;
> >  import org.eigenbase.sql.SqlOperator;
> >  import org.eigenbase.sql.parser.SqlParserPos;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> > index 1c7714e..dd62c67 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> > @@ -27,7 +27,7 @@ import
> org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
> >  import org.apache.drill.exec.vector.ValueVector;
> >  import org.apache.drill.exec.vector.complex.AbstractMapVector;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.base.Preconditions;
> >
> >  public class VectorContainer extends AbstractMapVector implements
> Iterable<VectorWrapper<?>>, VectorAccessible {
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> > index 232ec07..4ac79fd 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> > @@ -29,7 +29,7 @@ import
> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >  import org.apache.hadoop.fs.FSDataInputStream;
> >  import org.apache.hadoop.fs.FileStatus;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JsonIgnore;
> >  import com.google.common.collect.ImmutableList;
> >  import com.google.common.collect.Range;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> > index 39d71e9..a26ea1f 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> > @@ -39,7 +39,7 @@ import
> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >  import org.apache.drill.exec.store.dfs.shim.FileSystemCreator;
> >  import org.apache.hadoop.conf.Configuration;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.beust.jcommander.internal.Maps;
> >
> >  import org.apache.hadoop.fs.FileSystem;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> > index 718da23..e702c9c 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> > @@ -49,7 +49,7 @@ import
> org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
> >  import org.apache.hadoop.conf.Configuration;
> >  import org.apache.hadoop.io.compress.CompressionCodecFactory;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.collect.ImmutableSet;
> >
> >  public abstract class EasyFormatPlugin<T extends FormatPluginConfig>
> implements FormatPlugin {
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> > index 03e2095..d0cd8cc 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> > @@ -40,7 +40,7 @@ import
> org.apache.drill.exec.store.schedule.BlockMapBuilder;
> >  import org.apache.drill.exec.store.schedule.CompleteFileWork;
> >  import
> org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JacksonInject;
> >  import com.fasterxml.jackson.annotation.JsonCreator;
> >  import com.fasterxml.jackson.annotation.JsonIgnore;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> > index 3dc9ac4..a0837bc 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.store.hive;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import org.apache.drill.common.exceptions.ExecutionSetupException;
> >  import org.apache.drill.exec.ops.FragmentContext;
> >  import org.apache.drill.exec.physical.impl.BatchCreator;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> > index 3858804..df2e428 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.store.hive;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.fasterxml.jackson.annotation.JsonCreator;
> >  import com.fasterxml.jackson.annotation.JsonIgnore;
> >  import com.fasterxml.jackson.annotation.JsonProperty;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> > index d25abad..56d0485 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> > @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
> >  import org.apache.drill.exec.physical.EndpointAffinity;
> >  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
> >  import com.carrotsearch.hppc.cursors.ObjectFloatCursor;
> >  import com.carrotsearch.hppc.cursors.ObjectLongCursor;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> > index 1ad134e..f27e8e6 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> > @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
> >  import org.apache.hadoop.fs.FileSystem;
> >  import org.apache.hadoop.fs.Path;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.codahale.metrics.MetricRegistry;
> >  import com.codahale.metrics.Timer;
> >  import com.google.common.base.Stopwatch;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> > index e5b8ad4..6222514 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.util;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  import org.apache.commons.lang.StringUtils;
> >  import org.apache.drill.common.expression.ExpressionStringBuilder;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> > index a43225e..a19be27 100644
> > ---
> a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> > +++
> b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> > @@ -26,7 +26,7 @@ import org.apache.drill.exec.record.VectorAccessible;
> >  import org.apache.drill.exec.record.VectorWrapper;
> >  import org.apache.drill.exec.rpc.RpcException;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class VectorUtil {
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > index 49f41e3..3e0be69 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> > @@ -43,7 +43,7 @@ import org.apache.drill.exec.vector.ValueVector;
> >  import org.apache.drill.exec.vector.VarBinaryVector;
> >  import org.junit.Test;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  public class TestVectorCache extends ExecTest{
> >
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> > index d3f6da5..2666233 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> > @@ -17,7 +17,7 @@
> >   */
> >  package org.apache.drill.exec.cache;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  import org.apache.drill.common.config.DrillConfig;
> >  import org.apache.drill.common.expression.ExpressionPosition;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> > index 62f9a21..7d04340 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> > @@ -31,7 +31,7 @@ import
> org.apache.drill.exec.record.selection.SelectionVector2;
> >  import org.apache.drill.exec.record.selection.SelectionVector4;
> >  import org.apache.drill.exec.vector.ValueVector;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >
> >  @Deprecated
> >  public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> > index 5ca7e72..4c1ce93 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> > @@ -56,7 +56,7 @@ import org.junit.AfterClass;
> >  import org.junit.Ignore;
> >  import org.junit.Test;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import com.google.common.base.Charsets;
> >  import com.google.common.io.Files;
> >  import com.codahale.metrics.MetricRegistry;
> >
> >
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> > ----------------------------------------------------------------------
> > diff --git
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> > index 14d6fe2..3fb04aa 100644
> > ---
> a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> > +++
> b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> > @@ -20,7 +20,7 @@ package
> org.apache.drill.exec.physical.impl.mergereceiver;
> >
> >  import java.util.List;
> >
> > -import com.beust.jcommander.internal.Lists;
> > +import com.google.common.collect.Lists;
> >  import org.apache.drill.common.util.FileUtils;
> >  import org.apache.drill.exec.client.DrillClient;
> >  import org.apache.drill.exec.pop.PopUnitTestBase;
> >
>

Re: [21/24] git commit: Remove references to jcommander's copy of Guava's Lists class.

Posted by Timothy Chen <tn...@gmail.com>.
May we also remove all the Hive12.Lists reference as well?

+import com.google.hive12.common.collect.Lists;

I see these cropping up everywhere now.

Tim

On Wed, May 21, 2014 at 6:14 PM,  <ja...@apache.org> wrote:
> Remove references to jcommander's copy of Guava's Lists class.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/864ebde0
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/864ebde0
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/864ebde0
>
> Branch: refs/heads/diagnostics2
> Commit: 864ebde0384030d5d553251dc879ea3c2b98164c
> Parents: fbdca8f
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Wed May 21 13:34:40 2014 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Wed May 21 13:43:33 2014 -0700
>
> ----------------------------------------------------------------------
>  .../src/main/java/org/apache/drill/exec/client/DumpCat.java      | 2 +-
>  .../java/org/apache/drill/exec/compile/ClassTransformer.java     | 2 +-
>  .../drill/exec/compile/sig/ConstantExpressionIdentifier.java     | 2 +-
>  .../java/org/apache/drill/exec/compile/sig/SignatureHolder.java  | 2 +-
>  .../src/main/java/org/apache/drill/exec/expr/ClassGenerator.java | 2 +-
>  .../src/main/java/org/apache/drill/exec/ops/FragmentContext.java | 2 +-
>  .../src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java  | 2 +-
>  .../java/org/apache/drill/exec/physical/config/HashJoinPOP.java  | 4 ++--
>  .../java/org/apache/drill/exec/physical/config/MergeJoinPOP.java | 4 ++--
>  .../drill/exec/physical/config/OrderedPartitionSender.java       | 2 +-
>  .../apache/drill/exec/physical/impl/limit/LimitRecordBatch.java  | 2 +-
>  .../exec/physical/impl/xsort/PriorityQueueCopierTemplate.java    | 2 +-
>  .../org/apache/drill/exec/planner/logical/DrillJoinRule.java     | 2 +-
>  .../java/org/apache/drill/exec/planner/physical/HashAggPrel.java | 2 +-
>  .../org/apache/drill/exec/planner/physical/HashJoinPrel.java     | 2 +-
>  .../org/apache/drill/exec/planner/physical/MergeJoinPrel.java    | 2 +-
>  .../java/org/apache/drill/exec/planner/physical/PrelUtil.java    | 2 +-
>  .../org/apache/drill/exec/planner/physical/StreamAggPrel.java    | 2 +-
>  .../apache/drill/exec/planner/sql/parser/DrillParserUtil.java    | 2 +-
>  .../main/java/org/apache/drill/exec/record/VectorContainer.java  | 2 +-
>  .../java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java | 2 +-
>  .../java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java   | 2 +-
>  .../org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java   | 2 +-
>  .../java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java | 2 +-
>  .../org/apache/drill/exec/store/hive/HiveScanBatchCreator.java   | 2 +-
>  .../main/java/org/apache/drill/exec/store/hive/HiveTable.java    | 2 +-
>  .../org/apache/drill/exec/store/schedule/AffinityCreator.java    | 2 +-
>  .../org/apache/drill/exec/store/schedule/BlockMapBuilder.java    | 2 +-
>  .../src/main/java/org/apache/drill/exec/util/BatchPrinter.java   | 2 +-
>  .../src/main/java/org/apache/drill/exec/util/VectorUtil.java     | 2 +-
>  .../test/java/org/apache/drill/exec/cache/TestVectorCache.java   | 2 +-
>  .../test/java/org/apache/drill/exec/cache/TestWriteToDisk.java   | 2 +-
>  .../java/org/apache/drill/exec/physical/impl/SimpleRootExec.java | 2 +-
>  .../org/apache/drill/exec/physical/impl/join/TestMergeJoin.java  | 2 +-
>  .../exec/physical/impl/mergereceiver/TestMergingReceiver.java    | 2 +-
>  35 files changed, 37 insertions(+), 37 deletions(-)
> ----------------------------------------------------------------------
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> index 7e8a4a2..2d65e91 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
> @@ -36,7 +36,7 @@ import com.beust.jcommander.IParameterValidator;
>  import com.beust.jcommander.JCommander;
>  import com.beust.jcommander.Parameter;
>  import com.beust.jcommander.ParameterException;
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class DumpCat {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> index 09fdfa9..3dfc4ea 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
> @@ -30,7 +30,7 @@ import org.codehaus.commons.compiler.CompileException;
>  import org.objectweb.asm.ClassReader;
>  import org.objectweb.asm.tree.ClassNode;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.beust.jcommander.internal.Maps;
>  import com.beust.jcommander.internal.Sets;
>  import com.google.common.base.Preconditions;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> index c0c8484..8a1efdf 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
> @@ -45,7 +45,7 @@ import org.apache.drill.common.expression.ValueExpressions.Decimal38Expression;
>  import org.apache.drill.common.expression.ValueExpressions.QuotedString;
>  import org.apache.drill.common.expression.visitors.ExprVisitor;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, IdentityHashMap<LogicalExpression, Object>, RuntimeException>{
>    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConstantExpressionIdentifier.class);
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> index 1eb6732..158570a 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
> @@ -24,7 +24,7 @@ import java.util.Iterator;
>  import java.util.List;
>  import java.util.Map;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.beust.jcommander.internal.Maps;
>  import com.google.common.collect.ImmutableMap;
>  import com.google.common.collect.Iterators;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> index fc48552..51ef003 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
> @@ -37,7 +37,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.expr.fn.DrillFuncHolder.WorkspaceReference;
>  import org.apache.drill.exec.record.TypedFieldId;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.beust.jcommander.internal.Maps;
>  import com.google.common.base.Preconditions;
>  import com.sun.codemodel.JArray;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> index 3f9f181..7812d11 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
> @@ -48,7 +48,7 @@ import org.apache.drill.exec.server.options.OptionList;
>  import org.apache.drill.exec.server.options.OptionManager;
>  import org.apache.drill.exec.work.batch.IncomingBuffers;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.beust.jcommander.internal.Maps;
>
>  /**
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> index 78c0e3c..24d9cfe 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
> @@ -60,7 +60,7 @@ import org.apache.drill.exec.store.StoragePlugin;
>  import org.eigenbase.rel.RelFieldCollation.Direction;
>  import org.eigenbase.rel.RelFieldCollation.NullDirection;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class BasicOptimizer extends Optimizer{
>    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicOptimizer.class);
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> index 1ef7e97..06c23f3 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
> @@ -32,7 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
>  import org.apache.drill.exec.physical.base.PhysicalVisitor;
>  import org.apache.drill.exec.physical.base.Size;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonProperty;
>  import com.google.common.base.Preconditions;
> @@ -111,7 +111,7 @@ public class HashJoinPOP extends AbstractBase {
>
>      public HashJoinPOP flipIfRight(){
>          if(joinType == JoinRelType.RIGHT){
> -            List<JoinCondition> flippedConditions = Lists.newArrayList(conditions.size());
> +            List<JoinCondition> flippedConditions = Lists.newArrayList();
>              for(JoinCondition c : conditions){
>                  flippedConditions.add(c.flip());
>              }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> index be9cf95..b79a07e 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
> @@ -30,7 +30,7 @@ import org.apache.drill.exec.physical.base.Size;
>  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
>  import org.eigenbase.rel.JoinRelType;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonProperty;
>  import com.fasterxml.jackson.annotation.JsonTypeName;
> @@ -106,7 +106,7 @@ public class MergeJoinPOP extends AbstractBase{
>
>    public MergeJoinPOP flipIfRight(){
>      if(joinType == JoinRelType.RIGHT){
> -      List<JoinCondition> flippedConditions = Lists.newArrayList(conditions.size());
> +      List<JoinCondition> flippedConditions = Lists.newArrayList();
>        for(JoinCondition c : conditions){
>          flippedConditions.add(c.flip());
>        }
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> index 83076a4..d73926e 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
> @@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.base.Size;
>  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>  import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonProperty;
>  import com.fasterxml.jackson.annotation.JsonTypeName;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> index ed56e79..648fd89 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.physical.impl.limit;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.base.Objects;
>  import org.apache.drill.exec.exception.SchemaChangeException;
>  import org.apache.drill.exec.memory.OutOfMemoryException;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> index 4221ae2..686cc48 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.physical.impl.xsort;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.base.Stopwatch;
>  import org.apache.drill.common.expression.ExpressionPosition;
>  import org.apache.drill.common.expression.SchemaPath;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> index 96854a7..e940b2e 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
> @@ -28,7 +28,7 @@ import org.eigenbase.rex.RexUtil;
>  import org.eigenbase.sql.fun.SqlStdOperatorTable;
>  import org.eigenbase.trace.EigenbaseTrace;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  import java.util.ArrayList;
>  import java.util.List;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> index b33805c..b06c9e9 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
> @@ -48,7 +48,7 @@ import org.eigenbase.relopt.RelOptCost;
>  import org.eigenbase.relopt.RelOptPlanner;
>  import org.eigenbase.relopt.RelTraitSet;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class HashAggPrel extends AggregateRelBase implements Prel{
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> index b03ef62..9ac0834 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
> @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
>  import org.eigenbase.rex.RexNode;
>  import org.eigenbase.util.Pair;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class HashJoinPrel  extends JoinPrel {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> index 1987e99..ee9480d 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
> @@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
>  import org.eigenbase.rex.RexNode;
>  import org.eigenbase.util.Pair;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class MergeJoinPrel  extends JoinPrel {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> index 1cd480c..1de2db3 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
> @@ -49,7 +49,7 @@ import org.eigenbase.rex.RexLiteral;
>  import org.eigenbase.rex.RexNode;
>  import org.eigenbase.rex.RexVisitorImpl;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.collect.ImmutableList;
>  import com.google.common.collect.Sets;
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> index b35d1bb..f7cda32 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
> @@ -49,7 +49,7 @@ import org.eigenbase.relopt.RelOptCost;
>  import org.eigenbase.relopt.RelOptPlanner;
>  import org.eigenbase.relopt.RelTraitSet;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class StreamAggPrel extends AggregateRelBase implements Prel{
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> index 4b427be..ed36903 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
> @@ -18,7 +18,7 @@
>
>  package org.apache.drill.exec.planner.sql.parser;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import org.eigenbase.sql.SqlNode;
>  import org.eigenbase.sql.SqlOperator;
>  import org.eigenbase.sql.parser.SqlParserPos;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> index 1c7714e..dd62c67 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
> @@ -27,7 +27,7 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
>  import org.apache.drill.exec.vector.ValueVector;
>  import org.apache.drill.exec.vector.complex.AbstractMapVector;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.base.Preconditions;
>
>  public class VectorContainer extends AbstractMapVector implements Iterable<VectorWrapper<?>>, VectorAccessible {
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> index 232ec07..4ac79fd 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
> @@ -29,7 +29,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>  import org.apache.hadoop.fs.FSDataInputStream;
>  import org.apache.hadoop.fs.FileStatus;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JsonIgnore;
>  import com.google.common.collect.ImmutableList;
>  import com.google.common.collect.Range;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> index 39d71e9..a26ea1f 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
> @@ -39,7 +39,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>  import org.apache.drill.exec.store.dfs.shim.FileSystemCreator;
>  import org.apache.hadoop.conf.Configuration;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.beust.jcommander.internal.Maps;
>
>  import org.apache.hadoop.fs.FileSystem;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> index 718da23..e702c9c 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
> @@ -49,7 +49,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
>  import org.apache.hadoop.conf.Configuration;
>  import org.apache.hadoop.io.compress.CompressionCodecFactory;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.collect.ImmutableSet;
>
>  public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements FormatPlugin {
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> index 03e2095..d0cd8cc 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
> @@ -40,7 +40,7 @@ import org.apache.drill.exec.store.schedule.BlockMapBuilder;
>  import org.apache.drill.exec.store.schedule.CompleteFileWork;
>  import org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JacksonInject;
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonIgnore;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> index 3dc9ac4..a0837bc 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.store.hive;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.impl.BatchCreator;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> index 3858804..df2e428 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.store.hive;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.fasterxml.jackson.annotation.JsonCreator;
>  import com.fasterxml.jackson.annotation.JsonIgnore;
>  import com.fasterxml.jackson.annotation.JsonProperty;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> index d25abad..56d0485 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
> @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
>  import org.apache.drill.exec.physical.EndpointAffinity;
>  import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
>  import com.carrotsearch.hppc.cursors.ObjectFloatCursor;
>  import com.carrotsearch.hppc.cursors.ObjectLongCursor;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> index 1ad134e..f27e8e6 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
> @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
>  import org.apache.hadoop.fs.FileSystem;
>  import org.apache.hadoop.fs.Path;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.codahale.metrics.MetricRegistry;
>  import com.codahale.metrics.Timer;
>  import com.google.common.base.Stopwatch;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> index e5b8ad4..6222514 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.util;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  import org.apache.commons.lang.StringUtils;
>  import org.apache.drill.common.expression.ExpressionStringBuilder;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> index a43225e..a19be27 100644
> --- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
> @@ -26,7 +26,7 @@ import org.apache.drill.exec.record.VectorAccessible;
>  import org.apache.drill.exec.record.VectorWrapper;
>  import org.apache.drill.exec.rpc.RpcException;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class VectorUtil {
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> index 49f41e3..3e0be69 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
> @@ -43,7 +43,7 @@ import org.apache.drill.exec.vector.ValueVector;
>  import org.apache.drill.exec.vector.VarBinaryVector;
>  import org.junit.Test;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  public class TestVectorCache extends ExecTest{
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> index d3f6da5..2666233 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
> @@ -17,7 +17,7 @@
>   */
>  package org.apache.drill.exec.cache;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  import org.apache.drill.common.config.DrillConfig;
>  import org.apache.drill.common.expression.ExpressionPosition;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> index 62f9a21..7d04340 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
> @@ -31,7 +31,7 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
>  import org.apache.drill.exec.record.selection.SelectionVector4;
>  import org.apache.drill.exec.vector.ValueVector;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>
>  @Deprecated
>  public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> index 5ca7e72..4c1ce93 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
> @@ -56,7 +56,7 @@ import org.junit.AfterClass;
>  import org.junit.Ignore;
>  import org.junit.Test;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import com.google.common.base.Charsets;
>  import com.google.common.io.Files;
>  import com.codahale.metrics.MetricRegistry;
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> ----------------------------------------------------------------------
> diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> index 14d6fe2..3fb04aa 100644
> --- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
> @@ -20,7 +20,7 @@ package org.apache.drill.exec.physical.impl.mergereceiver;
>
>  import java.util.List;
>
> -import com.beust.jcommander.internal.Lists;
> +import com.google.common.collect.Lists;
>  import org.apache.drill.common.util.FileUtils;
>  import org.apache.drill.exec.client.DrillClient;
>  import org.apache.drill.exec.pop.PopUnitTestBase;
>

[21/24] git commit: Remove references to jcommander's copy of Guava's Lists class.

Posted by ja...@apache.org.
Remove references to jcommander's copy of Guava's Lists class.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/864ebde0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/864ebde0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/864ebde0

Branch: refs/heads/diagnostics2
Commit: 864ebde0384030d5d553251dc879ea3c2b98164c
Parents: fbdca8f
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 13:34:40 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 13:43:33 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/drill/exec/client/DumpCat.java      | 2 +-
 .../java/org/apache/drill/exec/compile/ClassTransformer.java     | 2 +-
 .../drill/exec/compile/sig/ConstantExpressionIdentifier.java     | 2 +-
 .../java/org/apache/drill/exec/compile/sig/SignatureHolder.java  | 2 +-
 .../src/main/java/org/apache/drill/exec/expr/ClassGenerator.java | 2 +-
 .../src/main/java/org/apache/drill/exec/ops/FragmentContext.java | 2 +-
 .../src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java  | 2 +-
 .../java/org/apache/drill/exec/physical/config/HashJoinPOP.java  | 4 ++--
 .../java/org/apache/drill/exec/physical/config/MergeJoinPOP.java | 4 ++--
 .../drill/exec/physical/config/OrderedPartitionSender.java       | 2 +-
 .../apache/drill/exec/physical/impl/limit/LimitRecordBatch.java  | 2 +-
 .../exec/physical/impl/xsort/PriorityQueueCopierTemplate.java    | 2 +-
 .../org/apache/drill/exec/planner/logical/DrillJoinRule.java     | 2 +-
 .../java/org/apache/drill/exec/planner/physical/HashAggPrel.java | 2 +-
 .../org/apache/drill/exec/planner/physical/HashJoinPrel.java     | 2 +-
 .../org/apache/drill/exec/planner/physical/MergeJoinPrel.java    | 2 +-
 .../java/org/apache/drill/exec/planner/physical/PrelUtil.java    | 2 +-
 .../org/apache/drill/exec/planner/physical/StreamAggPrel.java    | 2 +-
 .../apache/drill/exec/planner/sql/parser/DrillParserUtil.java    | 2 +-
 .../main/java/org/apache/drill/exec/record/VectorContainer.java  | 2 +-
 .../java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java | 2 +-
 .../java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java   | 2 +-
 .../org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java   | 2 +-
 .../java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java | 2 +-
 .../org/apache/drill/exec/store/hive/HiveScanBatchCreator.java   | 2 +-
 .../main/java/org/apache/drill/exec/store/hive/HiveTable.java    | 2 +-
 .../org/apache/drill/exec/store/schedule/AffinityCreator.java    | 2 +-
 .../org/apache/drill/exec/store/schedule/BlockMapBuilder.java    | 2 +-
 .../src/main/java/org/apache/drill/exec/util/BatchPrinter.java   | 2 +-
 .../src/main/java/org/apache/drill/exec/util/VectorUtil.java     | 2 +-
 .../test/java/org/apache/drill/exec/cache/TestVectorCache.java   | 2 +-
 .../test/java/org/apache/drill/exec/cache/TestWriteToDisk.java   | 2 +-
 .../java/org/apache/drill/exec/physical/impl/SimpleRootExec.java | 2 +-
 .../org/apache/drill/exec/physical/impl/join/TestMergeJoin.java  | 2 +-
 .../exec/physical/impl/mergereceiver/TestMergingReceiver.java    | 2 +-
 35 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
index 7e8a4a2..2d65e91 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
@@ -36,7 +36,7 @@ import com.beust.jcommander.IParameterValidator;
 import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.ParameterException;
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class DumpCat {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
index 09fdfa9..3dfc4ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/ClassTransformer.java
@@ -30,7 +30,7 @@ import org.codehaus.commons.compiler.CompileException;
 import org.objectweb.asm.ClassReader;
 import org.objectweb.asm.tree.ClassNode;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.beust.jcommander.internal.Maps;
 import com.beust.jcommander.internal.Sets;
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
index c0c8484..8a1efdf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -45,7 +45,7 @@ import org.apache.drill.common.expression.ValueExpressions.Decimal38Expression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, IdentityHashMap<LogicalExpression, Object>, RuntimeException>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConstantExpressionIdentifier.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
index 1eb6732..158570a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.beust.jcommander.internal.Maps;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
index fc48552..51ef003 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java
@@ -37,7 +37,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.DrillFuncHolder.WorkspaceReference;
 import org.apache.drill.exec.record.TypedFieldId;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.beust.jcommander.internal.Maps;
 import com.google.common.base.Preconditions;
 import com.sun.codemodel.JArray;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 3f9f181..7812d11 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -48,7 +48,7 @@ import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.beust.jcommander.internal.Maps;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 78c0e3c..24d9cfe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -60,7 +60,7 @@ import org.apache.drill.exec.store.StoragePlugin;
 import org.eigenbase.rel.RelFieldCollation.Direction;
 import org.eigenbase.rel.RelFieldCollation.NullDirection;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class BasicOptimizer extends Optimizer{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicOptimizer.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
index 1ef7e97..06c23f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
@@ -32,7 +32,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
@@ -111,7 +111,7 @@ public class HashJoinPOP extends AbstractBase {
 
     public HashJoinPOP flipIfRight(){
         if(joinType == JoinRelType.RIGHT){
-            List<JoinCondition> flippedConditions = Lists.newArrayList(conditions.size());
+            List<JoinCondition> flippedConditions = Lists.newArrayList();
             for(JoinCondition c : conditions){
                 flippedConditions.add(c.flip());
             }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
index be9cf95..b79a07e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.eigenbase.rel.JoinRelType;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
@@ -106,7 +106,7 @@ public class MergeJoinPOP extends AbstractBase{
 
   public MergeJoinPOP flipIfRight(){
     if(joinType == JoinRelType.RIGHT){
-      List<JoinCondition> flippedConditions = Lists.newArrayList(conditions.size());
+      List<JoinCondition> flippedConditions = Lists.newArrayList();
       for(JoinCondition c : conditions){
         flippedConditions.add(c.flip());
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
index 83076a4..d73926e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
index ed56e79..648fd89 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.limit;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.base.Objects;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.OutOfMemoryException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
index 4221ae2..686cc48 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.xsort;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.base.Stopwatch;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index 96854a7..e940b2e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
@@ -28,7 +28,7 @@ import org.eigenbase.rex.RexUtil;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.trace.EigenbaseTrace;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
index b33805c..b06c9e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashAggPrel.java
@@ -48,7 +48,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class HashAggPrel extends AggregateRelBase implements Prel{
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
index b03ef62..9ac0834 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashJoinPrel.java
@@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.util.Pair;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class HashJoinPrel  extends JoinPrel {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index 1987e99..ee9480d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -40,7 +40,7 @@ import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.util.Pair;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class MergeJoinPrel  extends JoinPrel {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index 1cd480c..1de2db3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -49,7 +49,7 @@ import org.eigenbase.rex.RexLiteral;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.rex.RexVisitorImpl;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Sets;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index b35d1bb..f7cda32 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -49,7 +49,7 @@ import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class StreamAggPrel extends AggregateRelBase implements Prel{
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
index 4b427be..ed36903 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillParserUtil.java
@@ -18,7 +18,7 @@
 
 package org.apache.drill.exec.planner.sql.parser;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.SqlOperator;
 import org.eigenbase.sql.parser.SqlParserPos;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 1c7714e..dd62c67 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractMapVector;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.base.Preconditions;
 
 public class VectorContainer extends AbstractMapVector implements Iterable<VectorWrapper<?>>, VectorAccessible {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
index 232ec07..4ac79fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/BasicFormatMatcher.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Range;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index 39d71e9..a26ea1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -39,7 +39,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.drill.exec.store.dfs.shim.FileSystemCreator;
 import org.apache.hadoop.conf.Configuration;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.beust.jcommander.internal.Maps;
 
 import org.apache.hadoop.fs.FileSystem;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 718da23..e702c9c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -49,7 +49,7 @@ import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CompressionCodecFactory;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.collect.ImmutableSet;
 
 public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements FormatPlugin {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index 03e2095..d0cd8cc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -40,7 +40,7 @@ import org.apache.drill.exec.store.schedule.BlockMapBuilder;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.drill.exec.store.schedule.CompleteFileWork.FileWorkImpl;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
index 3dc9ac4..a0837bc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveScanBatchCreator.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.store.hive;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.BatchCreator;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
index 3858804..df2e428 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveTable.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.store.hive;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
index d25abad..56d0485 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/AffinityCreator.java
@@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.carrotsearch.hppc.ObjectFloatOpenHashMap;
 import com.carrotsearch.hppc.cursors.ObjectFloatCursor;
 import com.carrotsearch.hppc.cursors.ObjectLongCursor;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
index 1ad134e..f27e8e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/schedule/BlockMapBuilder.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
 import com.google.common.base.Stopwatch;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
index e5b8ad4..6222514 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.util;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.drill.common.expression.ExpressionStringBuilder;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
index a43225e..a19be27 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
@@ -26,7 +26,7 @@ import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.RpcException;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class VectorUtil {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
index 49f41e3..3e0be69 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
@@ -43,7 +43,7 @@ import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarBinaryVector;
 import org.junit.Test;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 public class TestVectorCache extends ExecTest{
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
index d3f6da5..2666233 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestWriteToDisk.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.cache;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index 62f9a21..7d04340 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -31,7 +31,7 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 
 @Deprecated
 public class SimpleRootExec implements RootExec, Iterable<ValueVector>{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 5ca7e72..4c1ce93 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -56,7 +56,7 @@ import org.junit.AfterClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import com.codahale.metrics.MetricRegistry;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/864ebde0/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index 14d6fe2..3fb04aa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.physical.impl.mergereceiver;
 
 import java.util.List;
 
-import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.Lists;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.pop.PopUnitTestBase;


[17/24] git commit: Fixing build issues.

Posted by ja...@apache.org.
Fixing build issues.


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

Branch: refs/heads/diagnostics2
Commit: b8cef6b6e0099bc146ddcccb492664b7238dd9b2
Parents: cb90852
Author: Aditya Kishore <ad...@maprtech.com>
Authored: Tue May 20 11:49:10 2014 -0700
Committer: Aditya Kishore <ad...@maprtech.com>
Committed: Tue May 20 11:49:10 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ops/MetricDef.java     | 17 +++++++++++++++++
 .../org/apache/drill/exec/ops/OpProfileDef.java  | 17 +++++++++++++++++
 .../partitionsender/PartitionSenderStats.java    | 17 +++++++++++++++++
 .../drill/exec/server/rest/DrillRestServer.java  | 18 +++++++++++++++++-
 .../apache/drill/exec/server/rest/DrillRoot.java | 19 ++++++++++++++++++-
 .../java/org/apache/drill/exec/util/Pointer.java | 17 +++++++++++++++++
 .../drill/exec/work/foreman/FragmentData.java    | 17 +++++++++++++++++
 .../drill/exec/work/foreman/QueryStatus.java     | 17 +++++++++++++++++
 .../src/main/resources/rest/status/list.ftl      | 11 +++++++++++
 .../src/main/resources/rest/status/profile.ftl   | 11 +++++++++++
 .../drill/exec/server/DrillClientFactory.java    | 17 +++++++++++++++++
 .../apache/drill/exec/server/HelloResource.java  | 17 +++++++++++++++++
 .../org/apache/drill/exec/server/TestJersey.java | 17 +++++++++++++++++
 .../drill/exec/server/rest/RootResource.java     | 17 +++++++++++++++++
 14 files changed, 227 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
index e08a2b2..b3ece41 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.ops;
 
 public interface MetricDef {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
index 61f6d20..fb68e4a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.ops;
 
 public class OpProfileDef {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
index 99b9120..4790596 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.physical.impl.partitionsender;
 
 import org.apache.drill.exec.ops.MetricDef;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
index cfc3819..c0e0f19 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
@@ -1,7 +1,23 @@
+/**
+ * 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.drill.exec.server.rest;
 
 import org.apache.drill.exec.client.DrillClient;
-import org.apache.drill.exec.server.HelloResource;
 import org.apache.drill.exec.work.WorkManager;
 import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.jackson.JacksonFeature;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
index 27e0d17..996825d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.server.rest;
 
 import java.util.List;
@@ -16,7 +33,7 @@ import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
 import org.apache.drill.exec.work.WorkManager;
 import org.glassfish.jersey.server.mvc.Viewable;
 
-import com.google.hive12.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 @Path("/")
 public class DrillRoot {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
index 8139943..074ed76 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.util;
 
 public class Pointer<T> {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
index 3bbe692..d9dd33e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.work.foreman;
 
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
index 991e9ac..8cc2cf1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.work.foreman;
 
 import org.apache.drill.exec.cache.DistributedCache;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/resources/rest/status/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/list.ftl b/exec/java-exec/src/main/resources/rest/status/list.ftl
index 6bde590..ab1ca30 100644
--- a/exec/java-exec/src/main/resources/rest/status/list.ftl
+++ b/exec/java-exec/src/main/resources/rest/status/list.ftl
@@ -1,3 +1,14 @@
+<#-- 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. -->
+
 <html>
 Welcome to Drill! 
 <br />

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/main/resources/rest/status/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/profile.ftl b/exec/java-exec/src/main/resources/rest/status/profile.ftl
index 94ff6ef..4989ae9 100644
--- a/exec/java-exec/src/main/resources/rest/status/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/status/profile.ftl
@@ -1,3 +1,14 @@
+<#-- 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. -->
+
 <html>
 <a href="/queries">back</a><br />
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
index 216cb3b..4230518 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/DrillClientFactory.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.server;
 
 import org.apache.drill.exec.client.DrillClient;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
index cdb04ec..bfd4651 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/HelloResource.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.server;
 
 import javax.inject.Inject;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
index 6e873ed..d47e559 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.server;
 
 import org.apache.drill.exec.client.DrillClient;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/b8cef6b6/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
index 42fc39e..59adad9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
@@ -1,3 +1,20 @@
+/**
+ * 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.drill.exec.server.rest;
 
 import javax.ws.rs.Path;


[06/24] status changes

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 0631f6a..26c74a1 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -116,6 +116,440 @@ public final class UserBitShared {
     // @@protoc_insertion_point(enum_scope:exec.shared.RpcChannel)
   }
 
+  /**
+   * Protobuf enum {@code exec.shared.QueryType}
+   */
+  public enum QueryType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SQL = 1;</code>
+     */
+    SQL(0, 1),
+    /**
+     * <code>LOGICAL = 2;</code>
+     */
+    LOGICAL(1, 2),
+    /**
+     * <code>PHYSICAL = 3;</code>
+     */
+    PHYSICAL(2, 3),
+    ;
+
+    /**
+     * <code>SQL = 1;</code>
+     */
+    public static final int SQL_VALUE = 1;
+    /**
+     * <code>LOGICAL = 2;</code>
+     */
+    public static final int LOGICAL_VALUE = 2;
+    /**
+     * <code>PHYSICAL = 3;</code>
+     */
+    public static final int PHYSICAL_VALUE = 3;
+
+
+    public final int getNumber() { return value; }
+
+    public static QueryType valueOf(int value) {
+      switch (value) {
+        case 1: return SQL;
+        case 2: return LOGICAL;
+        case 3: return PHYSICAL;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<QueryType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<QueryType>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<QueryType>() {
+            public QueryType findValueByNumber(int number) {
+              return QueryType.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(1);
+    }
+
+    private static final QueryType[] VALUES = values();
+
+    public static QueryType valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private QueryType(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:exec.shared.QueryType)
+  }
+
+  /**
+   * Protobuf enum {@code exec.shared.CoreOperatorType}
+   */
+  public enum CoreOperatorType
+      implements com.google.protobuf.ProtocolMessageEnum {
+    /**
+     * <code>SINGLE_SENDER = 0;</code>
+     */
+    SINGLE_SENDER(0, 0),
+    /**
+     * <code>BROADCAST_SENDER = 1;</code>
+     */
+    BROADCAST_SENDER(1, 1),
+    /**
+     * <code>FILTER = 2;</code>
+     */
+    FILTER(2, 2),
+    /**
+     * <code>HASH_AGGREGATE = 3;</code>
+     */
+    HASH_AGGREGATE(3, 3),
+    /**
+     * <code>HASH_JOIN = 4;</code>
+     */
+    HASH_JOIN(4, 4),
+    /**
+     * <code>MERGE_JOIN = 5;</code>
+     */
+    MERGE_JOIN(5, 5),
+    /**
+     * <code>HASH_PARTITION_SENDER = 6;</code>
+     */
+    HASH_PARTITION_SENDER(6, 6),
+    /**
+     * <code>LIMIT = 7;</code>
+     */
+    LIMIT(7, 7),
+    /**
+     * <code>MERGING_RECEIVER = 8;</code>
+     */
+    MERGING_RECEIVER(8, 8),
+    /**
+     * <code>ORDERED_PARTITION_SENDER = 9;</code>
+     */
+    ORDERED_PARTITION_SENDER(9, 9),
+    /**
+     * <code>PROJECT = 10;</code>
+     */
+    PROJECT(10, 10),
+    /**
+     * <code>RANDOM_RECEIVER = 11;</code>
+     */
+    RANDOM_RECEIVER(11, 11),
+    /**
+     * <code>RANGE_SENDER = 12;</code>
+     */
+    RANGE_SENDER(12, 12),
+    /**
+     * <code>SCREEN = 13;</code>
+     */
+    SCREEN(13, 13),
+    /**
+     * <code>SELECTION_VECTOR_REMOVER = 14;</code>
+     */
+    SELECTION_VECTOR_REMOVER(14, 14),
+    /**
+     * <code>STREAMING_AGGREGATE = 15;</code>
+     */
+    STREAMING_AGGREGATE(15, 15),
+    /**
+     * <code>TOP_N_SORT = 16;</code>
+     */
+    TOP_N_SORT(16, 16),
+    /**
+     * <code>EXTERNAL_SORT = 17;</code>
+     */
+    EXTERNAL_SORT(17, 17),
+    /**
+     * <code>TRACE = 18;</code>
+     */
+    TRACE(18, 18),
+    /**
+     * <code>UNION = 19;</code>
+     */
+    UNION(19, 19),
+    /**
+     * <code>OLD_SORT = 20;</code>
+     */
+    OLD_SORT(20, 20),
+    /**
+     * <code>PARQUET_ROW_GROUP_SCAN = 21;</code>
+     */
+    PARQUET_ROW_GROUP_SCAN(21, 21),
+    /**
+     * <code>HIVE_SUB_SCAN = 22;</code>
+     */
+    HIVE_SUB_SCAN(22, 22),
+    /**
+     * <code>SYSTEM_TABLE_SCAN = 23;</code>
+     */
+    SYSTEM_TABLE_SCAN(23, 23),
+    /**
+     * <code>MOCK_SUB_SCAN = 24;</code>
+     */
+    MOCK_SUB_SCAN(24, 24),
+    /**
+     * <code>PARQUET_WRITER = 25;</code>
+     */
+    PARQUET_WRITER(25, 25),
+    /**
+     * <code>DIRECT_SUB_SCAN = 26;</code>
+     */
+    DIRECT_SUB_SCAN(26, 26),
+    /**
+     * <code>TEXT_WRITER = 27;</code>
+     */
+    TEXT_WRITER(27, 27),
+    /**
+     * <code>TEXT_SUB_SCAN = 28;</code>
+     */
+    TEXT_SUB_SCAN(28, 28),
+    /**
+     * <code>JSON_SUB_SCAN = 29;</code>
+     */
+    JSON_SUB_SCAN(29, 29),
+    /**
+     * <code>INFO_SCHEMA_SUB_SCAN = 30;</code>
+     */
+    INFO_SCHEMA_SUB_SCAN(30, 30),
+    ;
+
+    /**
+     * <code>SINGLE_SENDER = 0;</code>
+     */
+    public static final int SINGLE_SENDER_VALUE = 0;
+    /**
+     * <code>BROADCAST_SENDER = 1;</code>
+     */
+    public static final int BROADCAST_SENDER_VALUE = 1;
+    /**
+     * <code>FILTER = 2;</code>
+     */
+    public static final int FILTER_VALUE = 2;
+    /**
+     * <code>HASH_AGGREGATE = 3;</code>
+     */
+    public static final int HASH_AGGREGATE_VALUE = 3;
+    /**
+     * <code>HASH_JOIN = 4;</code>
+     */
+    public static final int HASH_JOIN_VALUE = 4;
+    /**
+     * <code>MERGE_JOIN = 5;</code>
+     */
+    public static final int MERGE_JOIN_VALUE = 5;
+    /**
+     * <code>HASH_PARTITION_SENDER = 6;</code>
+     */
+    public static final int HASH_PARTITION_SENDER_VALUE = 6;
+    /**
+     * <code>LIMIT = 7;</code>
+     */
+    public static final int LIMIT_VALUE = 7;
+    /**
+     * <code>MERGING_RECEIVER = 8;</code>
+     */
+    public static final int MERGING_RECEIVER_VALUE = 8;
+    /**
+     * <code>ORDERED_PARTITION_SENDER = 9;</code>
+     */
+    public static final int ORDERED_PARTITION_SENDER_VALUE = 9;
+    /**
+     * <code>PROJECT = 10;</code>
+     */
+    public static final int PROJECT_VALUE = 10;
+    /**
+     * <code>RANDOM_RECEIVER = 11;</code>
+     */
+    public static final int RANDOM_RECEIVER_VALUE = 11;
+    /**
+     * <code>RANGE_SENDER = 12;</code>
+     */
+    public static final int RANGE_SENDER_VALUE = 12;
+    /**
+     * <code>SCREEN = 13;</code>
+     */
+    public static final int SCREEN_VALUE = 13;
+    /**
+     * <code>SELECTION_VECTOR_REMOVER = 14;</code>
+     */
+    public static final int SELECTION_VECTOR_REMOVER_VALUE = 14;
+    /**
+     * <code>STREAMING_AGGREGATE = 15;</code>
+     */
+    public static final int STREAMING_AGGREGATE_VALUE = 15;
+    /**
+     * <code>TOP_N_SORT = 16;</code>
+     */
+    public static final int TOP_N_SORT_VALUE = 16;
+    /**
+     * <code>EXTERNAL_SORT = 17;</code>
+     */
+    public static final int EXTERNAL_SORT_VALUE = 17;
+    /**
+     * <code>TRACE = 18;</code>
+     */
+    public static final int TRACE_VALUE = 18;
+    /**
+     * <code>UNION = 19;</code>
+     */
+    public static final int UNION_VALUE = 19;
+    /**
+     * <code>OLD_SORT = 20;</code>
+     */
+    public static final int OLD_SORT_VALUE = 20;
+    /**
+     * <code>PARQUET_ROW_GROUP_SCAN = 21;</code>
+     */
+    public static final int PARQUET_ROW_GROUP_SCAN_VALUE = 21;
+    /**
+     * <code>HIVE_SUB_SCAN = 22;</code>
+     */
+    public static final int HIVE_SUB_SCAN_VALUE = 22;
+    /**
+     * <code>SYSTEM_TABLE_SCAN = 23;</code>
+     */
+    public static final int SYSTEM_TABLE_SCAN_VALUE = 23;
+    /**
+     * <code>MOCK_SUB_SCAN = 24;</code>
+     */
+    public static final int MOCK_SUB_SCAN_VALUE = 24;
+    /**
+     * <code>PARQUET_WRITER = 25;</code>
+     */
+    public static final int PARQUET_WRITER_VALUE = 25;
+    /**
+     * <code>DIRECT_SUB_SCAN = 26;</code>
+     */
+    public static final int DIRECT_SUB_SCAN_VALUE = 26;
+    /**
+     * <code>TEXT_WRITER = 27;</code>
+     */
+    public static final int TEXT_WRITER_VALUE = 27;
+    /**
+     * <code>TEXT_SUB_SCAN = 28;</code>
+     */
+    public static final int TEXT_SUB_SCAN_VALUE = 28;
+    /**
+     * <code>JSON_SUB_SCAN = 29;</code>
+     */
+    public static final int JSON_SUB_SCAN_VALUE = 29;
+    /**
+     * <code>INFO_SCHEMA_SUB_SCAN = 30;</code>
+     */
+    public static final int INFO_SCHEMA_SUB_SCAN_VALUE = 30;
+
+
+    public final int getNumber() { return value; }
+
+    public static CoreOperatorType valueOf(int value) {
+      switch (value) {
+        case 0: return SINGLE_SENDER;
+        case 1: return BROADCAST_SENDER;
+        case 2: return FILTER;
+        case 3: return HASH_AGGREGATE;
+        case 4: return HASH_JOIN;
+        case 5: return MERGE_JOIN;
+        case 6: return HASH_PARTITION_SENDER;
+        case 7: return LIMIT;
+        case 8: return MERGING_RECEIVER;
+        case 9: return ORDERED_PARTITION_SENDER;
+        case 10: return PROJECT;
+        case 11: return RANDOM_RECEIVER;
+        case 12: return RANGE_SENDER;
+        case 13: return SCREEN;
+        case 14: return SELECTION_VECTOR_REMOVER;
+        case 15: return STREAMING_AGGREGATE;
+        case 16: return TOP_N_SORT;
+        case 17: return EXTERNAL_SORT;
+        case 18: return TRACE;
+        case 19: return UNION;
+        case 20: return OLD_SORT;
+        case 21: return PARQUET_ROW_GROUP_SCAN;
+        case 22: return HIVE_SUB_SCAN;
+        case 23: return SYSTEM_TABLE_SCAN;
+        case 24: return MOCK_SUB_SCAN;
+        case 25: return PARQUET_WRITER;
+        case 26: return DIRECT_SUB_SCAN;
+        case 27: return TEXT_WRITER;
+        case 28: return TEXT_SUB_SCAN;
+        case 29: return JSON_SUB_SCAN;
+        case 30: return INFO_SCHEMA_SUB_SCAN;
+        default: return null;
+      }
+    }
+
+    public static com.google.protobuf.Internal.EnumLiteMap<CoreOperatorType>
+        internalGetValueMap() {
+      return internalValueMap;
+    }
+    private static com.google.protobuf.Internal.EnumLiteMap<CoreOperatorType>
+        internalValueMap =
+          new com.google.protobuf.Internal.EnumLiteMap<CoreOperatorType>() {
+            public CoreOperatorType findValueByNumber(int number) {
+              return CoreOperatorType.valueOf(number);
+            }
+          };
+
+    public final com.google.protobuf.Descriptors.EnumValueDescriptor
+        getValueDescriptor() {
+      return getDescriptor().getValues().get(index);
+    }
+    public final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+    public static final com.google.protobuf.Descriptors.EnumDescriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.getDescriptor().getEnumTypes().get(2);
+    }
+
+    private static final CoreOperatorType[] VALUES = values();
+
+    public static CoreOperatorType valueOf(
+        com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
+      if (desc.getType() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "EnumValueDescriptor is not for this type.");
+      }
+      return VALUES[desc.getIndex()];
+    }
+
+    private final int index;
+    private final int value;
+
+    private CoreOperatorType(int index, int value) {
+      this.index = index;
+      this.value = value;
+    }
+
+    // @@protoc_insertion_point(enum_scope:exec.shared.CoreOperatorType)
+  }
+
   public interface UserCredentialsOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -6388,24 +6822,5731 @@ public final class UserBitShared {
     // @@protoc_insertion_point(class_scope:exec.shared.SerializedField)
   }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_exec_shared_UserCredentials_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_exec_shared_UserCredentials_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_exec_shared_QueryId_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_exec_shared_QueryId_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_exec_shared_DrillPBError_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_exec_shared_DrillPBError_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_exec_shared_ParsingError_descriptor;
-  private static
+  public interface QueryProfileOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional .exec.shared.QueryId id = 1;
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    boolean hasId();
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.QueryId getId();
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getIdOrBuilder();
+
+    // optional .exec.shared.QueryType type = 2;
+    /**
+     * <code>optional .exec.shared.QueryType type = 2;</code>
+     */
+    boolean hasType();
+    /**
+     * <code>optional .exec.shared.QueryType type = 2;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.QueryType getType();
+
+    // optional int64 start = 3;
+    /**
+     * <code>optional int64 start = 3;</code>
+     */
+    boolean hasStart();
+    /**
+     * <code>optional int64 start = 3;</code>
+     */
+    long getStart();
+
+    // optional int64 end = 4;
+    /**
+     * <code>optional int64 end = 4;</code>
+     */
+    boolean hasEnd();
+    /**
+     * <code>optional int64 end = 4;</code>
+     */
+    long getEnd();
+
+    // optional string query = 5;
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    boolean hasQuery();
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    java.lang.String getQuery();
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    com.google.protobuf.ByteString
+        getQueryBytes();
+
+    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> 
+        getFragmentProfileList();
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index);
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    int getFragmentProfileCount();
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
+        getFragmentProfileOrBuilderList();
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code exec.shared.QueryProfile}
+   */
+  public static final class QueryProfile extends
+      com.google.protobuf.GeneratedMessage
+      implements QueryProfileOrBuilder {
+    // Use QueryProfile.newBuilder() to construct.
+    private QueryProfile(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private QueryProfile(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final QueryProfile defaultInstance;
+    public static QueryProfile getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public QueryProfile getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private QueryProfile(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              org.apache.drill.exec.proto.UserBitShared.QueryId.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000001) == 0x00000001)) {
+                subBuilder = id_.toBuilder();
+              }
+              id_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.QueryId.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(id_);
+                id_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000001;
+              break;
+            }
+            case 16: {
+              int rawValue = input.readEnum();
+              org.apache.drill.exec.proto.UserBitShared.QueryType value = org.apache.drill.exec.proto.UserBitShared.QueryType.valueOf(rawValue);
+              if (value == null) {
+                unknownFields.mergeVarintField(2, rawValue);
+              } else {
+                bitField0_ |= 0x00000002;
+                type_ = value;
+              }
+              break;
+            }
+            case 24: {
+              bitField0_ |= 0x00000004;
+              start_ = input.readInt64();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              end_ = input.readInt64();
+              break;
+            }
+            case 42: {
+              bitField0_ |= 0x00000010;
+              query_ = input.readBytes();
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                fragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile>();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              fragmentProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          fragmentProfile_ = java.util.Collections.unmodifiableList(fragmentProfile_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryProfile_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryProfile_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.drill.exec.proto.UserBitShared.QueryProfile.class, org.apache.drill.exec.proto.UserBitShared.QueryProfile.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<QueryProfile> PARSER =
+        new com.google.protobuf.AbstractParser<QueryProfile>() {
+      public QueryProfile parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new QueryProfile(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<QueryProfile> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional .exec.shared.QueryId id = 1;
+    public static final int ID_FIELD_NUMBER = 1;
+    private org.apache.drill.exec.proto.UserBitShared.QueryId id_;
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    public boolean hasId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.QueryId getId() {
+      return id_;
+    }
+    /**
+     * <code>optional .exec.shared.QueryId id = 1;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getIdOrBuilder() {
+      return id_;
+    }
+
+    // optional .exec.shared.QueryType type = 2;
+    public static final int TYPE_FIELD_NUMBER = 2;
+    private org.apache.drill.exec.proto.UserBitShared.QueryType type_;
+    /**
+     * <code>optional .exec.shared.QueryType type = 2;</code>
+     */
+    public boolean hasType() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .exec.shared.QueryType type = 2;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.QueryType getType() {
+      return type_;
+    }
+
+    // optional int64 start = 3;
+    public static final int START_FIELD_NUMBER = 3;
+    private long start_;
+    /**
+     * <code>optional int64 start = 3;</code>
+     */
+    public boolean hasStart() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int64 start = 3;</code>
+     */
+    public long getStart() {
+      return start_;
+    }
+
+    // optional int64 end = 4;
+    public static final int END_FIELD_NUMBER = 4;
+    private long end_;
+    /**
+     * <code>optional int64 end = 4;</code>
+     */
+    public boolean hasEnd() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional int64 end = 4;</code>
+     */
+    public long getEnd() {
+      return end_;
+    }
+
+    // optional string query = 5;
+    public static final int QUERY_FIELD_NUMBER = 5;
+    private java.lang.Object query_;
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    public boolean hasQuery() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    public java.lang.String getQuery() {
+      java.lang.Object ref = query_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          query_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string query = 5;</code>
+     */
+    public com.google.protobuf.ByteString
+        getQueryBytes() {
+      java.lang.Object ref = query_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        query_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
+    public static final int FRAGMENT_PROFILE_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> fragmentProfile_;
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> getFragmentProfileList() {
+      return fragmentProfile_;
+    }
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
+        getFragmentProfileOrBuilderList() {
+      return fragmentProfile_;
+    }
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    public int getFragmentProfileCount() {
+      return fragmentProfile_.size();
+    }
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index) {
+      return fragmentProfile_.get(index);
+    }
+    /**
+     * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
+        int index) {
+      return fragmentProfile_.get(index);
+    }
+
+    private void initFields() {
+      id_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+      type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
+      start_ = 0L;
+      end_ = 0L;
+      query_ = "";
+      fragmentProfile_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeMessage(1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeEnum(2, type_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt64(3, start_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt64(4, end_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(5, getQueryBytes());
+      }
+      for (int i = 0; i < fragmentProfile_.size(); i++) {
+        output.writeMessage(6, fragmentProfile_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(1, id_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeEnumSize(2, type_.getNumber());
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(3, start_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, end_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(5, getQueryBytes());
+      }
+      for (int i = 0; i < fragmentProfile_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(6, fragmentProfile_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.QueryProfile parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.drill.exec.proto.UserBitShared.QueryProfile prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code exec.shared.QueryProfile}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.drill.exec.proto.UserBitShared.QueryProfileOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryProfile_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryProfile_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.drill.exec.proto.UserBitShared.QueryProfile.class, org.apache.drill.exec.proto.UserBitShared.QueryProfile.Builder.class);
+      }
+
+      // Construct using org.apache.drill.exec.proto.UserBitShared.QueryProfile.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getIdFieldBuilder();
+          getFragmentProfileFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        if (idBuilder_ == null) {
+          id_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+        } else {
+          idBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        start_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        end_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        query_ = "";
+        bitField0_ = (bitField0_ & ~0x00000010);
+        if (fragmentProfileBuilder_ == null) {
+          fragmentProfile_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000020);
+        } else {
+          fragmentProfileBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_QueryProfile_descriptor;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryProfile getDefaultInstanceForType() {
+        return org.apache.drill.exec.proto.UserBitShared.QueryProfile.getDefaultInstance();
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryProfile build() {
+        org.apache.drill.exec.proto.UserBitShared.QueryProfile result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.QueryProfile buildPartial() {
+        org.apache.drill.exec.proto.UserBitShared.QueryProfile result = new org.apache.drill.exec.proto.UserBitShared.QueryProfile(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        if (idBuilder_ == null) {
+          result.id_ = id_;
+        } else {
+          result.id_ = idBuilder_.build();
+        }
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.type_ = type_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.start_ = start_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.end_ = end_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.query_ = query_;
+        if (fragmentProfileBuilder_ == null) {
+          if (((bitField0_ & 0x00000020) == 0x00000020)) {
+            fragmentProfile_ = java.util.Collections.unmodifiableList(fragmentProfile_);
+            bitField0_ = (bitField0_ & ~0x00000020);
+          }
+          result.fragmentProfile_ = fragmentProfile_;
+        } else {
+          result.fragmentProfile_ = fragmentProfileBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.drill.exec.proto.UserBitShared.QueryProfile) {
+          return mergeFrom((org.apache.drill.exec.proto.UserBitShared.QueryProfile)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.QueryProfile other) {
+        if (other == org.apache.drill.exec.proto.UserBitShared.QueryProfile.getDefaultInstance()) return this;
+        if (other.hasId()) {
+          mergeId(other.getId());
+        }
+        if (other.hasType()) {
+          setType(other.getType());
+        }
+        if (other.hasStart()) {
+          setStart(other.getStart());
+        }
+        if (other.hasEnd()) {
+          setEnd(other.getEnd());
+        }
+        if (other.hasQuery()) {
+          bitField0_ |= 0x00000010;
+          query_ = other.query_;
+          onChanged();
+        }
+        if (fragmentProfileBuilder_ == null) {
+          if (!other.fragmentProfile_.isEmpty()) {
+            if (fragmentProfile_.isEmpty()) {
+              fragmentProfile_ = other.fragmentProfile_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+            } else {
+              ensureFragmentProfileIsMutable();
+              fragmentProfile_.addAll(other.fragmentProfile_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.fragmentProfile_.isEmpty()) {
+            if (fragmentProfileBuilder_.isEmpty()) {
+              fragmentProfileBuilder_.dispose();
+              fragmentProfileBuilder_ = null;
+              fragmentProfile_ = other.fragmentProfile_;
+              bitField0_ = (bitField0_ & ~0x00000020);
+              fragmentProfileBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getFragmentProfileFieldBuilder() : null;
+            } else {
+              fragmentProfileBuilder_.addAllMessages(other.fragmentProfile_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.drill.exec.proto.UserBitShared.QueryProfile parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.drill.exec.proto.UserBitShared.QueryProfile) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional .exec.shared.QueryId id = 1;
+      private org.apache.drill.exec.proto.UserBitShared.QueryId id_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder> idBuilder_;
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public boolean hasId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.QueryId getId() {
+        if (idBuilder_ == null) {
+          return id_;
+        } else {
+          return idBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public Builder setId(org.apache.drill.exec.proto.UserBitShared.QueryId value) {
+        if (idBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          id_ = value;
+          onChanged();
+        } else {
+          idBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public Builder setId(
+          org.apache.drill.exec.proto.UserBitShared.QueryId.Builder builderForValue) {
+        if (idBuilder_ == null) {
+          id_ = builderForValue.build();
+          onChanged();
+        } else {
+          idBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public Builder mergeId(org.apache.drill.exec.proto.UserBitShared.QueryId value) {
+        if (idBuilder_ == null) {
+          if (((bitField0_ & 0x00000001) == 0x00000001) &&
+              id_ != org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance()) {
+            id_ =
+              org.apache.drill.exec.proto.UserBitShared.QueryId.newBuilder(id_).mergeFrom(value).buildPartial();
+          } else {
+            id_ = value;
+          }
+          onChanged();
+        } else {
+          idBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000001;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public Builder clearId() {
+        if (idBuilder_ == null) {
+          id_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
+          onChanged();
+        } else {
+          idBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.QueryId.Builder getIdBuilder() {
+        bitField0_ |= 0x00000001;
+        onChanged();
+        return getIdFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder getIdOrBuilder() {
+        if (idBuilder_ != null) {
+          return idBuilder_.getMessageOrBuilder();
+        } else {
+          return id_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.QueryId id = 1;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder> 
+          getIdFieldBuilder() {
+        if (idBuilder_ == null) {
+          idBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.QueryId, org.apache.drill.exec.proto.UserBitShared.QueryId.Builder, org.apache.drill.exec.proto.UserBitShared.QueryIdOrBuilder>(
+                  id_,
+                  getParentForChildren(),
+                  isClean());
+          id_ = null;
+        }
+        return idBuilder_;
+      }
+
+      // optional .exec.shared.QueryType type = 2;
+      private org.apache.drill.exec.proto.UserBitShared.QueryType type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
+      /**
+       * <code>optional .exec.shared.QueryType type = 2;</code>
+       */
+      public boolean hasType() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional .exec.shared.QueryType type = 2;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.QueryType getType() {
+        return type_;
+      }
+      /**
+       * <code>optional .exec.shared.QueryType type = 2;</code>
+       */
+      public Builder setType(org.apache.drill.exec.proto.UserBitShared.QueryType value) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+        bitField0_ |= 0x00000002;
+        type_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.QueryType type = 2;</code>
+       */
+      public Builder clearType() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        type_ = org.apache.drill.exec.proto.UserBitShared.QueryType.SQL;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 start = 3;
+      private long start_ ;
+      /**
+       * <code>optional int64 start = 3;</code>
+       */
+      public boolean hasStart() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional int64 start = 3;</code>
+       */
+      public long getStart() {
+        return start_;
+      }
+      /**
+       * <code>optional int64 start = 3;</code>
+       */
+      public Builder setStart(long value) {
+        bitField0_ |= 0x00000004;
+        start_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 start = 3;</code>
+       */
+      public Builder clearStart() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        start_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional int64 end = 4;
+      private long end_ ;
+      /**
+       * <code>optional int64 end = 4;</code>
+       */
+      public boolean hasEnd() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int64 end = 4;</code>
+       */
+      public long getEnd() {
+        return end_;
+      }
+      /**
+       * <code>optional int64 end = 4;</code>
+       */
+      public Builder setEnd(long value) {
+        bitField0_ |= 0x00000008;
+        end_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 end = 4;</code>
+       */
+      public Builder clearEnd() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        end_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // optional string query = 5;
+      private java.lang.Object query_ = "";
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public boolean hasQuery() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public java.lang.String getQuery() {
+        java.lang.Object ref = query_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          query_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public com.google.protobuf.ByteString
+          getQueryBytes() {
+        java.lang.Object ref = query_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          query_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public Builder setQuery(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        query_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public Builder clearQuery() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        query_ = getDefaultInstance().getQuery();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string query = 5;</code>
+       */
+      public Builder setQueryBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000010;
+        query_ = value;
+        onChanged();
+        return this;
+      }
+
+      // repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;
+      private java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> fragmentProfile_ =
+        java.util.Collections.emptyList();
+      private void ensureFragmentProfileIsMutable() {
+        if (!((bitField0_ & 0x00000020) == 0x00000020)) {
+          fragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile>(fragmentProfile_);
+          bitField0_ |= 0x00000020;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> fragmentProfileBuilder_;
+
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> getFragmentProfileList() {
+        if (fragmentProfileBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(fragmentProfile_);
+        } else {
+          return fragmentProfileBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public int getFragmentProfileCount() {
+        if (fragmentProfileBuilder_ == null) {
+          return fragmentProfile_.size();
+        } else {
+          return fragmentProfileBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getFragmentProfile(int index) {
+        if (fragmentProfileBuilder_ == null) {
+          return fragmentProfile_.get(index);
+        } else {
+          return fragmentProfileBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder setFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
+        if (fragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.set(index, value);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder setFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
+        if (fragmentProfileBuilder_ == null) {
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder addFragmentProfile(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
+        if (fragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.add(value);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder addFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile value) {
+        if (fragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.add(index, value);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder addFragmentProfile(
+          org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
+        if (fragmentProfileBuilder_ == null) {
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.add(builderForValue.build());
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder addFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder builderForValue) {
+        if (fragmentProfileBuilder_ == null) {
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder addAllFragmentProfile(
+          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile> values) {
+        if (fragmentProfileBuilder_ == null) {
+          ensureFragmentProfileIsMutable();
+          super.addAll(values, fragmentProfile_);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder clearFragmentProfile() {
+        if (fragmentProfileBuilder_ == null) {
+          fragmentProfile_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000020);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public Builder removeFragmentProfile(int index) {
+        if (fragmentProfileBuilder_ == null) {
+          ensureFragmentProfileIsMutable();
+          fragmentProfile_.remove(index);
+          onChanged();
+        } else {
+          fragmentProfileBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder getFragmentProfileBuilder(
+          int index) {
+        return getFragmentProfileFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder getFragmentProfileOrBuilder(
+          int index) {
+        if (fragmentProfileBuilder_ == null) {
+          return fragmentProfile_.get(index);  } else {
+          return fragmentProfileBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
+           getFragmentProfileOrBuilderList() {
+        if (fragmentProfileBuilder_ != null) {
+          return fragmentProfileBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(fragmentProfile_);
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder addFragmentProfileBuilder() {
+        return getFragmentProfileFieldBuilder().addBuilder(
+            org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder addFragmentProfileBuilder(
+          int index) {
+        return getFragmentProfileFieldBuilder().addBuilder(
+            index, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .exec.shared.MajorFragmentProfile fragment_profile = 6;</code>
+       */
+      public java.util.List<org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder> 
+           getFragmentProfileBuilderList() {
+        return getFragmentProfileFieldBuilder().getBuilderList();
+      }
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder> 
+          getFragmentProfileFieldBuilder() {
+        if (fragmentProfileBuilder_ == null) {
+          fragmentProfileBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder>(
+                  fragmentProfile_,
+                  ((bitField0_ & 0x00000020) == 0x00000020),
+                  getParentForChildren(),
+                  isClean());
+          fragmentProfile_ = null;
+        }
+        return fragmentProfileBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:exec.shared.QueryProfile)
+    }
+
+    static {
+      defaultInstance = new QueryProfile(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:exec.shared.QueryProfile)
+  }
+
+  public interface MajorFragmentProfileOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional int32 major_fragment_id = 1;
+    /**
+     * <code>optional int32 major_fragment_id = 1;</code>
+     */
+    boolean hasMajorFragmentId();
+    /**
+     * <code>optional int32 major_fragment_id = 1;</code>
+     */
+    int getMajorFragmentId();
+
+    // repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    java.util.List<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> 
+        getMinorFragmentProfileList();
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getMinorFragmentProfile(int index);
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    int getMinorFragmentProfileCount();
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> 
+        getMinorFragmentProfileOrBuilderList();
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getMinorFragmentProfileOrBuilder(
+        int index);
+  }
+  /**
+   * Protobuf type {@code exec.shared.MajorFragmentProfile}
+   */
+  public static final class MajorFragmentProfile extends
+      com.google.protobuf.GeneratedMessage
+      implements MajorFragmentProfileOrBuilder {
+    // Use MajorFragmentProfile.newBuilder() to construct.
+    private MajorFragmentProfile(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private MajorFragmentProfile(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final MajorFragmentProfile defaultInstance;
+    public static MajorFragmentProfile getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public MajorFragmentProfile getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private MajorFragmentProfile(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              majorFragmentId_ = input.readInt32();
+              break;
+            }
+            case 18: {
+              if (!((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+                minorFragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile>();
+                mutable_bitField0_ |= 0x00000002;
+              }
+              minorFragmentProfile_.add(input.readMessage(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.PARSER, extensionRegistry));
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000002) == 0x00000002)) {
+          minorFragmentProfile_ = java.util.Collections.unmodifiableList(minorFragmentProfile_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_MajorFragmentProfile_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_MajorFragmentProfile_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.class, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<MajorFragmentProfile> PARSER =
+        new com.google.protobuf.AbstractParser<MajorFragmentProfile>() {
+      public MajorFragmentProfile parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new MajorFragmentProfile(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<MajorFragmentProfile> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional int32 major_fragment_id = 1;
+    public static final int MAJOR_FRAGMENT_ID_FIELD_NUMBER = 1;
+    private int majorFragmentId_;
+    /**
+     * <code>optional int32 major_fragment_id = 1;</code>
+     */
+    public boolean hasMajorFragmentId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional int32 major_fragment_id = 1;</code>
+     */
+    public int getMajorFragmentId() {
+      return majorFragmentId_;
+    }
+
+    // repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;
+    public static final int MINOR_FRAGMENT_PROFILE_FIELD_NUMBER = 2;
+    private java.util.List<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> minorFragmentProfile_;
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    public java.util.List<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> getMinorFragmentProfileList() {
+      return minorFragmentProfile_;
+    }
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    public java.util.List<? extends org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> 
+        getMinorFragmentProfileOrBuilderList() {
+      return minorFragmentProfile_;
+    }
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    public int getMinorFragmentProfileCount() {
+      return minorFragmentProfile_.size();
+    }
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getMinorFragmentProfile(int index) {
+      return minorFragmentProfile_.get(index);
+    }
+    /**
+     * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder getMinorFragmentProfileOrBuilder(
+        int index) {
+      return minorFragmentProfile_.get(index);
+    }
+
+    private void initFields() {
+      majorFragmentId_ = 0;
+      minorFragmentProfile_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeInt32(1, majorFragmentId_);
+      }
+      for (int i = 0; i < minorFragmentProfile_.size(); i++) {
+        output.writeMessage(2, minorFragmentProfile_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(1, majorFragmentId_);
+      }
+      for (int i = 0; i < minorFragmentProfile_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(2, minorFragmentProfile_.get(i));
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code exec.shared.MajorFragmentProfile}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfileOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_MajorFragmentProfile_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_MajorFragmentProfile_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.class, org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.Builder.class);
+      }
+
+      // Construct using org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getMinorFragmentProfileFieldBuilder();
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        majorFragmentId_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        if (minorFragmentProfileBuilder_ == null) {
+          minorFragmentProfile_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+        } else {
+          minorFragmentProfileBuilder_.clear();
+        }
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_MajorFragmentProfile_descriptor;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile getDefaultInstanceForType() {
+        return org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance();
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile build() {
+        org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile buildPartial() {
+        org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile result = new org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.majorFragmentId_ = majorFragmentId_;
+        if (minorFragmentProfileBuilder_ == null) {
+          if (((bitField0_ & 0x00000002) == 0x00000002)) {
+            minorFragmentProfile_ = java.util.Collections.unmodifiableList(minorFragmentProfile_);
+            bitField0_ = (bitField0_ & ~0x00000002);
+          }
+          result.minorFragmentProfile_ = minorFragmentProfile_;
+        } else {
+          result.minorFragmentProfile_ = minorFragmentProfileBuilder_.build();
+        }
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile) {
+          return mergeFrom((org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile other) {
+        if (other == org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile.getDefaultInstance()) return this;
+        if (other.hasMajorFragmentId()) {
+          setMajorFragmentId(other.getMajorFragmentId());
+        }
+        if (minorFragmentProfileBuilder_ == null) {
+          if (!other.minorFragmentProfile_.isEmpty()) {
+            if (minorFragmentProfile_.isEmpty()) {
+              minorFragmentProfile_ = other.minorFragmentProfile_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+            } else {
+              ensureMinorFragmentProfileIsMutable();
+              minorFragmentProfile_.addAll(other.minorFragmentProfile_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.minorFragmentProfile_.isEmpty()) {
+            if (minorFragmentProfileBuilder_.isEmpty()) {
+              minorFragmentProfileBuilder_.dispose();
+              minorFragmentProfileBuilder_ = null;
+              minorFragmentProfile_ = other.minorFragmentProfile_;
+              bitField0_ = (bitField0_ & ~0x00000002);
+              minorFragmentProfileBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getMinorFragmentProfileFieldBuilder() : null;
+            } else {
+              minorFragmentProfileBuilder_.addAllMessages(other.minorFragmentProfile_);
+            }
+          }
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional int32 major_fragment_id = 1;
+      private int majorFragmentId_ ;
+      /**
+       * <code>optional int32 major_fragment_id = 1;</code>
+       */
+      public boolean hasMajorFragmentId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional int32 major_fragment_id = 1;</code>
+       */
+      public int getMajorFragmentId() {
+        return majorFragmentId_;
+      }
+      /**
+       * <code>optional int32 major_fragment_id = 1;</code>
+       */
+      public Builder setMajorFragmentId(int value) {
+        bitField0_ |= 0x00000001;
+        majorFragmentId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 major_fragment_id = 1;</code>
+       */
+      public Builder clearMajorFragmentId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        majorFragmentId_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;
+      private java.util.List<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> minorFragmentProfile_ =
+        java.util.Collections.emptyList();
+      private void ensureMinorFragmentProfileIsMutable() {
+        if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+          minorFragmentProfile_ = new java.util.ArrayList<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile>(minorFragmentProfile_);
+          bitField0_ |= 0x00000002;
+         }
+      }
+
+      private com.google.protobuf.RepeatedFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfileOrBuilder> minorFragmentProfileBuilder_;
+
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public java.util.List<org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> getMinorFragmentProfileList() {
+        if (minorFragmentProfileBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(minorFragmentProfile_);
+        } else {
+          return minorFragmentProfileBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public int getMinorFragmentProfileCount() {
+        if (minorFragmentProfileBuilder_ == null) {
+          return minorFragmentProfile_.size();
+        } else {
+          return minorFragmentProfileBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile getMinorFragmentProfile(int index) {
+        if (minorFragmentProfileBuilder_ == null) {
+          return minorFragmentProfile_.get(index);
+        } else {
+          return minorFragmentProfileBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder setMinorFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (minorFragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.set(index, value);
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder setMinorFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
+        if (minorFragmentProfileBuilder_ == null) {
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder addMinorFragmentProfile(org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (minorFragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.add(value);
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder addMinorFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile value) {
+        if (minorFragmentProfileBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.add(index, value);
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder addMinorFragmentProfile(
+          org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
+        if (minorFragmentProfileBuilder_ == null) {
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.add(builderForValue.build());
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder addMinorFragmentProfile(
+          int index, org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile.Builder builderForValue) {
+        if (minorFragmentProfileBuilder_ == null) {
+          ensureMinorFragmentProfileIsMutable();
+          minorFragmentProfile_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder addAllMinorFragmentProfile(
+          java.lang.Iterable<? extends org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile> values) {
+        if (minorFragmentProfileBuilder_ == null) {
+          ensureMinorFragmentProfileIsMutable();
+          super.addAll(values, minorFragmentProfile_);
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .exec.shared.MinorFragmentProfile minor_fragment_profile = 2;</code>
+       */
+      public Builder clearMinorFragmentProfile() {
+        if (minorFragmentProfileBuilder_ == null) {
+          minorFragmentProfile_ = java.util.Collections.emptyList();
+          bitField0_ = (bitField0_ & ~0x00000002);
+          onChanged();
+        } else {
+          minorFragmentProfileBuilder_.clear();
+        }
+     

<TRUNCATED>

[16/24] git commit: diag wip

Posted by ja...@apache.org.
diag wip


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

Branch: refs/heads/diagnostics2
Commit: cb90852a1c202ab5bbf64dd17b6ba367d6e7561c
Parents: e6121d0
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon May 19 16:39:36 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 16:39:36 2014 -0700

----------------------------------------------------------------------
 .../exec/cache/ProtobufDrillSerializable.java   |   73 +
 .../exec/cache/SerializationDefinition.java     |    4 +-
 .../drill/exec/cache/infinispan/ICache.java     |   19 +-
 .../infinispan/JacksonAdvancedExternalizer.java |   70 +
 .../ProtobufAdvancedExternalizer.java           |   67 +
 .../apache/drill/exec/ops/FragmentStats.java    |   36 +-
 .../apache/drill/exec/ops/OperatorContext.java  |    6 +-
 .../apache/drill/exec/ops/OperatorStats.java    |    8 +-
 .../drill/exec/physical/impl/ScreenCreator.java |   38 +-
 .../drill/exec/physical/impl/TraceInjector.java |    1 +
 .../partitionsender/OutgoingRecordBatch.java    |   12 +-
 .../PartitionSenderRootExec.java                |   13 +-
 .../partitionsender/PartitionSenderStats.java   |   15 +
 .../validate/IteratorValidatorInjector.java     |   15 +-
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   19 +-
 .../planner/sql/handlers/DefaultSqlHandler.java |   10 +-
 .../org/apache/drill/exec/rpc/BasicServer.java  |    4 +-
 .../drill/exec/server/rest/DrillRestServer.java |    1 +
 .../drill/exec/server/rest/DrillRoot.java       |   53 +-
 .../drill/exec/util/JsonStringArrayList.java    |    2 +-
 .../drill/exec/util/JsonStringHashMap.java      |    2 +-
 .../org/apache/drill/exec/util/Pointer.java     |   11 +
 .../org/apache/drill/exec/work/WorkManager.java |   52 +-
 .../apache/drill/exec/work/foreman/Foreman.java |   22 +-
 .../drill/exec/work/foreman/FragmentData.java   |   17 +-
 .../drill/exec/work/foreman/QueryManager.java   |   33 +-
 .../drill/exec/work/foreman/QueryStatus.java    |  130 +-
 .../work/fragment/AbstractStatusReporter.java   |   37 +-
 .../exec/work/fragment/FragmentExecutor.java    |   39 +-
 .../work/fragment/NonRootFragmentManager.java   |   13 +-
 .../exec/work/fragment/StatusReporter.java      |    2 +-
 .../src/main/resources/rest/status/list.ftl     |    9 +
 .../src/main/resources/rest/status/profile.ftl  |   19 +
 .../org/apache/drill/exec/proto/BitControl.java | 1263 ++-----------
 .../apache/drill/exec/proto/UserBitShared.java  | 1682 +++++++++++++-----
 protocol/src/main/protobuf/BitControl.proto     |   23 +-
 protocol/src/main/protobuf/UserBitShared.proto  |   25 +-
 37 files changed, 2048 insertions(+), 1797 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtobufDrillSerializable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtobufDrillSerializable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtobufDrillSerializable.java
new file mode 100644
index 0000000..9d6b645
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtobufDrillSerializable.java
@@ -0,0 +1,73 @@
+/**
+ * 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.drill.exec.cache;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+
+public abstract class ProtobufDrillSerializable<T extends Message> extends LoopedAbstractDrillSerializable implements DrillSerializable{
+  private Parser<T> parser;
+  private T obj;
+
+  public ProtobufDrillSerializable(T obj){
+    this.parser = (Parser<T>) obj.getParserForType();
+    this.obj = obj;
+  }
+
+  public ProtobufDrillSerializable(Parser<T> parser) {
+    this.parser = parser;
+  }
+
+  @Override
+  public void readFromStream(InputStream input) throws IOException {
+    obj = parser.parseDelimitedFrom(input);
+  }
+
+  @Override
+  public void writeToStream(OutputStream output) throws IOException {
+    obj.writeDelimitedTo(output);
+  }
+
+  public T getObj() {
+    return obj;
+  }
+
+  public static class CQueryProfile extends ProtobufDrillSerializable<QueryProfile>{
+
+    public CQueryProfile(BufferAllocator allocator){
+      super(QueryProfile.PARSER);
+    }
+
+    public CQueryProfile() {
+      super(QueryProfile.PARSER);
+
+    }
+
+    public CQueryProfile(QueryProfile obj) {
+      super(obj);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/SerializationDefinition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/SerializationDefinition.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/SerializationDefinition.java
index 95ba434..711ddf1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/SerializationDefinition.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/SerializationDefinition.java
@@ -18,12 +18,14 @@
 package org.apache.drill.exec.cache;
 
 import org.apache.drill.exec.planner.logical.StoragePlugins;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.server.options.OptionValue;
 
 public enum SerializationDefinition {
 
   OPTION(3002, OptionValue.class),
-  STORAGE_PLUGINS(3003, StoragePlugins.class)
+  STORAGE_PLUGINS(3003, StoragePlugins.class),
+  FRAGMENT_STATUS(3004, FragmentStatus.class)
   ;
   public final int id;
   public final Class<?> clazz;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
index 5533dd7..92ce08d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ICache.java
@@ -32,10 +32,12 @@ import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.cache.DistributedMap;
 import org.apache.drill.exec.cache.DistributedMultiMap;
 import org.apache.drill.exec.cache.DrillSerializable;
+import org.apache.drill.exec.cache.SerializationDefinition;
 import org.apache.drill.exec.cache.ProtoSerializable.FragmentHandleSerializable;
 import org.apache.drill.exec.cache.ProtoSerializable.PlanFragmentSerializable;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.infinispan.Cache;
@@ -65,13 +67,18 @@ public class ICache implements DistributedCache{
 
   public ICache(DrillConfig config, BufferAllocator allocator) throws Exception {
     String clusterName = config.getString(ExecConstants.SERVICE_NAME);
-    GlobalConfigurationBuilder gcb = new GlobalConfigurationBuilder();
-    gcb.transport() //
-    .defaultTransport().clusterName(clusterName).build();
-    gcb.serialization() //
-    .addAdvancedExternalizer(new VAAdvancedExternalizer(allocator));
+    GlobalConfiguration gc = new GlobalConfigurationBuilder() //
+      .transport() //
+        .defaultTransport() //
+        .clusterName(clusterName) //;
+        //
+      .serialization() //
+        .addAdvancedExternalizer(new VAAdvancedExternalizer(allocator)) //
+        .addAdvancedExternalizer(new JacksonAdvancedExternalizer<>(SerializationDefinition.OPTION, config.getMapper())) //
+        .addAdvancedExternalizer(new JacksonAdvancedExternalizer<>(SerializationDefinition.STORAGE_PLUGINS, config.getMapper())) //
+        .addAdvancedExternalizer(new ProtobufAdvancedExternalizer<>(SerializationDefinition.FRAGMENT_STATUS, FragmentStatus.PARSER)) //
+      .build();
 
-    GlobalConfiguration gc = gcb.build();
     Configuration c = new ConfigurationBuilder() //
       .clustering() //
       .cacheMode(CacheMode.DIST_ASYNC) //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/JacksonAdvancedExternalizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/JacksonAdvancedExternalizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/JacksonAdvancedExternalizer.java
new file mode 100644
index 0000000..81f4877
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/JacksonAdvancedExternalizer.java
@@ -0,0 +1,70 @@
+/**
+ * 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.drill.exec.cache.infinispan;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.drill.common.util.DataInputInputStream;
+import org.apache.drill.exec.cache.SerializationDefinition;
+import org.infinispan.commons.marshall.AdvancedExternalizer;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.hazelcast.nio.ObjectDataInput;
+import com.hazelcast.nio.ObjectDataOutput;
+
+public class JacksonAdvancedExternalizer<T> implements AdvancedExternalizer<T>  {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JacksonAdvancedExternalizer.class);
+
+  private final Class<?> clazz;
+  private final ObjectMapper mapper;
+  private final int id;
+
+  public JacksonAdvancedExternalizer(SerializationDefinition def, ObjectMapper mapper){
+    this.clazz =  def.clazz;
+    this.mapper = mapper;
+    this.id = def.id;
+  }
+
+  @Override
+  public T readObject(ObjectInput in) throws IOException, ClassNotFoundException {
+    return (T) mapper.readValue(DataInputInputStream.constructInputStream(in), clazz);
+  }
+
+  @Override
+  public void writeObject(ObjectOutput out, T object) throws IOException {
+    out.write(mapper.writeValueAsBytes(object));
+  }
+
+  @Override
+  public Integer getId() {
+    return id;
+  }
+
+  @Override
+  public Set<Class<? extends T>> getTypeClasses() {
+    return (Set<Class<? extends T>>) (Object) Collections.singleton(clazz);
+  }
+
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ProtobufAdvancedExternalizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ProtobufAdvancedExternalizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ProtobufAdvancedExternalizer.java
new file mode 100644
index 0000000..821443a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/cache/infinispan/ProtobufAdvancedExternalizer.java
@@ -0,0 +1,67 @@
+/**
+ * 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.drill.exec.cache.infinispan;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.drill.common.util.DataInputInputStream;
+import org.apache.drill.exec.cache.SerializationDefinition;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.infinispan.commons.marshall.AdvancedExternalizer;
+
+import com.google.protobuf.Message;
+import com.google.protobuf.Parser;
+
+public class ProtobufAdvancedExternalizer<T extends Message> implements AdvancedExternalizer<T>  {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtobufAdvancedExternalizer.class);
+
+  private final Class<?> clazz;
+  private final int id;
+  private final Parser<T> parser;
+
+  public ProtobufAdvancedExternalizer(SerializationDefinition def, Parser<T> parser){
+    this.clazz =  def.clazz;
+    this.parser = parser;
+    this.id = def.id;
+  }
+
+  @Override
+  public T readObject(ObjectInput in) throws IOException, ClassNotFoundException {
+    return parser.parseDelimitedFrom(DataInputInputStream.constructInputStream(in));
+  }
+
+  @Override
+  public void writeObject(ObjectOutput out, T object) throws IOException {
+    out.write(object.toByteArray());
+  }
+
+  @Override
+  public Integer getId() {
+    return id;
+  }
+
+  @Override
+  public Set<Class<? extends T>> getTypeClasses() {
+    return (Set<Class<? extends T>>) (Object) Collections.singleton(clazz);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
index 205c3f1..24a02b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
@@ -19,66 +19,38 @@ package org.apache.drill.exec.ops;
 
 import java.util.List;
 
-import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
-import org.apache.drill.exec.work.fragment.FragmentExecutor;
 
 import com.codahale.metrics.MetricRegistry;
-import com.codahale.metrics.Timer;
 import com.google.hive12.common.collect.Lists;
 
 public class FragmentStats {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
 
-  private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentExecutor.class,
-      "completionTimes");
-  private final static String METRIC_BATCHES_COMPLETED = MetricRegistry
-      .name(FragmentExecutor.class, "batchesCompleted");
-  private final static String METRIC_RECORDS_COMPLETED = MetricRegistry
-      .name(FragmentExecutor.class, "recordsCompleted");
-  private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentExecutor.class, "dataProcessed");
-
-
-
-  private final MetricRegistry metrics;
 
   private List<OperatorStats> operators = Lists.newArrayList();
-
-  public final SingleThreadNestedCounter batchesCompleted;
-  public final SingleThreadNestedCounter recordsCompleted;
-  public final SingleThreadNestedCounter dataProcessed;
-  public final Timer fragmentTime;
   private final long startTime;
 
   public FragmentStats(MetricRegistry metrics) {
-    this.metrics = metrics;
     this.startTime = System.currentTimeMillis();
-    this.fragmentTime = metrics.timer(METRIC_TIMER_FRAGMENT_TIME);
-    this.batchesCompleted = new SingleThreadNestedCounter(metrics, METRIC_BATCHES_COMPLETED);
-    this.recordsCompleted = new SingleThreadNestedCounter(metrics, METRIC_RECORDS_COMPLETED);
-    this.dataProcessed = new SingleThreadNestedCounter(metrics, METRIC_DATA_PROCESSED);
   }
 
-  public void addMetricsToStatus(FragmentStatus.Builder stats) {
-    stats.setBatchesCompleted(batchesCompleted.get());
-    stats.setDataProcessed(dataProcessed.get());
-    stats.setRecordsCompleted(recordsCompleted.get());
+  public void addMetricsToStatus(MinorFragmentProfile.Builder prfB) {
 
-    MinorFragmentProfile.Builder prfB = MinorFragmentProfile.newBuilder();
     prfB.setStartTime(startTime);
     prfB.setEndTime(System.currentTimeMillis());
 
     for(OperatorStats o : operators){
       prfB.addOperatorProfile(o.getProfile());
     }
-
-    stats.setProfile(prfB);
   }
 
   public OperatorStats getOperatorStats(OpProfileDef profileDef){
     OperatorStats stats = new OperatorStats(profileDef);
-    operators.add(stats);
+    if(profileDef.operatorType != -1){
+      operators.add(stats);
+    }
     return stats;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 824ec6a..116b616 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -30,16 +30,14 @@ public class OperatorContext implements Closeable {
   private final BufferAllocator allocator;
   private boolean closed = false;
   private PhysicalOperator popConfig;
-  private int operatorId;
   private OperatorStats stats;
 
   public OperatorContext(PhysicalOperator popConfig, FragmentContext context) throws OutOfMemoryException {
     this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation());
     this.popConfig = popConfig;
-    this.operatorId = popConfig.getOperatorId();
 
     OpProfileDef def = new OpProfileDef();
-    def.operatorId = operatorId;
+    def.operatorId = popConfig.getOperatorId();
     def.incomingCount = getChildCount(popConfig);
     def.operatorType = popConfig.getOperatorType();
     this.stats = context.getStats().getOperatorStats(def);
@@ -52,6 +50,8 @@ public class OperatorContext implements Closeable {
       iter.next();
       i++;
     }
+
+    if(i == 0) i = 1;
     return i;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
index bd00560..cde1876 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.ops;
 
 import org.apache.commons.collections.Buffer;
+import org.apache.drill.exec.proto.UserBitShared.MetricValue;
 import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
 import org.apache.drill.exec.proto.UserBitShared.StreamProfile;
 
@@ -37,8 +38,6 @@ public class OperatorStats {
   public long[] batchesReceivedByInput;
   private long[] schemaCountByInput;
 
-  private long batchesOutput;
-  private long recordsOutput;
 
   private boolean inProcessing = false;
   private boolean inSetup = false;
@@ -103,7 +102,6 @@ public class OperatorStats {
         .newBuilder() //
         .setOperatorType(operatorType) //
         .setOperatorId(operatorId) //
-        .setOutputProfile(StreamProfile.newBuilder().setBatches(batchesOutput).setRecords(recordsOutput)) //
         .setSetupNanos(setupNanos) //
         .setProcessNanos(processingNanos);
 
@@ -113,13 +111,13 @@ public class OperatorStats {
 
     for(int i =0; i < longMetrics.allocated.length; i++){
       if(longMetrics.allocated[i]){
-        b.addMetricBuilder().setMetricId(longMetrics.keys[i]).setLongValue(longMetrics.values[i]);
+        b.addMetric(MetricValue.newBuilder().setMetricId(longMetrics.keys[i]).setLongValue(longMetrics.values[i]));
       }
     }
 
     for(int i =0; i < doubleMetrics.allocated.length; i++){
       if(doubleMetrics.allocated[i]){
-        b.addMetricBuilder().setMetricId(doubleMetrics.keys[i]).setDoubleValue(doubleMetrics.values[i]);
+        b.addMetric(MetricValue.newBuilder().setMetricId(doubleMetrics.keys[i]).setDoubleValue(doubleMetrics.values[i]));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index a0ff28a..c92633f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -40,28 +40,28 @@ import com.google.common.base.Preconditions;
 
 public class ScreenCreator implements RootCreator<Screen>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
-  
-  
-  
+
+
+
   @Override
   public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
     Preconditions.checkNotNull(children);
     Preconditions.checkArgument(children.size() == 1);
     return new ScreenRoot(context, children.iterator().next());
   }
-  
-  
+
+
   static class ScreenRoot implements RootExec{
     static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenRoot.class);
     volatile boolean ok = true;
-    
+
     private final SendingAccountor sendCount = new SendingAccountor();
-    
+
     final RecordBatch incoming;
     final FragmentContext context;
     final UserClientConnection connection;
     private RecordMaterializer materializer;
-    
+
     public ScreenRoot(FragmentContext context, RecordBatch incoming){
       assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client.  As such, this should always be true.";
 
@@ -69,7 +69,7 @@ public class ScreenCreator implements RootCreator<Screen>{
       this.incoming = incoming;
       this.connection = context.getConnection();
     }
-    
+
     @Override
     public boolean next() {
       if(!ok){
@@ -97,7 +97,7 @@ public class ScreenCreator implements RootCreator<Screen>{
       }
       case NONE: {
         sendCount.waitForSendComplete();
-        context.getStats().batchesCompleted.inc(1);
+//        context.getStats().batchesCompleted.inc(1);
         QueryResult header = QueryResult.newBuilder() //
             .setQueryId(context.getHandle().getQueryId()) //
             .setRowCount(0) //
@@ -107,19 +107,19 @@ public class ScreenCreator implements RootCreator<Screen>{
         QueryWritableBatch batch = new QueryWritableBatch(header);
         connection.sendResult(listener, batch);
         sendCount.increment();
-        
+
         return false;
       }
       case OK_NEW_SCHEMA:
         materializer = new VectorRecordMaterializer(context, incoming);
         // fall through.
       case OK:
-        context.getStats().batchesCompleted.inc(1);
-        context.getStats().recordsCompleted.inc(incoming.getRecordCount());
+//        context.getStats().batchesCompleted.inc(1);
+//        context.getStats().recordsCompleted.inc(incoming.getRecordCount());
         QueryWritableBatch batch = materializer.convertNext(false);
         connection.sendResult(listener, batch);
         sendCount.increment();
-        
+
         return true;
       default:
         throw new UnsupportedOperationException();
@@ -133,7 +133,7 @@ public class ScreenCreator implements RootCreator<Screen>{
     }
 
     private SendListener listener = new SendListener();
-    
+
     private class SendListener extends BaseRpcOutcomeListener<Ack>{
 
 
@@ -151,15 +151,15 @@ public class ScreenCreator implements RootCreator<Screen>{
         ErrorHelper.logAndConvertError(context.getIdentity(), "Failure while sending fragment to client.", ex, logger);
         ok = false;
       }
-      
+
     }
 
     RecordBatch getIncoming() {
       return incoming;
     }
-    
-    
+
+
   }
-  
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TraceInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TraceInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TraceInjector.java
index 9c859a8..dddb53f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TraceInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TraceInjector.java
@@ -84,6 +84,7 @@ public class TraceInjector extends AbstractPhysicalVisitor<PhysicalOperator, Fra
         /* Inject trace operator */
         if (list.size() > 0)
             newOp = op.getNewWithChildren(list);
+            newOp.setOperatorId(op.getOperatorId());
 
         return newOp;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index deef25f..646f1a0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.impl.SendingAccountor;
 import org.apache.drill.exec.proto.ExecProtos;
@@ -43,7 +44,6 @@ import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.data.DataTunnel;
-import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.allocator.VectorAllocator;
 import org.apache.drill.exec.work.ErrorHelper;
@@ -74,8 +74,9 @@ public class OutgoingRecordBatch implements VectorAccessible {
   private int recordCapacity;
   private static int DEFAULT_ALLOC_SIZE = 20000;
   private static int DEFAULT_VARIABLE_WIDTH_SIZE = 2048;
+  private OperatorStats stats;
 
-  public OutgoingRecordBatch(SendingAccountor sendCount, HashPartitionSender operator, DataTunnel tunnel, RecordBatch incoming,
+  public OutgoingRecordBatch(OperatorStats stats, SendingAccountor sendCount, HashPartitionSender operator, DataTunnel tunnel, RecordBatch incoming,
                              FragmentContext context, BufferAllocator allocator, int oppositeMinorFragmentId) {
     this.incoming = incoming;
     this.context = context;
@@ -83,13 +84,18 @@ public class OutgoingRecordBatch implements VectorAccessible {
     this.operator = operator;
     this.tunnel = tunnel;
     this.sendCount = sendCount;
+    this.stats = stats;
     this.oppositeMinorFragmentId = oppositeMinorFragmentId;
   }
 
   public void flushIfNecessary() {
     if (recordCount == recordCapacity) logger.debug("Flush is necesary:  Count is " + recordCount + ", capacity is " + recordCapacity);
     try {
-      if (recordCount == recordCapacity) flush();
+      if (recordCount == recordCapacity){
+        flush();
+        stats.addLongStat(PartitionSenderStats.BATCHES_SENT, 1l);
+        stats.addLongStat(PartitionSenderStats.RECORDS_SENT, recordCount);
+      }
     } catch (SchemaChangeException e) {
       incoming.kill();
       logger.error("Error flushing outgoing batches", e);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index bcd484c..d999019 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -29,17 +29,16 @@ import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.physical.config.HashPartitionSender;
 import org.apache.drill.exec.physical.impl.RootExec;
 import org.apache.drill.exec.physical.impl.SendingAccountor;
 import org.apache.drill.exec.physical.impl.filter.ReturnValueExpression;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.record.AbstractRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TypedFieldId;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -63,8 +62,8 @@ public class PartitionSenderRootExec implements RootExec {
   private FragmentContext context;
   private OperatorContext oContext;
   private boolean ok = true;
-  private AtomicLong batchesSent = new AtomicLong(0);
   private final SendingAccountor sendCount = new SendingAccountor();
+  private final OperatorStats stats;
 
 
   public PartitionSenderRootExec(FragmentContext context,
@@ -75,11 +74,12 @@ public class PartitionSenderRootExec implements RootExec {
     this.operator = operator;
     this.context = context;
     this.oContext = new OperatorContext(operator, context);
+    this.stats = oContext.getStats();
     this.outgoing = new OutgoingRecordBatch[operator.getDestinations().size()];
     int fieldId = 0;
     for (CoordinationProtos.DrillbitEndpoint endpoint : operator.getDestinations()) {
       FragmentHandle opposite = context.getHandle().toBuilder().setMajorFragmentId(operator.getOppositeMajorFragmentId()).setMinorFragmentId(fieldId).build();
-      outgoing[fieldId] = new OutgoingRecordBatch(sendCount, operator,
+      outgoing[fieldId] = new OutgoingRecordBatch(stats, sendCount, operator,
                                                     context.getDataTunnel(endpoint, opposite),
                                                     incoming,
                                                     context,
@@ -91,6 +91,7 @@ public class PartitionSenderRootExec implements RootExec {
 
   @Override
   public boolean next() {
+    boolean newSchema = false;
 
     if (!ok) {
       stop();
@@ -122,6 +123,7 @@ public class PartitionSenderRootExec implements RootExec {
         return false;
 
       case OK_NEW_SCHEMA:
+        newSchema = true;
         try {
           // send all existing batches
           if (partitioner != null) {
@@ -139,12 +141,11 @@ public class PartitionSenderRootExec implements RootExec {
           return false;
         }
       case OK:
+        stats.batchReceived(0, incoming.getRecordCount(), newSchema);
         partitioner.partitionBatch(incoming);
         for (VectorWrapper v : incoming) {
           v.clear();
         }
-        context.getStats().batchesCompleted.inc(1);
-        context.getStats().recordsCompleted.inc(incoming.getRecordCount());
         return true;
       case NOT_YET:
       default:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
new file mode 100644
index 0000000..99b9120
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderStats.java
@@ -0,0 +1,15 @@
+package org.apache.drill.exec.physical.impl.partitionsender;
+
+import org.apache.drill.exec.ops.MetricDef;
+
+public enum PartitionSenderStats implements MetricDef {
+
+  BATCHES_SENT,
+  RECORDS_SENT;
+
+  @Override
+  public int metricId() {
+    return ordinal();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
index aff71bf..428f335 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
@@ -35,16 +35,16 @@ public class IteratorValidatorInjector extends
   public static FragmentRoot rewritePlanWithIteratorValidator(FragmentContext context, FragmentRoot root) throws ExecutionSetupException {
     IteratorValidatorInjector inject = new IteratorValidatorInjector();
     PhysicalOperator newOp = root.accept(inject, context);
-    
+
     if( !(newOp instanceof FragmentRoot) ) throw new IllegalStateException("This shouldn't happen.");
 
     return (FragmentRoot) newOp;
-    
+
   }
 
   /**
    * Traverse the physical plan and inject the IteratorValidator operator after every operator.
-   * 
+   *
    * @param op
    *          Physical operator under which the IteratorValidator operator will be injected
    * @param context
@@ -61,12 +61,17 @@ public class IteratorValidatorInjector extends
 
     /* Get the list of child operators */
     for (PhysicalOperator child : op) {
-      newChildren.add(new IteratorValidator(child.accept(this, context)));
+      PhysicalOperator validator = new IteratorValidator(child.accept(this, context));
+      validator.setOperatorId(op.getOperatorId() + 1000);
+      newChildren.add(validator);
     }
 
     /* Inject trace operator */
-    if (newChildren.size() > 0)
+    if (newChildren.size() > 0){
       newOp = op.getNewWithChildren(newChildren);
+      newOp.setOperatorId(op.getOperatorId());
+    }
+
 
     return newOp;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 7477440..df66dcf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -32,7 +32,6 @@ import net.hydromatic.optiq.tools.ValidationException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.cost.DrillCostBase;
-import org.apache.drill.exec.planner.cost.DrillCostBase.DrillCostFactory;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
@@ -40,23 +39,15 @@ import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
 import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
-import org.apache.drill.exec.planner.sql.handlers.UseSchemaHandler;
-import org.apache.drill.exec.planner.sql.parser.SqlDescribeTable;
-import org.apache.drill.exec.planner.sql.parser.SqlShowSchemas;
-import org.apache.drill.exec.planner.sql.parser.SqlShowTables;
-import org.apache.drill.exec.planner.sql.parser.SqlUseSchema;
-import org.apache.drill.exec.planner.physical.PrelUtil;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillParserImpl;
 import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.util.Pointer;
 import org.eigenbase.rel.RelCollationTraitDef;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.metadata.RelMetadataQuery;
 import org.eigenbase.relopt.ConventionTraitDef;
 import org.eigenbase.relopt.RelOptCostFactory;
 import org.eigenbase.relopt.RelTraitDef;
 import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.parser.SqlParseException;
-import org.eigenbase.sql2rel.StandardConvertletTable;
 
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
@@ -76,7 +67,7 @@ public class DrillSqlWorker {
     traitDefs.add(RelCollationTraitDef.INSTANCE);
     this.context = context;
     DrillOperatorTable table = new DrillOperatorTable(context.getFunctionRegistry());
-    RelOptCostFactory costFactory = (context.getPlannerSettings().useDefaultCosting()) ? 
+    RelOptCostFactory costFactory = (context.getPlannerSettings().useDefaultCosting()) ?
         null : new DrillCostBase.DrillCostFactory() ;
     StdFrameworkConfig config = StdFrameworkConfig.newBuilder() //
         .lex(Lex.MYSQL) //
@@ -109,6 +100,10 @@ public class DrillSqlWorker {
   }
 
   public PhysicalPlan getPlan(String sql) throws SqlParseException, ValidationException, RelConversionException, IOException{
+    return getPlan(null);
+  }
+
+  public PhysicalPlan getPlan(String sql, Pointer<String> textPlan) throws SqlParseException, ValidationException, RelConversionException, IOException{
     SqlNode sqlNode = planner.parse(sql);
 
     AbstractSqlHandler handler;
@@ -128,7 +123,7 @@ public class DrillSqlWorker {
       }
       // fallthrough
     default:
-      handler = new DefaultSqlHandler(planner, context);
+      handler = new DefaultSqlHandler(planner, context, textPlan);
     }
 
     return handler.getPlan(sqlNode);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 36ec0e8..e8bd837 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.physical.SelectionVectorPrelVisitor;
 import org.apache.drill.exec.planner.physical.explain.PrelSequencer;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
+import org.apache.drill.exec.util.Pointer;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
 import org.eigenbase.relopt.RelTraitSet;
@@ -59,11 +60,17 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
 
   protected final Planner planner;
   protected final QueryContext context;
+  private Pointer<String> textPlan;
 
   public DefaultSqlHandler(Planner planner, QueryContext context) {
+    this(planner, context, null);
+  }
+
+  public DefaultSqlHandler(Planner planner, QueryContext context, Pointer<String> textPlan) {
     super();
     this.planner = planner;
     this.context = context;
+    this.textPlan = textPlan;
   }
 
   protected void log(String name, RelNode node) {
@@ -73,8 +80,9 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
   }
 
   protected void log(String name, Prel node) {
+    if(textPlan != null) textPlan.value = PrelSequencer.printWithIds(node, SqlExplainLevel.ALL_ATTRIBUTES);
     if (logger.isDebugEnabled()) {
-      logger.debug(name + " : \n" + PrelSequencer.printWithIds(node, SqlExplainLevel.ALL_ATTRIBUTES));
+      logger.debug(name + " : \n" + textPlan.value);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index 8f533e3..a3307cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -64,7 +64,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
         .option(ChannelOption.SO_SNDBUF, 1 << 17) //
         .group(eventLoopGroup) //
         .childOption(ChannelOption.ALLOCATOR, alloc) //
-        .handler(new LoggingHandler(LogLevel.INFO)) //
+//        .handler(new LoggingHandler(LogLevel.INFO)) //
         .childHandler(new ChannelInitializer<SocketChannel>() {
           @Override
           protected void initChannel(SocketChannel ch) throws Exception {
@@ -90,7 +90,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
   }
 
   public abstract ProtobufLengthDecoder getDecoder(BufferAllocator allocator, OutOfMemoryHandler outOfMemoryHandler);
-  
+
   @Override
   public boolean isClient() {
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
index dfc10c2..cfc3819 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRestServer.java
@@ -15,6 +15,7 @@ public class DrillRestServer extends ResourceConfig {
   public DrillRestServer(final WorkManager workManager) {
 //    registerClasses(HelloResource.class);
     register(JacksonFeature.class);
+    register(DrillRoot.class);
     register(FreemarkerMvcFeature.class);
     property(ServerProperties.METAINF_SERVICES_LOOKUP_DISABLE, true);
     register(new AbstractBinder() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
index dae6b43..27e0d17 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/DrillRoot.java
@@ -1,15 +1,60 @@
 package org.apache.drill.exec.server.rest;
 
+import java.util.List;
+import java.util.Map;
+
+import javax.inject.Inject;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.drill.exec.cache.DistributedMap;
+import org.apache.drill.exec.cache.ProtobufDrillSerializable.CQueryProfile;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.work.WorkManager;
+import org.glassfish.jersey.server.mvc.Viewable;
+
+import com.google.hive12.hive12.common.collect.Lists;
 
 @Path("/")
 public class DrillRoot {
-    @GET
-    @Produces("text/plain")
-    public String getHello() {
-      return "hello world";
+
+  @Inject WorkManager work;
+
+  @GET
+  @Path("status")
+  @Produces("text/plain")
+  public String getHello() {
+    return "running";
+  }
+
+  @GET
+  @Path("queries")
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getQueries() {
+    DistributedMap<CQueryProfile> profiles = work.getContext().getCache().getNamedMap("sys.queries", CQueryProfile.class);
+
+    List<String> ids = Lists.newArrayList();
+    for(Map.Entry<String, CQueryProfile> entry : profiles){
+      ids.add(entry.getKey());
     }
 
+    return new Viewable("/rest/status/list.ftl", ids);
+  }
+
+  @GET
+  @Path("/query/{queryid}")
+  @Produces(MediaType.TEXT_HTML)
+  public Viewable getQuery(@PathParam("queryid") String queryId) {
+    DistributedMap<CQueryProfile> profiles = work.getContext().getCache().getNamedMap("sys.queries", CQueryProfile.class);
+    CQueryProfile c = profiles.get(queryId);
+    QueryProfile q = c == null ? QueryProfile.getDefaultInstance() : c.getObj();
+
+    return new Viewable("/rest/status/profile.ftl", q);
+
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
index bc378a1..88478f2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.io.Text;
 
 import java.util.ArrayList;
 
-public class JsonStringArrayList extends ArrayList {
+public class JsonStringArrayList<E> extends ArrayList<E> {
 
   private static ObjectMapper mapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
index 124252a..4b264bf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
@@ -28,7 +28,7 @@ import java.util.HashMap;
  * Simple class that extends the regular java.util.HashMap but overrides the
  * toString() method of the HashMap class to produce a JSON string instead
  */
-public class JsonStringHashMap extends HashMap {
+public class JsonStringHashMap<K, V> extends HashMap<K, V> {
 
   private static ObjectMapper mapper;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
new file mode 100644
index 0000000..8139943
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/Pointer.java
@@ -0,0 +1,11 @@
+package org.apache.drill.exec.util;
+
+public class Pointer<T> {
+  public T value;
+
+  public Pointer(){}
+
+  public Pointer(T value){
+    this.value = value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index b121586..45e7ee8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -22,9 +22,12 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.*;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
@@ -41,6 +44,7 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.work.batch.ControlHandlerImpl;
 import org.apache.drill.exec.work.batch.ControlMessageHandler;
 import org.apache.drill.exec.work.foreman.Foreman;
+import org.apache.drill.exec.work.foreman.QueryStatus;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 import org.apache.drill.exec.work.user.UserWorker;
@@ -50,15 +54,17 @@ import com.google.common.collect.Queues;
 
 public class WorkManager implements Closeable{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
-  
+
   private Set<FragmentManager> incomingFragments = Collections.newSetFromMap(Maps.<FragmentManager, Boolean> newConcurrentMap());
 
   private PriorityBlockingQueue<Runnable> pendingTasks = Queues.newPriorityBlockingQueue();
-  
+
   private Map<FragmentHandle, FragmentExecutor> runningFragments = Maps.newConcurrentMap();
-  
+
   private ConcurrentMap<QueryId, Foreman> queries = Maps.newConcurrentMap();
 
+  private ConcurrentMap<QueryId, QueryStatus> status = Maps.newConcurrentMap();
+
   private BootStrapContext bContext;
   private DrillbitContext dContext;
 
@@ -69,7 +75,7 @@ public class WorkManager implements Closeable{
   private final WorkEventBus workBus;
   private ExecutorService executor;
   private final EventThread eventThread;
-  
+
   public WorkManager(BootStrapContext context){
     this.bee = new WorkerBee();
     this.workBus = new WorkEventBus(bee);
@@ -79,22 +85,22 @@ public class WorkManager implements Closeable{
     this.eventThread = new EventThread();
     this.dataHandler = new DataResponseHandlerImpl(bee);
   }
-  
+
   public void start(DrillbitEndpoint endpoint, DistributedCache cache, Controller controller, DataConnectionCreator data, ClusterCoordinator coord){
     this.dContext = new DrillbitContext(endpoint, bContext, coord, controller, data, cache, workBus);
  //   executor = Executors.newFixedThreadPool(dContext.getConfig().getInt(ExecConstants.EXECUTOR_THREADS)
     executor = Executors.newCachedThreadPool(new NamedThreadFactory("WorkManager-"));
     eventThread.start();
   }
-  
+
   public WorkEventBus getWorkBus(){
     return workBus;
   }
-  
+
   public DataResponseHandler getDataHandler() {
     return dataHandler;
   }
-  
+
   public ControlMessageHandler getControlMessageHandler(){
     return controlMessageWorker;
   }
@@ -102,7 +108,7 @@ public class WorkManager implements Closeable{
   public UserWorker getUserWorker(){
     return userWorker;
   }
-  
+
   @Override
   public void close() throws IOException {
     try {
@@ -111,7 +117,7 @@ public class WorkManager implements Closeable{
       logger.warn("Executor interrupted while awaiting termination");
     }
   }
-  
+
 
   public DrillbitContext getContext() {
     return dContext;
@@ -124,7 +130,7 @@ public class WorkManager implements Closeable{
       logger.debug("Adding pending task {}", runner);
       pendingTasks.add(runner);
     }
-    
+
     public void addNewForeman(Foreman foreman){
       pendingTasks.add(foreman);
     }
@@ -133,20 +139,20 @@ public class WorkManager implements Closeable{
     public void addFragmentPendingRemote(FragmentManager handler){
       incomingFragments.add(handler);
     }
-    
+
     public void startFragmentPendingRemote(FragmentManager handler){
       incomingFragments.remove(handler);
       pendingTasks.add(handler.getRunnable());
     }
-    
+
     public FragmentExecutor getFragmentRunner(FragmentHandle handle){
       return runningFragments.get(handle);
     }
-    
+
     public Foreman getForemanForQueryId(QueryId queryId){
       return queries.get(queryId);
     }
-    
+
     public void retireForeman(Foreman foreman){
       queries.remove(foreman.getQueryId(), foreman);
     }
@@ -173,18 +179,18 @@ public class WorkManager implements Closeable{
       Runnable r = pendingTasks.take();
       if(r != null){
         logger.debug("Starting pending task {}", r);
-        executor.execute(r);  
+        executor.execute(r);
       }
-      
+
     }
     } catch (InterruptedException e) {
       logger.info("Work Manager stopping as it was interrupted.");
     }
   }
-   
-   
+
+
  }
 
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 3e8d3e1..eb1d738 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -21,23 +21,13 @@ import io.netty.buffer.ByteBuf;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 
-import com.fasterxml.jackson.databind.JsonNode;
-
-import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
-import org.apache.drill.common.types.TypeProtos.MinorType;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.cache.DistributedMultiMap;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.exception.OptimizerException;
 import org.apache.drill.exec.ops.QueryContext;
@@ -55,28 +45,24 @@ import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
-import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.rpc.BaseRpcOutcomeListener;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.util.AtomicState;
-import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.ErrorHelper;
 import org.apache.drill.exec.work.QueryWorkUnit;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
 
-import com.google.common.base.Charsets;
 import com.google.common.collect.Lists;
 
 /**
@@ -99,7 +85,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     this.queryRequest = queryRequest;
     this.context = new QueryContext(connection.getSession(), queryId, dContext);
     this.initiatingClient = connection;
-    this.fragmentManager = new QueryManager(queryId, bee.getContext().getCache(), new ForemanManagerListener(), dContext.getController());
+    this.fragmentManager = new QueryManager(queryId, queryRequest, bee.getContext().getCache(), new ForemanManagerListener(), dContext.getController());
     this.bee = bee;
 
     this.state = new AtomicState<QueryState>(QueryState.PENDING) {
@@ -335,7 +321,9 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   private void runSQL(String sql) {
     try{
       DrillSqlWorker sqlWorker = new DrillSqlWorker(context);
-      PhysicalPlan plan = sqlWorker.getPlan(sql);
+      Pointer<String> textPlan = new Pointer<>();
+      PhysicalPlan plan = sqlWorker.getPlan(sql, textPlan);
+      fragmentManager.getStatus().setPlanText(textPlan.value);
       runPhysicalPlan(plan);
     }catch(Exception e){
       fail("Failure while parsing sql.", e);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
index 509000f..3bbe692 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentData.java
@@ -1,9 +1,10 @@
 package org.apache.drill.exec.work.foreman;
 
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.FragmentState;
+import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 
 public class FragmentData {
   private final boolean isLocal;
@@ -13,7 +14,8 @@ public class FragmentData {
 
   public FragmentData(FragmentHandle handle, DrillbitEndpoint endpoint, boolean isLocal) {
     super();
-    this.status = FragmentStatus.newBuilder().setHandle(handle).setState(FragmentState.SENDING).build();
+    MinorFragmentProfile f = MinorFragmentProfile.newBuilder().setState(FragmentState.SENDING).build();
+    this.status = FragmentStatus.newBuilder().setHandle(handle).setProfile(f).build();
     this.endpoint = endpoint;
     this.isLocal = isLocal;
   }
@@ -39,5 +41,16 @@ public class FragmentData {
     return endpoint;
   }
 
+  public FragmentHandle getHandle(){
+    return status.getHandle();
+  }
+
+  @Override
+  public String toString() {
+    return "FragmentData [isLocal=" + isLocal + ", status=" + status + ", lastStatusUpdate=" + lastStatusUpdate
+        + ", endpoint=" + endpoint + "]";
+  }
+
+
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 62d9375..586b221 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserProtos.QueryResult;
 import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.Controller;
@@ -58,21 +59,32 @@ public class QueryManager implements FragmentStatusListener{
   private ForemanManagerListener foreman;
   private AtomicInteger remainingFragmentCount;
   private WorkEventBus workBus;
+  private QueryId queryId;
   private FragmentExecutor rootRunner;
-  private volatile QueryId queryId;
+  private RunQuery query;
 
-  public QueryManager(QueryId queryId, DistributedCache cache, ForemanManagerListener foreman, Controller controller) {
+  public QueryManager(QueryId id, RunQuery query, DistributedCache cache, ForemanManagerListener foreman, Controller controller) {
     super();
     this.foreman = foreman;
+    this.query = query;
+    this.queryId =  id;
     this.controller = controller;
     this.remainingFragmentCount = new AtomicInteger(0);
-    this.status = new QueryStatus(queryId, cache);
+    this.status = new QueryStatus(query, id, cache);
+  }
+
+  public QueryStatus getStatus(){
+    return status;
+  }
+
+  public void addTextPlan(String textPlan){
+
   }
 
   public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments, List<PlanFragment> intermediateFragments) throws ExecutionSetupException{
     logger.debug("Setting up fragment runs.");
     remainingFragmentCount.set(leafFragments.size()+1);
-    queryId = rootFragment.getHandle().getQueryId();
+    assert queryId == rootFragment.getHandle().getQueryId();
     workBus = bee.getContext().getWorkBus();
 
     // set up the root fragment first so we'll have incoming buffers available.
@@ -84,7 +96,7 @@ public class QueryManager implements FragmentStatusListener{
       logger.debug("Setting buffers on root context.");
       rootContext.setBuffers(buffers);
       // add fragment to local node.
-      status.add(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
+      status.add(new FragmentData(rootFragment.getHandle(), null, true));
       logger.debug("Fragment added to local node.");
       rootRunner = new FragmentExecutor(rootContext, rootOperator, new RootStatusHandler(rootContext, rootFragment));
       RootFragmentManager fragmentManager = new RootFragmentManager(rootFragment.getHandle(), buffers, rootRunner);
@@ -103,7 +115,7 @@ public class QueryManager implements FragmentStatusListener{
     // keep track of intermediate fragments (not root or leaf)
     for (PlanFragment f : intermediateFragments) {
       logger.debug("Tracking intermediate remote node {} with data {}", f.getAssignment(), f.getFragmentJson());
-      status.add(f.getHandle(), new FragmentData(f.getHandle(), f.getAssignment(), false));
+      status.add(new FragmentData(f.getHandle(), f.getAssignment(), false));
     }
 
     // send remote (leaf) fragments.
@@ -116,7 +128,7 @@ public class QueryManager implements FragmentStatusListener{
 
   private void sendRemoteFragment(PlanFragment fragment){
     logger.debug("Sending remote fragment to node {} with data {}", fragment.getAssignment(), fragment.getFragmentJson());
-    status.add(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
+    status.add(new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
     FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
     controller.getTunnel(fragment.getAssignment()).sendFragment(listener, fragment);
   }
@@ -125,7 +137,7 @@ public class QueryManager implements FragmentStatusListener{
   @Override
   public void statusUpdate(FragmentStatus status) {
     logger.debug("New fragment status was provided to Foreman of {}", status);
-    switch(status.getState()){
+    switch(status.getProfile().getState()){
     case AWAITING_ALLOCATION:
       updateStatus(status);
       break;
@@ -142,7 +154,7 @@ public class QueryManager implements FragmentStatusListener{
       updateStatus(status);
       break;
     default:
-      throw new UnsupportedOperationException();
+      throw new UnsupportedOperationException(String.format("Received status of %s", status));
     }
   }
 
@@ -154,6 +166,7 @@ public class QueryManager implements FragmentStatusListener{
     updateStatus(status);
     int remaining = remainingFragmentCount.decrementAndGet();
     if(remaining == 0){
+      logger.info("Outcome status: {}", this.status);
       QueryResult result = QueryResult.newBuilder() //
               .setQueryState(QueryState.COMPLETED) //
               .setQueryId(queryId) //
@@ -165,7 +178,7 @@ public class QueryManager implements FragmentStatusListener{
   private void fail(FragmentStatus status){
     updateStatus(status);
     stopQuery();
-    QueryResult result = QueryResult.newBuilder().setQueryId(queryId).setQueryState(QueryState.FAILED).addError(status.getError()).build();
+    QueryResult result = QueryResult.newBuilder().setQueryId(queryId).setQueryState(QueryState.FAILED).addError(status.getProfile().getError()).build();
     foreman.cleanupAndSendResult(result);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
index 64ec671..991e9ac 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryStatus.java
@@ -1,33 +1,143 @@
 package org.apache.drill.exec.work.foreman;
 
-import java.util.Map;
-
 import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.DistributedMap;
+import org.apache.drill.exec.cache.ProtobufDrillSerializable.CQueryProfile;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryProfile;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 
-import com.google.common.collect.Maps;
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
 
 public class QueryStatus {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryStatus.class);
 
-  public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.
+
+  // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.
+  private IntObjectOpenHashMap<IntObjectOpenHashMap<FragmentData>> map = new IntObjectOpenHashMap<IntObjectOpenHashMap<FragmentData>>();
 
   private final String queryId;
+  private final QueryId id;
+  private RunQuery query;
+  private String planText;
+
+  private final DistributedMap<CQueryProfile> profileCache;
 
-  public QueryStatus(QueryId id, DistributedCache cache){
+  public QueryStatus(RunQuery query, QueryId id, DistributedCache cache){
+    this.id = id;
+    this.query = query;
     this.queryId = QueryIdHelper.getQueryId(id);
-//    cache.getMultiMap(QueryStatus.class);
+    this.profileCache = cache.getNamedMap("sys.queries", CQueryProfile.class);
+  }
+
+  public void setPlanText(String planText){
+    this.planText = planText;
+    updateCache();
 
   }
+  void add(FragmentData data){
+    int majorFragmentId = data.getHandle().getMajorFragmentId();
+    int minorFragmentId = data.getHandle().getMinorFragmentId();
+    IntObjectOpenHashMap<FragmentData> minorMap = map.get(majorFragmentId);
+    if(minorMap == null){
+      minorMap = new IntObjectOpenHashMap<FragmentData>();
+      map.put(majorFragmentId, minorMap);
+    }
 
-  void add(FragmentHandle handle, FragmentData data){
-    if(map.put(handle,  data) != null) throw new IllegalStateException();
+    minorMap.put(minorFragmentId, data);
   }
 
   void update(FragmentStatus status){
-    map.get(status.getHandle()).setStatus(status);
+    int majorFragmentId = status.getHandle().getMajorFragmentId();
+    int minorFragmentId = status.getHandle().getMinorFragmentId();
+    map.get(majorFragmentId).get(minorFragmentId).setStatus(status);
+    updateCache();
+  }
+
+  private void updateCache(){
+    profileCache.put(queryId, new CQueryProfile(getAsProfile()));
+  }
+
+  public String toString(){
+    return map.toString();
+  }
+
+  public static class FragmentId{
+    int major;
+    int minor;
+
+    public FragmentId(FragmentStatus status){
+      this.major = status.getHandle().getMajorFragmentId();
+      this.minor = status.getHandle().getMinorFragmentId();
+    }
+
+    public FragmentId(FragmentData data){
+      this.major = data.getHandle().getMajorFragmentId();
+      this.minor = data.getHandle().getMinorFragmentId();
+    }
+
+    public FragmentId(int major, int minor) {
+      super();
+      this.major = major;
+      this.minor = minor;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + major;
+      result = prime * result + minor;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      FragmentId other = (FragmentId) obj;
+      if (major != other.major)
+        return false;
+      if (minor != other.minor)
+        return false;
+      return true;
+    }
+
+    public String toString(){
+      return major + ":" + minor;
+    }
+  }
+
+  public QueryProfile getAsProfile(){
+    QueryProfile.Builder b = QueryProfile.newBuilder();
+    b.setQuery(query.getPlan());
+    b.setType(query.getType());
+    if(planText != null) b.setPlan(planText);
+    b.setId(id);
+    for(int i = 0; i < map.allocated.length; i++){
+      if(map.allocated[i]){
+        int majorFragmentId = map.keys[i];
+        IntObjectOpenHashMap<FragmentData> minorMap = (IntObjectOpenHashMap<FragmentData>) ((Object[]) map.values)[i];
+
+        MajorFragmentProfile.Builder fb = MajorFragmentProfile.newBuilder();
+        fb.setMajorFragmentId(majorFragmentId);
+        for(int v = 0; v < minorMap.allocated.length; v++){
+          if(minorMap.allocated[v]){
+            FragmentData data = (FragmentData) ((Object[]) minorMap.values)[v];
+            fb.addMinorFragmentProfile(data.getStatus().getProfile());
+          }
+        }
+        b.addFragmentProfile(fb);
+      }
+    }
+
+    return b.build();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
index 105afc2..30e4b6a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AbstractStatusReporter.java
@@ -19,31 +19,39 @@ package org.apache.drill.exec.work.fragment;
 
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.FragmentState;
+import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 import org.apache.drill.exec.work.ErrorHelper;
 
 public abstract class AbstractStatusReporter implements StatusReporter{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStatusReporter.class);
-  
+
   private FragmentContext context;
   private volatile long startNanos;
-  
+
   public AbstractStatusReporter(FragmentContext context) {
     super();
     this.context = context;
   }
-  
+
   private  FragmentStatus.Builder getBuilder(FragmentState state){
+    return getBuilder(state, null, null);
+  }
+  private  FragmentStatus.Builder getBuilder(FragmentState state, String message, Throwable t){
     FragmentStatus.Builder status = FragmentStatus.newBuilder();
-    context.getStats().addMetricsToStatus(status);
-    status.setState(state);
-    status.setRunningTime(System.nanoTime() - startNanos);
+    MinorFragmentProfile.Builder b = MinorFragmentProfile.newBuilder();
+    context.getStats().addMetricsToStatus(b);
+    b.setState(state);
+    if(t != null){
+      b.setError(ErrorHelper.logAndConvertError(context.getIdentity(), message, t, logger));
+    }
     status.setHandle(context.getHandle());
-    status.setMemoryUse(context.getAllocator().getAllocatedMemory());
+    b.setMemoryUsed(context.getAllocator().getAllocatedMemory());
+    status.setProfile(b);
     return status;
   }
-  
+
   @Override
   public void stateChanged(FragmentHandle handle, FragmentState newState) {
     FragmentStatus.Builder status = getBuilder(newState);
@@ -70,14 +78,14 @@ public abstract class AbstractStatusReporter implements StatusReporter{
       break;
     default:
       break;
-    
+
     }
   }
-  
+
   protected void awaitingAllocation(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
     statusChange(handle, statusBuilder.build());
   }
-  
+
   protected void running(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
     statusChange(handle, statusBuilder.build());
   }
@@ -89,13 +97,12 @@ public abstract class AbstractStatusReporter implements StatusReporter{
   protected void finished(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
     statusChange(handle, statusBuilder.build());
   }
-  
+
   protected abstract void statusChange(FragmentHandle handle, FragmentStatus status);
 
   @Override
   public final void fail(FragmentHandle handle, String message, Throwable excep) {
-    FragmentStatus.Builder status = getBuilder(FragmentState.FAILED);
-    status.setError(ErrorHelper.logAndConvertError(context.getIdentity(), message, excep, logger));
+    FragmentStatus.Builder status = getBuilder(FragmentState.FAILED, message, excep);
     fail(handle, status);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
index 7890fc9..70f5dd0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentExecutor.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.work.fragment;
 
-import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -26,8 +25,7 @@ import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.RootExec;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
-import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
-import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.work.CancelableQuery;
@@ -47,7 +45,7 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
   private RootExec root;
   private final FragmentContext context;
   private final StatusReporter listener;
-  
+
   public FragmentExecutor(FragmentContext context, FragmentRoot rootOperator, StatusReporter listener){
     this.context = context;
     this.rootOperator = rootOperator;
@@ -56,11 +54,7 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
 
   @Override
   public FragmentStatus getStatus() {
-    return FragmentStatus.newBuilder() //
-        .setBatchesCompleted(context.getStats().batchesCompleted.get()) //
-        .setDataProcessed(context.getStats().dataProcessed.get()) //
-        .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
-        .build();
+    throw new UnsupportedOperationException();
   }
 
   @Override
@@ -71,7 +65,7 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
   public UserClientConnection getClient(){
     return context.getConnection();
   }
-  
+
   @Override
   public void run() {
     final String originalThread = Thread.currentThread().getName();
@@ -81,7 +75,7 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
         context.getHandle().getMinorFragmentId()
         );
     Thread.currentThread().setName(newThreadName);
-    
+
     boolean closed = false;
     try {
       root = ImplCreator.getExec(context, rootOperator);
@@ -95,33 +89,32 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
       internalFail(new RuntimeException(String.format("Run was called when fragment was in %s state.  FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
       return;
     }
-    
-    Timer.Context t = context.getStats().fragmentTime.time();
-    
+
+
+
     // run the query until root.next returns false.
     try{
       while(state.get() == FragmentState.RUNNING_VALUE){
         if(!root.next()){
           if(context.isFailed()){
-            updateState(FragmentState.RUNNING, FragmentState.FAILED, false);  
+            updateState(FragmentState.RUNNING, FragmentState.FAILED, false);
           }else{
             updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
           }
 
         }
       }
-      
+
       root.stop();
-      
+
       closed = true;
-      
+
       context.close();
     }catch(Exception ex){
       logger.debug("Caught exception while running fragment", ex);
       internalFail(ex);
     }finally{
       Thread.currentThread().setName(originalThread);
-      t.stop();
       if(!closed) try{
         context.close();
       }catch(RuntimeException e){
@@ -130,17 +123,17 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
     }
     logger.debug("Fragment runner complete. {}:{}", context.getHandle().getMajorFragmentId(), context.getHandle().getMinorFragmentId());
   }
-  
+
   private void internalFail(Throwable excep){
     state.set(FragmentState.FAILED_VALUE);
     listener.fail(context.getHandle(), "Failure while running fragment.", excep);
   }
-  
+
   private void updateState(FragmentState update){
     state.set(update.getNumber());
     listener.stateChanged(context.getHandle(), update);
   }
-  
+
   private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
     boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
     if (!success && exceptionOnFailure) {
@@ -161,5 +154,5 @@ public class FragmentExecutor implements Runnable, CancelableQuery, StatusProvid
   public FragmentContext getContext(){
     return context;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
index c7c3439..84fb806 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
@@ -42,7 +42,7 @@ import org.apache.drill.exec.work.batch.IncomingBuffers;
  */
 public class NonRootFragmentManager implements FragmentManager {
   private final PlanFragment fragment;
-  private FragmentLeaf root;
+  private FragmentRoot root;
   private final IncomingBuffers buffers;
   private final StatusReporter runnerListener;
   private volatile FragmentExecutor runner;
@@ -50,7 +50,7 @@ public class NonRootFragmentManager implements FragmentManager {
   private final FragmentContext context;
   private final PhysicalPlanReader reader;
   private List<RemoteConnection> connections = new CopyOnWriteArrayList<>();
-  
+
   public NonRootFragmentManager(PlanFragment fragment, DrillbitContext context) throws FragmentSetupException{
     try{
       this.fragment = fragment;
@@ -82,14 +82,7 @@ public class NonRootFragmentManager implements FragmentManager {
     synchronized(this){
       if(runner != null) throw new IllegalStateException("Get Runnable can only be run once.");
       if(cancel) return null;
-      FragmentRoot fragRoot = null;
-      try {
-        fragRoot = reader.readFragmentOperator(fragment.getFragmentJson());
-      } catch (IOException e) {
-        runnerListener.fail(fragment.getHandle(), "Failure while setting up remote fragment.", e);
-        return null;
-      }
-      runner = new FragmentExecutor(context, fragRoot, runnerListener);
+      runner = new FragmentExecutor(context, root, runnerListener);
       return this.runner;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
index 45c1f5e..26b5d68 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/StatusReporter.java
@@ -17,8 +17,8 @@
  */
 package org.apache.drill.exec.work.fragment;
 
-import org.apache.drill.exec.proto.BitControl.FragmentStatus.FragmentState;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.FragmentState;
 
 /**
  * The status handler is responsible for receiving changes in fragment status and propagating them back to the foreman.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/resources/rest/status/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/list.ftl b/exec/java-exec/src/main/resources/rest/status/list.ftl
new file mode 100644
index 0000000..6bde590
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/status/list.ftl
@@ -0,0 +1,9 @@
+<html>
+Welcome to Drill! 
+<br />
+<#list model as query>
+
+<a href="/query/${query}">${query}</a><br />
+
+</#list>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/cb90852a/exec/java-exec/src/main/resources/rest/status/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/status/profile.ftl b/exec/java-exec/src/main/resources/rest/status/profile.ftl
new file mode 100644
index 0000000..94ff6ef
--- /dev/null
+++ b/exec/java-exec/src/main/resources/rest/status/profile.ftl
@@ -0,0 +1,19 @@
+<html>
+<a href="/queries">back</a><br />
+
+<pre>
+${model.query}
+</pre> 
+<br /><br />
+<pre>
+${model.plan}
+</pre>
+
+
+<pre>
+${model.toString()}
+</pre>
+
+
+
+<html>
\ No newline at end of file


[09/24] git commit: status changes

Posted by ja...@apache.org.
status changes


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/5472140a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/5472140a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/5472140a

Branch: refs/heads/diagnostics2
Commit: 5472140ab228376ed6e205e60edec473771b9dba
Parents: 73d3bd0
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon May 19 07:57:25 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 19 09:12:09 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/types/Types.java    |    5 +-
 exec/java-exec/pom.xml                          |   58 +-
 .../apache/drill/exec/client/DrillClient.java   |    2 +-
 .../drill/exec/client/QuerySubmitter.java       |   10 +-
 .../apache/drill/exec/ops/FragmentStats.java    |   37 +-
 .../org/apache/drill/exec/ops/MetricDef.java    |    6 +
 .../org/apache/drill/exec/ops/OpProfileDef.java |   20 +
 .../apache/drill/exec/ops/OperatorContext.java  |   25 +
 .../apache/drill/exec/ops/OperatorStats.java    |  117 +-
 .../exec/physical/base/AbstractExchange.java    |   12 +-
 .../exec/physical/base/AbstractGroupScan.java   |    4 +
 .../exec/physical/base/PhysicalOperator.java    |    3 +
 .../exec/physical/config/BroadcastSender.java   |    9 +
 .../exec/physical/config/ExternalSort.java      |   19 +-
 .../drill/exec/physical/config/Filter.java      |   13 +-
 .../exec/physical/config/HashAggregate.java     |   30 +-
 .../drill/exec/physical/config/HashJoinPOP.java |    8 +-
 .../physical/config/HashPartitionSender.java    |    9 +-
 .../exec/physical/config/IteratorValidator.java |    7 +-
 .../drill/exec/physical/config/Limit.java       |    5 +
 .../exec/physical/config/MergeJoinPOP.java      |    6 +
 .../physical/config/MergingReceiverPOP.java     |    5 +
 .../physical/config/OrderedPartitionSender.java |    9 +-
 .../drill/exec/physical/config/Project.java     |   12 +-
 .../exec/physical/config/RandomReceiver.java    |   14 +-
 .../drill/exec/physical/config/RangeSender.java |   10 +-
 .../drill/exec/physical/config/Screen.java      |    7 +-
 .../physical/config/SelectionVectorRemover.java |    8 +-
 .../exec/physical/config/SingleSender.java      |   13 +-
 .../apache/drill/exec/physical/config/Sort.java |   20 +-
 .../physical/config/StreamingAggregate.java     |   13 +-
 .../apache/drill/exec/physical/config/TopN.java |   18 +-
 .../drill/exec/physical/config/Trace.java       |    6 +
 .../drill/exec/physical/config/Union.java       |    6 +
 .../exec/physical/impl/WriterRecordBatch.java   |  153 +-
 .../physical/impl/aggregate/HashAggBatch.java   |  126 +-
 .../impl/aggregate/StreamingAggBatch.java       |  141 +-
 .../exec/physical/impl/join/HashJoinBatch.java  |   46 +-
 .../exec/physical/impl/join/JoinStatus.java     |   19 +-
 .../exec/physical/impl/join/MergeJoinBatch.java |   81 +-
 .../impl/mergereceiver/MergingRecordBatch.java  |   50 +-
 .../OrderedPartitionRecordBatch.java            |  173 +-
 .../drill/exec/record/AbstractRecordBatch.java  |   27 +-
 .../exec/record/AbstractSingleRecordBatch.java  |   68 +-
 .../org/apache/drill/exec/server/Drillbit.java  |   20 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |   16 +-
 .../drill/exec/store/dfs/easy/EasySubScan.java  |   17 +-
 .../drill/exec/store/dfs/easy/EasyWriter.java   |    5 +
 .../drill/exec/store/direct/DirectSubScan.java  |    7 +
 .../exec/store/easy/json/JSONFormatPlugin.java  |   14 +
 .../exec/store/easy/text/TextFormatPlugin.java  |   15 +-
 .../drill/exec/store/hive/HiveSubScan.java      |    7 +
 .../exec/store/ischema/InfoSchemaSubScan.java   |   13 +-
 .../drill/exec/store/mock/MockStorePOP.java     |    8 +-
 .../drill/exec/store/mock/MockSubScanPOP.java   |    8 +-
 .../exec/store/parquet/ParquetRowGroupScan.java |    6 +
 .../drill/exec/store/parquet/ParquetWriter.java |    8 +
 .../apache/drill/exec/work/foreman/Foreman.java |    4 +-
 .../drill/exec/work/foreman/FragmentData.java   |   43 +
 .../drill/exec/work/foreman/QueryManager.java   |  106 +-
 .../drill/exec/work/foreman/QueryStatus.java    |   33 +
 .../exec/physical/impl/PerformanceTests.java    |   31 -
 .../physical/impl/TestBroadcastExchange.java    |   19 +-
 .../impl/TestExecutionAbstractions.java         |  225 -
 .../drill/exec/server/DrillClientFactory.java   |   21 +
 .../apache/drill/exec/server/HelloResource.java |   22 +
 .../apache/drill/exec/server/TestJersey.java    |   48 +
 .../drill/exec/server/rest/RootResource.java    |    9 +
 exec/java-exec/src/test/resources/logback.xml   |    2 +-
 .../org/apache/drill/jdbc/DrillResultSet.java   |   26 +-
 .../apache/drill/common/types/TypeProtos.java   |  117 +-
 .../org/apache/drill/exec/proto/BitControl.java |  245 +-
 .../apache/drill/exec/proto/UserBitShared.java  | 6290 +++++++++++++++++-
 .../org/apache/drill/exec/proto/UserProtos.java |  184 +-
 protocol/src/main/protobuf/BitControl.proto     |    1 +
 protocol/src/main/protobuf/User.proto           |    8 +-
 protocol/src/main/protobuf/UserBitShared.proto  |   88 +
 77 files changed, 7963 insertions(+), 1133 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/common/src/main/java/org/apache/drill/common/types/Types.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/types/Types.java b/common/src/main/java/org/apache/drill/common/types/Types.java
index 5d0812f..8a826b1 100644
--- a/common/src/main/java/org/apache/drill/common/types/Types.java
+++ b/common/src/main/java/org/apache/drill/common/types/Types.java
@@ -113,8 +113,6 @@ public class Types {
     case INTERVALYEAR:
     case INTERVALDAY:
     case LATE:
-    case REPEATMAP:
-      return java.sql.Types.OTHER;
     case SMALLINT:
       return java.sql.Types.SMALLINT;
     case TIME:
@@ -226,7 +224,6 @@ public class Types {
     case LATE:
       return Comparability.UNKNOWN;
     case MAP:
-    case REPEATMAP:
       return Comparability.NONE;
     case BIT:
       return Comparability.EQUAL;
@@ -274,7 +271,7 @@ public class Types {
   public static MajorType withScaleAndPrecision(MinorType type, DataMode mode, int scale, int precision) {
     return MajorType.newBuilder().setMinorType(type).setMode(mode).setScale(scale).setPrecision(precision).build();
   }
-  
+
   public static MajorType required(MinorType type){
     return MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(type).build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index e282c00..d693630 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -72,6 +72,26 @@
       <version>2.7.3</version>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <version>9.1.5.v20140505</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+      <version>9.1.5.v20140505</version>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-jetty-servlet</artifactId>
+      <version>2.8</version>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson</artifactId>
+      <version>2.8</version>
+    </dependency>
+    <dependency>
       <groupId>net.hydromatic</groupId>
       <artifactId>optiq-core</artifactId>
     </dependency>
@@ -81,17 +101,11 @@
       <version>2.3.19</version>
     </dependency>
     <dependency>
-      <groupId>com.google.caliper</groupId>
-      <artifactId>caliper</artifactId>
-      <version>1.0-beta-1</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>pentaho</groupId>
       <artifactId>mondrian-data-foodmart-json</artifactId>
       <version>0.3.2</version>
       <scope>test</scope>
-    </dependency>    
+    </dependency>
     <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-column</artifactId>
@@ -220,9 +234,9 @@
       <version>6.1.26</version>
     </dependency>
     <dependency>
-        <groupId>joda-time</groupId>
-        <artifactId>joda-time</artifactId>
-        <version>2.3</version>
+      <groupId>joda-time</groupId>
+      <artifactId>joda-time</artifactId>
+      <version>2.3</version>
     </dependency>
   </dependencies>
 
@@ -238,7 +252,18 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-core</artifactId>
+          <exclusions>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-core</artifactId>
+            </exclusion>
+            <exclusion>
+              <groupId>com.sun.jersey</groupId>
+              <artifactId>jersey-json</artifactId>
+            </exclusion>
+          </exclusions>
         </dependency>
+
       </dependencies>
     </profile>
     <profile>
@@ -343,7 +368,7 @@
             <configuration>
               <tasks>
                 <copy todir="${project.build.directory}/codegen">
-                  <fileset dir="src/main/codegen"/>
+                  <fileset dir="src/main/codegen" />
                 </copy>
               </tasks>
             </configuration>
@@ -351,8 +376,8 @@
         </executions>
       </plugin>
       <plugin>
-        <!-- Extract parser grammar template from optiq-core.jar and put it under
-             ${project.build.directory} where all freemarker templates are. -->
+        <!-- Extract parser grammar template from optiq-core.jar and put 
+          it under ${project.build.directory} where all freemarker templates are. -->
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
         <version>2.8</version>
@@ -477,7 +502,8 @@
     </plugins>
     <pluginManagement>
       <plugins>
-        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+        <!--This plugin's configuration is used to store Eclipse m2e settings 
+          only. It has no influence on the Maven build itself. -->
         <plugin>
           <groupId>org.eclipse.m2e</groupId>
           <artifactId>lifecycle-mapping</artifactId>
@@ -495,10 +521,10 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <execute >
+                    <execute>
                       <runOnIncremental>false</runOnIncremental>
                       <runOnConfiguration>true</runOnConfiguration>
-                   </execute >
+                    </execute>
                   </action>
                 </pluginExecution>
               </pluginExecutions>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index fc650b9..3b87dc4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -38,9 +38,9 @@ import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.proto.UserProtos;
 import org.apache.drill.exec.proto.UserProtos.Property;
-import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
index 7967957..99e0c80 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.util.VectorUtil;
@@ -149,19 +149,19 @@ public class QuerySubmitter {
     PrintingResultsListener listener;
 
     String[] queries;
-    UserProtos.QueryType queryType;
+    QueryType queryType;
     type = type.toLowerCase();
     switch(type) {
       case "sql":
-        queryType = UserProtos.QueryType.SQL;
+        queryType = QueryType.SQL;
         queries = plan.trim().split(";");
         break;
       case "logical":
-        queryType = UserProtos.QueryType.LOGICAL;
+        queryType = QueryType.LOGICAL;
         queries = new String[]{ plan };
         break;
       case "physical":
-        queryType = UserProtos.QueryType.PHYSICAL;
+        queryType = QueryType.PHYSICAL;
         queries = new String[]{ plan };
         break;
       default:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
index 068b7fd..205c3f1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
@@ -17,16 +17,20 @@
  */
 package org.apache.drill.exec.ops;
 
+import java.util.List;
+
 import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
+import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 
 import com.codahale.metrics.MetricRegistry;
 import com.codahale.metrics.Timer;
+import com.google.hive12.common.collect.Lists;
 
 public class FragmentStats {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
-  
+
   private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentExecutor.class,
       "completionTimes");
   private final static String METRIC_BATCHES_COMPLETED = MetricRegistry
@@ -35,28 +39,47 @@ public class FragmentStats {
       .name(FragmentExecutor.class, "recordsCompleted");
   private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentExecutor.class, "dataProcessed");
 
-  
-  
+
+
   private final MetricRegistry metrics;
-  
+
+  private List<OperatorStats> operators = Lists.newArrayList();
+
   public final SingleThreadNestedCounter batchesCompleted;
   public final SingleThreadNestedCounter recordsCompleted;
   public final SingleThreadNestedCounter dataProcessed;
   public final Timer fragmentTime;
+  private final long startTime;
 
   public FragmentStats(MetricRegistry metrics) {
     this.metrics = metrics;
+    this.startTime = System.currentTimeMillis();
     this.fragmentTime = metrics.timer(METRIC_TIMER_FRAGMENT_TIME);
     this.batchesCompleted = new SingleThreadNestedCounter(metrics, METRIC_BATCHES_COMPLETED);
     this.recordsCompleted = new SingleThreadNestedCounter(metrics, METRIC_RECORDS_COMPLETED);
-    this.dataProcessed = new SingleThreadNestedCounter(metrics, METRIC_DATA_PROCESSED);    
+    this.dataProcessed = new SingleThreadNestedCounter(metrics, METRIC_DATA_PROCESSED);
   }
-  
+
   public void addMetricsToStatus(FragmentStatus.Builder stats) {
     stats.setBatchesCompleted(batchesCompleted.get());
     stats.setDataProcessed(dataProcessed.get());
     stats.setRecordsCompleted(recordsCompleted.get());
+
+    MinorFragmentProfile.Builder prfB = MinorFragmentProfile.newBuilder();
+    prfB.setStartTime(startTime);
+    prfB.setEndTime(System.currentTimeMillis());
+
+    for(OperatorStats o : operators){
+      prfB.addOperatorProfile(o.getProfile());
+    }
+
+    stats.setProfile(prfB);
+  }
+
+  public OperatorStats getOperatorStats(OpProfileDef profileDef){
+    OperatorStats stats = new OperatorStats(profileDef);
+    operators.add(stats);
+    return stats;
   }
-  
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
new file mode 100644
index 0000000..e08a2b2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/MetricDef.java
@@ -0,0 +1,6 @@
+package org.apache.drill.exec.ops;
+
+public interface MetricDef {
+  public String name();
+  public int metricId();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
new file mode 100644
index 0000000..61f6d20
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OpProfileDef.java
@@ -0,0 +1,20 @@
+package org.apache.drill.exec.ops;
+
+public class OpProfileDef {
+
+  public int operatorId;
+  public int operatorType;
+  public int incomingCount;
+
+  public int getOperatorId(){
+    return operatorId;
+  }
+
+  public int getOperatorType(){
+    return operatorType;
+  }
+  public int getIncomingCount(){
+    return incomingCount;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
index 3b7b4c1..824ec6a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContext.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.ops;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.util.Hook.Closeable;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.OutOfMemoryException;
@@ -28,10 +30,29 @@ public class OperatorContext implements Closeable {
   private final BufferAllocator allocator;
   private boolean closed = false;
   private PhysicalOperator popConfig;
+  private int operatorId;
+  private OperatorStats stats;
 
   public OperatorContext(PhysicalOperator popConfig, FragmentContext context) throws OutOfMemoryException {
     this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation());
     this.popConfig = popConfig;
+    this.operatorId = popConfig.getOperatorId();
+
+    OpProfileDef def = new OpProfileDef();
+    def.operatorId = operatorId;
+    def.incomingCount = getChildCount(popConfig);
+    def.operatorType = popConfig.getOperatorType();
+    this.stats = context.getStats().getOperatorStats(def);
+  }
+
+  private static int getChildCount(PhysicalOperator popConfig){
+    Iterator<PhysicalOperator> iter = popConfig.iterator();
+    int i = 0;
+    while(iter.hasNext()){
+      iter.next();
+      i++;
+    }
+    return i;
   }
 
   public BufferAllocator getAllocator() {
@@ -57,4 +78,8 @@ public class OperatorContext implements Closeable {
     }
     closed = true;
   }
+
+  public OperatorStats getStats(){
+    return stats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
index ce73a0d..dc463b3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorStats.java
@@ -17,10 +17,119 @@
  */
 package org.apache.drill.exec.ops;
 
+import org.apache.commons.collections.Buffer;
+import org.apache.drill.exec.proto.UserBitShared.OperatorProfile;
+import org.apache.drill.exec.proto.UserBitShared.StreamProfile;
+
+import com.carrotsearch.hppc.IntDoubleOpenHashMap;
+import com.carrotsearch.hppc.IntLongOpenHashMap;
+
 public class OperatorStats {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorStats.class);
-  
-  private long batchesCompleted;
-  private long dataCompleted;
-  
+
+  private final int operatorId;
+  private final int operatorType;
+
+  private IntLongOpenHashMap longMetrics = new IntLongOpenHashMap();
+  private IntDoubleOpenHashMap doubleMetrics = new IntDoubleOpenHashMap();
+
+  public long[] recordsReceivedByInput;
+  public long[] batchesReceivedByInput;
+  private long[] schemaCountByInput;
+
+  private long batchesOutput;
+  private long recordsOutput;
+
+  private boolean inProcessing = false;
+  private boolean inSetup = false;
+
+  private long processingNanos;
+  private long setupNanos;
+
+  private long processingMark;
+  private long setupMark;
+
+  private long schemas;
+
+  public OperatorStats(OpProfileDef def){
+    this(def.getOperatorId(), def.getOperatorType(), def.getIncomingCount());
+  }
+
+  private OperatorStats(int operatorId, int operatorType, int inputCount) {
+    super();
+    this.operatorId = operatorId;
+    this.operatorType = operatorType;
+    this.recordsReceivedByInput = new long[inputCount];
+    this.batchesReceivedByInput = new long[inputCount];
+  }
+
+  public void startSetup() {
+    assert !inSetup;
+    stopProcessing();
+    inSetup = true;
+    setupMark = System.nanoTime();
+  }
+
+  public void stopSetup() {
+    assert inSetup;
+    startProcessing();
+    setupNanos += System.nanoTime() - setupMark;
+    inSetup = false;
+  }
+
+  public void startProcessing() {
+    assert !inProcessing;
+    processingMark = System.nanoTime();
+    inProcessing = true;
+  }
+
+  public void stopProcessing() {
+    assert inProcessing;
+    processingNanos += System.nanoTime() - processingMark;
+  }
+
+  public void batchReceived(int inputIndex, long records, boolean newSchema) {
+    recordsReceivedByInput[inputIndex] += records;
+    batchesReceivedByInput[inputIndex]++;
+    if(newSchema){
+      schemaCountByInput[inputIndex]++;
+    }
+  }
+
+  public OperatorProfile getProfile() {
+    OperatorProfile.Builder b = OperatorProfile //
+        .newBuilder() //
+        .setOperatorType(operatorType) //
+        .setOperatorId(operatorId) //
+        .setOutputProfile(StreamProfile.newBuilder().setBatches(batchesOutput).setRecords(recordsOutput)) //
+        .setSetupNanos(setupNanos) //
+        .setProcessNanos(processingNanos);
+
+    for(int i = 0; i < recordsReceivedByInput.length; i++){
+      b.addInputProfile(StreamProfile.newBuilder().setBatches(batchesReceivedByInput[i]).setRecords(recordsReceivedByInput[i]).setSchemas(this.schemaCountByInput[i]));
+    }
+
+    for(int i =0; i < longMetrics.allocated.length; i++){
+      if(longMetrics.allocated[i]){
+        b.addMetricBuilder().setMetricId(longMetrics.keys[i]).setLongValue(longMetrics.values[i]);
+      }
+    }
+
+    for(int i =0; i < doubleMetrics.allocated.length; i++){
+      if(doubleMetrics.allocated[i]){
+        b.addMetricBuilder().setMetricId(doubleMetrics.keys[i]).setDoubleValue(doubleMetrics.values[i]);
+      }
+    }
+
+    return b.build();
+  }
+
+  public void addLongStat(MetricDef metric, long value){
+    longMetrics.putOrAdd(metric.metricId(), value, value);
+  }
+
+  public void addDoubleStat(MetricDef metric, double value){
+    doubleMetrics.putOrAdd(metric.metricId(), value, value);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
index a09eb14..4a918b4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
@@ -47,20 +47,20 @@ public abstract class AbstractExchange extends AbstractSingle implements Exchang
 
   protected abstract void setupSenders(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
   protected abstract void setupReceivers(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
-  
+
   @Override
   public final void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException {
     this.senderMajorFragmentId = majorFragmentId;
     setupSenders(senderLocations);
   }
-  
+
 
   @Override
   public final void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
     this.receiverMajorFragmentId = majorFragmentId;
     setupReceivers(receiverLocations);
   }
-  
+
   @Override
   public OperatorCost getAggregateSendCost() {
     return getExchangeCost().getSendCost();
@@ -87,4 +87,10 @@ public abstract class AbstractExchange extends AbstractSingle implements Exchang
     return getExchangeCost().getCombinedCost();
   }
 
+  @Override
+  public int getOperatorType() {
+    throw new UnsupportedOperationException();
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
index 0a5c4fb..4978450 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractGroupScan.java
@@ -82,4 +82,8 @@ public abstract class AbstractGroupScan extends AbstractBase implements GroupSca
     throw new UnsupportedOperationException(String.format("%s does not have exact column value count!", this.getClass().getCanonicalName()));
   }
 
+  @Override
+  public int getOperatorType() {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
index 483c364..4dd2aef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -104,4 +104,7 @@ public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
 
   @JsonProperty("@id")
   public void setOperatorId(int id);
+
+  @JsonIgnore
+  public int getOperatorType();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/BroadcastSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/BroadcastSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/BroadcastSender.java
index 9c0388a..88c40a1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/BroadcastSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/BroadcastSender.java
@@ -21,10 +21,12 @@ package org.apache.drill.exec.physical.config;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.AbstractSender;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import java.util.List;
 
@@ -63,4 +65,11 @@ public class BroadcastSender extends AbstractSender {
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
     return physicalVisitor.visitBroadcastSender(this, value);
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.BROADCAST_SENDER_VALUE;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
index e02bb07..a9e38ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/ExternalSort.java
@@ -20,11 +20,13 @@ package org.apache.drill.exec.physical.config;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import java.util.List;
 
@@ -56,14 +58,14 @@ public class ExternalSort extends Sort {
     long n = childSize.getRecordCount();
     long width = childSize.getRecordSize();
 
-    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
+    //TODO: Magic Number, let's assume 1/10 of data can fit in memory.
     int k = 10;
     long n2 = n/k;
-    double cpuCost = 
-        k * n2 * (Math.log(n2)/Math.log(2)) + // 
+    double cpuCost =
+        k * n2 * (Math.log(n2)/Math.log(2)) + //
         n * (Math.log(k)/Math.log(2));
     double diskCost = n*width*2;
-    
+
     return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
   }
 
@@ -72,7 +74,10 @@ public class ExternalSort extends Sort {
     return new ExternalSort(child, orderings, reverse);
   }
 
-    
-  
-  
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.EXTERNAL_SORT_VALUE;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
index f125d05..99b5f46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -36,7 +37,7 @@ public class Filter extends AbstractSingle {
 
   private final LogicalExpression expr;
   private final float selectivity;
-  
+
   @JsonCreator
   public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("selectivity") float selectivity) {
     super(child);
@@ -67,7 +68,7 @@ public class Filter extends AbstractSingle {
   public Size getSize() {
     return new Size( (long) (child.getSize().getRecordCount()*selectivity), child.getSize().getRecordSize());
   }
-   
+
   @Override
   public SelectionVectorMode getSVMode() {
     if (child.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
@@ -76,5 +77,11 @@ public class Filter extends AbstractSingle {
       return SelectionVectorMode.TWO_BYTE;
     }
   }
-  
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.FILTER_VALUE;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashAggregate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashAggregate.java
index eb77d78..d8fdc4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashAggregate.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.physical.impl.common.HashTableConfig;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -42,7 +43,7 @@ public class HashAggregate extends AbstractSingle {
 
   // configuration parameters for the hash table
   private final HashTableConfig htConfig;
-  
+
   @JsonCreator
   public HashAggregate(@JsonProperty("child") PhysicalOperator child, @JsonProperty("keys") NamedExpression[] groupByExprs, @JsonProperty("exprs") NamedExpression[] aggrExprs, @JsonProperty("cardinality") float cardinality) {
     super(child);
@@ -50,12 +51,12 @@ public class HashAggregate extends AbstractSingle {
     this.aggrExprs = aggrExprs;
     this.cardinality = cardinality;
 
-    int initial_capacity = cardinality > HashTable.DEFAULT_INITIAL_CAPACITY ? 
+    int initial_capacity = cardinality > HashTable.DEFAULT_INITIAL_CAPACITY ?
       (int) cardinality : HashTable.DEFAULT_INITIAL_CAPACITY;
 
-    this.htConfig = new HashTableConfig(initial_capacity,                                        
-                                        HashTable.DEFAULT_LOAD_FACTOR, 
-                                        groupByExprs, 
+    this.htConfig = new HashTableConfig(initial_capacity,
+                                        HashTable.DEFAULT_LOAD_FACTOR,
+                                        groupByExprs,
                                         null /* no probe exprs */) ;
   }
 
@@ -90,8 +91,8 @@ public class HashAggregate extends AbstractSingle {
     int numExprs = getGroupByExprs().length;
 
     double cpuCost = n * numExprs * hashCpuCost;
-    double diskCost = 0;      // for now assume hash table fits in memory 
-        
+    double diskCost = 0;      // for now assume hash table fits in memory
+
     return new OperatorCost(0, (float) diskCost, (float) n*width, (float) cpuCost);
   }
 
@@ -99,7 +100,7 @@ public class HashAggregate extends AbstractSingle {
 	  logger.debug("HashAggregate cost: cpu = {}, disk = {}, memory = {}, network = {}.", HACost.getCpu(), HACost.getDisk(), HACost.getMemory(), HACost.getNetwork());
 	  logger.debug("Streaming aggregate cost: cpu = {}, disk = {}, memory = {}, network = {}.", SACost.getCpu(), SACost.getDisk(), SACost.getMemory(), SACost.getNetwork());
   }
-  
+
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
     return new HashAggregate(child, groupByExprs, aggrExprs, cardinality);
@@ -110,10 +111,13 @@ public class HashAggregate extends AbstractSingle {
     // not a great hack...
     return new Size( (long) (child.getSize().getRecordCount()*cardinality), child.getSize().getRecordSize());
   }
-  
-  
 
-  
-  
-  
+
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.HASH_AGGREGATE_VALUE;
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
index 4ae27b8..1ef7e97 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashJoinPOP.java
@@ -37,9 +37,10 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterators;
+
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.physical.impl.common.HashTableConfig;
-
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.eigenbase.rel.JoinRelType;
 
 @JsonTypeName("hash-join")
@@ -119,4 +120,9 @@ public class HashJoinPOP extends AbstractBase {
             return this;
         }
     }
+
+    @Override
+    public int getOperatorType() {
+      return CoreOperatorType.HASH_JOIN_VALUE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
index 702c787..bdb1362 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
@@ -24,6 +24,7 @@ import org.apache.drill.exec.physical.base.AbstractSender;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -59,11 +60,15 @@ public class HashPartitionSender extends AbstractSender {
   public LogicalExpression getExpr() {
     return expr;
   }
-  
+
   @Override
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
     return physicalVisitor.visitHashPartitionSender(this, value);
   }
-  
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.HASH_PARTITION_SENDER_VALUE;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
index 67bba96..94bffea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
@@ -21,13 +21,14 @@ import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 public class IteratorValidator extends AbstractSingle{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidator.class);
 
   public IteratorValidator(PhysicalOperator child) {
     super(child);
-    
+
   }
 
   @Override
@@ -45,4 +46,8 @@ public class IteratorValidator extends AbstractSingle{
     return new IteratorValidator(child);
   }
 
+  @Override
+  public int getOperatorType() {
+    return -1;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Limit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Limit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Limit.java
index 7d1d485..0db58ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Limit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Limit.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 @JsonTypeName("limit")
@@ -68,4 +69,8 @@ public class Limit extends AbstractSingle {
     return SelectionVectorMode.TWO_BYTE;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.LIMIT_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
index 264ee94..be9cf95 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.physical.base.AbstractBase;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.eigenbase.rel.JoinRelType;
 
 import com.beust.jcommander.internal.Lists;
@@ -115,4 +116,9 @@ public class MergeJoinPOP extends AbstractBase{
     }
 
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.MERGE_JOIN_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergingReceiverPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergingReceiverPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergingReceiverPOP.java
index 549c65c..da5e7e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergingReceiverPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergingReceiverPOP.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractReceiver;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -82,4 +83,8 @@ public class MergingReceiverPOP extends AbstractReceiver{
     return orderings;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.MERGING_RECEIVER_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
index 55632a2..83076a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/OrderedPartitionSender.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.beust.jcommander.internal.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -41,7 +42,7 @@ public class OrderedPartitionSender extends AbstractSender {
   private final FieldReference ref;
   private final List<DrillbitEndpoint> endpoints;
   private final int sendingWidth;
-  
+
   private int recordsToSample;
   private int samplingFactor;
   private float completionFactor;
@@ -90,7 +91,7 @@ public class OrderedPartitionSender extends AbstractSender {
   public OperatorCost getCost() {
     return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
   }
-  
+
   @Override
   public Size getSize() {
     //TODO: This should really change the row width...
@@ -115,4 +116,8 @@ public class OrderedPartitionSender extends AbstractSender {
     return completionFactor;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.ORDERED_PARTITION_SENDER_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
index 0e6b0fd..8a3f5e2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -36,7 +37,7 @@ public class Project extends AbstractSingle{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
 
   private final List<NamedExpression> exprs;
-  
+
   @JsonCreator
   public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
     super(child);
@@ -56,7 +57,7 @@ public class Project extends AbstractSingle{
   public OperatorCost getCost() {
     return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
   }
-  
+
   @Override
   public Size getSize() {
     //TODO: This should really change the row width...
@@ -67,9 +68,14 @@ public class Project extends AbstractSingle{
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
     return new Project(exprs, child);
   }
-  
+
   @Override
   public SelectionVectorMode getSVMode() {
     return child.getSVMode();
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.PROJECT_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
index 676de38..f517f7c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -37,14 +38,14 @@ public class RandomReceiver extends AbstractReceiver{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
 
   private List<DrillbitEndpoint> senders;
-  
+
   @JsonCreator
   public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId,
                         @JsonProperty("senders") List<DrillbitEndpoint> senders) {
     super(oppositeMajorFragmentId);
     this.senders = senders;
   }
-  
+
   @Override
   @JsonProperty("senders")
   public List<DrillbitEndpoint> getProvidingEndpoints() {
@@ -62,7 +63,7 @@ public class RandomReceiver extends AbstractReceiver{
     return new OperatorCost(1,1,1,1);
   }
 
-  
+
   @Override
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
     return physicalVisitor.visitRandomReceiver(this, value);
@@ -74,7 +75,8 @@ public class RandomReceiver extends AbstractReceiver{
     return new Size(1,1);
   }
 
-  
-
-  
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.RANDOM_RECEIVER_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
index 08a6c1b..c8c8f43 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.drill.exec.physical.base.AbstractSender;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -33,7 +34,7 @@ public class RangeSender extends AbstractSender{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
 
   List<EndpointPartition> partitions;
-  
+
   @JsonCreator
   public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
     super(oppositeMajorFragmentId, child);
@@ -55,7 +56,7 @@ public class RangeSender extends AbstractSender{
   public static class EndpointPartition{
     private final PartitionRange range;
     private final DrillbitEndpoint endpoint;
-    
+
     @JsonCreator
     public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
       super();
@@ -69,4 +70,9 @@ public class RangeSender extends AbstractSender{
       return endpoint;
     }
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.RANGE_SENDER_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
index c4d78f9..4f5cb47 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -30,6 +30,7 @@ import org.apache.drill.exec.physical.base.Root;
 import org.apache.drill.exec.physical.base.Size;
 import org.apache.drill.exec.physical.base.Store;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JacksonInject;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -103,5 +104,9 @@ public class Screen extends AbstractStore {
     return physicalVisitor.visitScreen(this, value);
   }
 
-  
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.SCREEN_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
index 5e891ec..13cbbe2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
@@ -22,6 +22,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -32,7 +33,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 public class SelectionVectorRemover extends AbstractSingle {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorRemover.class);
-  
+
   @JsonCreator
   public SelectionVectorRemover(@JsonProperty("child") PhysicalOperator child) {
     super(child);
@@ -62,4 +63,9 @@ public class SelectionVectorRemover extends AbstractSingle {
   public SelectionVectorMode getSVMode() {
     return SelectionVectorMode.NONE;
   }
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.SELECTION_VECTOR_REMOVER_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
index 9894164..82962ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
@@ -26,6 +26,7 @@ import org.apache.drill.exec.physical.base.AbstractSender;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -40,7 +41,7 @@ public class SingleSender extends AbstractSender {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
 
   private final DrillbitEndpoint destination;
-  
+
   @JsonCreator
   public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
     super(oppositeMajorFragmentId, child);
@@ -68,11 +69,15 @@ public class SingleSender extends AbstractSender {
   public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
     return physicalVisitor.visitSingleSender(this, value);
   }
- 
+
 
   public DrillbitEndpoint getDestination() {
     return destination;
   }
- 
-  
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.SINGLE_SENDER_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
index c9ac137..d7415d8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
@@ -34,10 +35,10 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 @JsonTypeName("sort")
 public class Sort extends AbstractSingle{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
-  
+
   protected final List<Ordering> orderings;
   protected boolean reverse = false;
-  
+
   @JsonCreator
   public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<Ordering> orderings, @JsonProperty("reverse") boolean reverse) {
     super(child);
@@ -64,14 +65,14 @@ public class Sort extends AbstractSingle{
     long n = childSize.getRecordCount();
     long width = childSize.getRecordSize();
 
-    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
+    //TODO: Magic Number, let's assume 1/10 of data can fit in memory.
     int k = 10;
     long n2 = n/k;
-    double cpuCost = 
-        k * n2 * (Math.log(n2)/Math.log(2)) + // 
+    double cpuCost =
+        k * n2 * (Math.log(n2)/Math.log(2)) + //
         n * (Math.log(k)/Math.log(2));
     double diskCost = n*width*2;
-    
+
     return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
   }
 
@@ -84,6 +85,9 @@ public class Sort extends AbstractSingle{
   public SelectionVectorMode getSVMode() {
     return SelectionVectorMode.FOUR_BYTE;
   }
-  
-  
+
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.OLD_SORT_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
index 2dcdce1..6e06c24 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/StreamingAggregate.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -37,7 +38,7 @@ public class StreamingAggregate extends AbstractSingle {
   private final NamedExpression[] exprs;
 
   private final float cardinality;
-  
+
   @JsonCreator
   public StreamingAggregate(@JsonProperty("child") PhysicalOperator child, @JsonProperty("keys") NamedExpression[] keys, @JsonProperty("exprs") NamedExpression[] exprs, @JsonProperty("cardinality") float cardinality) {
     super(child);
@@ -74,10 +75,10 @@ public class StreamingAggregate extends AbstractSingle {
     // not a great hack...
     return new Size( (long) (child.getSize().getRecordCount()*cardinality), child.getSize().getRecordSize());
   }
-  
-  
 
-  
-  
-  
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.STREAMING_AGGREGATE_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/TopN.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/TopN.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/TopN.java
index 79c5782..dfd142a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/TopN.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/TopN.java
@@ -20,12 +20,14 @@ package org.apache.drill.exec.physical.config;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+
 import org.apache.drill.common.logical.data.Order.Ordering;
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import java.util.List;
 
@@ -64,14 +66,14 @@ public class TopN extends Sort {
     long n = childSize.getRecordCount();
     long width = childSize.getRecordSize();
 
-    //TODO: Magic Number, let's assume 1/10 of data can fit in memory. 
+    //TODO: Magic Number, let's assume 1/10 of data can fit in memory.
     int k = 10;
     long n2 = n/k;
-    double cpuCost = 
-        k * n2 * (Math.log(n2)/Math.log(2)) + // 
+    double cpuCost =
+        k * n2 * (Math.log(n2)/Math.log(2)) + //
         n * (Math.log(k)/Math.log(2));
     double diskCost = n*width*2;
-    
+
     return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
   }
 
@@ -80,7 +82,9 @@ public class TopN extends Sort {
     return new TopN(child, orderings, reverse, limit);
   }
 
-    
-  
-  
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.TOP_N_SORT_VALUE;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Trace.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Trace.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Trace.java
index a81d3e9..f4ba842 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Trace.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Trace.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
@@ -67,4 +68,9 @@ public class Trace extends AbstractSingle {
     protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
         return new Trace(child, traceTag);
     }
+
+    @Override
+    public int getOperatorType() {
+      return CoreOperatorType.TRACE_VALUE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
index fcefc37..522100f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
@@ -20,8 +20,10 @@ package org.apache.drill.exec.physical.config;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
+
 import org.apache.drill.exec.physical.OperatorCost;
 import org.apache.drill.exec.physical.base.*;
+import org.apache.drill.exec.proto.UserBitShared.CoreOperatorType;
 
 import java.util.List;
 
@@ -56,4 +58,8 @@ public class Union extends AbstractMultiple {
     return cost;
   }
 
+  @Override
+  public int getOperatorType() {
+    return CoreOperatorType.UNION_VALUE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
index 81a4d58..1113af4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WriterRecordBatch.java
@@ -69,88 +69,97 @@ public class WriterRecordBatch extends AbstractRecordBatch<Writer> {
 
   @Override
   public IterOutcome next() {
-    if(processed) {
-      // if the upstream record batch is already processed and next() is called by
-      // downstream then return NONE to indicate completion
-      return IterOutcome.NONE;
-    }
+    stats.startProcessing();
+    try{
 
-    // process the complete upstream in one next() call
-    IterOutcome upstream;
-    do {
-      upstream = incoming.next();
-      if(first && upstream == IterOutcome.OK)
-        upstream = IterOutcome.OK_NEW_SCHEMA;
-      first = false;
-
-      switch(upstream) {
-        case NOT_YET:
-        case NONE:
-        case STOP:
-          cleanup();
-          if (upstream == IterOutcome.STOP)
-            return upstream;
-          break;
-
-        case OK_NEW_SCHEMA:
-          try{
-            setupNewSchema();
-          }catch(Exception ex){
-            kill();
-            logger.error("Failure during query", ex);
-            context.fail(ex);
-            return IterOutcome.STOP;
-          }
-          // fall through.
-        case OK:
-          try {
-            counter += eventBasedRecordWriter.write();
-            logger.debug("Total records written so far: {}", counter);
-          } catch(IOException ex) {
-            throw new RuntimeException(ex);
-          }
-
-          for(VectorWrapper v : incoming)
-            v.getValueVector().clear();
-
-          break;
-
-        default:
-          throw new UnsupportedOperationException();
+      if(processed) {
+        // if the upstream record batch is already processed and next() is called by
+        // downstream then return NONE to indicate completion
+        return IterOutcome.NONE;
       }
-    } while(upstream != IterOutcome.NONE);
-
-    // Create two vectors for:
-    //   1. Fragment unique id.
-    //   2. Summary: currently contains number of records written.
-    MaterializedField fragmentIdField = MaterializedField.create(SchemaPath.getSimplePath("Fragment"), Types.required(MinorType.VARCHAR));
-    MaterializedField summaryField = MaterializedField.create(SchemaPath.getSimplePath("Number of records written"), Types.required(MinorType.BIGINT));
-
-    VarCharVector fragmentIdVector = (VarCharVector) TypeHelper.getNewVector(fragmentIdField, context.getAllocator());
-    AllocationHelper.allocate(fragmentIdVector, 1, TypeHelper.getSize(Types.required(MinorType.VARCHAR)));
-    BigIntVector summaryVector = (BigIntVector) TypeHelper.getNewVector(summaryField, context.getAllocator());
-    AllocationHelper.allocate(summaryVector, 1, TypeHelper.getSize(Types.required(MinorType.VARCHAR)));
-
 
-    container.add(fragmentIdVector);
-    container.add(summaryVector);
-    container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
-
-    fragmentIdVector.getMutator().setSafe(0, fragmentUniqueId.getBytes());
-    fragmentIdVector.getMutator().setValueCount(1);
-    summaryVector.getMutator().setSafe(0, counter);
-    summaryVector.getMutator().setValueCount(1);
-
-    container.setRecordCount(1);
-    processed = true;
+      // process the complete upstream in one next() call
+      IterOutcome upstream;
+      do {
+        upstream = next(incoming);
+        if(first && upstream == IterOutcome.OK)
+          upstream = IterOutcome.OK_NEW_SCHEMA;
+        first = false;
+
+        switch(upstream) {
+          case NOT_YET:
+          case NONE:
+          case STOP:
+            cleanup();
+            if (upstream == IterOutcome.STOP)
+              return upstream;
+            break;
+
+          case OK_NEW_SCHEMA:
+            try{
+              setupNewSchema();
+            }catch(Exception ex){
+              kill();
+              logger.error("Failure during query", ex);
+              context.fail(ex);
+              return IterOutcome.STOP;
+            }
+            // fall through.
+          case OK:
+            try {
+              counter += eventBasedRecordWriter.write();
+              logger.debug("Total records written so far: {}", counter);
+            } catch(IOException ex) {
+              throw new RuntimeException(ex);
+            }
+
+            for(VectorWrapper v : incoming)
+              v.getValueVector().clear();
+
+            break;
+
+          default:
+            throw new UnsupportedOperationException();
+        }
+      } while(upstream != IterOutcome.NONE);
+
+      // Create two vectors for:
+      //   1. Fragment unique id.
+      //   2. Summary: currently contains number of records written.
+      MaterializedField fragmentIdField = MaterializedField.create(SchemaPath.getSimplePath("Fragment"), Types.required(MinorType.VARCHAR));
+      MaterializedField summaryField = MaterializedField.create(SchemaPath.getSimplePath("Number of records written"), Types.required(MinorType.BIGINT));
+
+      VarCharVector fragmentIdVector = (VarCharVector) TypeHelper.getNewVector(fragmentIdField, context.getAllocator());
+      AllocationHelper.allocate(fragmentIdVector, 1, TypeHelper.getSize(Types.required(MinorType.VARCHAR)));
+      BigIntVector summaryVector = (BigIntVector) TypeHelper.getNewVector(summaryField, context.getAllocator());
+      AllocationHelper.allocate(summaryVector, 1, TypeHelper.getSize(Types.required(MinorType.VARCHAR)));
+
+
+      container.add(fragmentIdVector);
+      container.add(summaryVector);
+      container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
+
+      fragmentIdVector.getMutator().setSafe(0, fragmentUniqueId.getBytes());
+      fragmentIdVector.getMutator().setValueCount(1);
+      summaryVector.getMutator().setSafe(0, counter);
+      summaryVector.getMutator().setValueCount(1);
+
+      container.setRecordCount(1);
+      processed = true;
+
+      return IterOutcome.OK_NEW_SCHEMA;
+    }finally{
+      stats.stopProcessing();
+    }
 
-    return IterOutcome.OK_NEW_SCHEMA;
   }
 
   protected void setupNewSchema() throws Exception {
     try {
       // update the schema in RecordWriter
+      stats.startSetup();
       recordWriter.updateSchema(incoming.getSchema());
+      stats.stopSetup();
     } catch(IOException ex) {
       throw new RuntimeException("Failed to update schema in RecordWriter", ex);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/5472140a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
index aa6cd54..1a22f3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/aggregate/HashAggBatch.java
@@ -76,14 +76,14 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
   private TypedFieldId[] groupByOutFieldIds ;
   private TypedFieldId[] aggrOutFieldIds ;      // field ids for the outgoing batch
 
-  private final GeneratorMapping UPDATE_AGGR_INSIDE = 
-    GeneratorMapping.create("setupInterior" /* setup method */, "updateAggrValuesInternal" /* eval method */, 
+  private final GeneratorMapping UPDATE_AGGR_INSIDE =
+    GeneratorMapping.create("setupInterior" /* setup method */, "updateAggrValuesInternal" /* eval method */,
                             "resetValues" /* reset */, "cleanup" /* cleanup */) ;
 
-  private final GeneratorMapping UPDATE_AGGR_OUTSIDE = 
-    GeneratorMapping.create("setupInterior" /* setup method */, "outputRecordValues" /* eval method */, 
+  private final GeneratorMapping UPDATE_AGGR_OUTSIDE =
+    GeneratorMapping.create("setupInterior" /* setup method */, "outputRecordValues" /* eval method */,
                             "resetValues" /* reset */, "cleanup" /* cleanup */) ;
-   
+
   private final MappingSet UpdateAggrValuesMapping = new MappingSet("incomingRowIdx" /* read index */, "outRowIdx" /* write index */, "htRowIdx" /* workspace index */, "incoming" /* read container */, "outgoing" /* write container */, "aggrValuesContainer" /* workspace container */, UPDATE_AGGR_INSIDE, UPDATE_AGGR_OUTSIDE, UPDATE_AGGR_INSIDE);
 
 
@@ -100,72 +100,78 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
 
   @Override
   public IterOutcome next() {
-
-    // this is only called on the first batch. Beyond this, the aggregator manages batches.
-    if (aggregator == null) {
-      IterOutcome outcome = incoming.next();
-      logger.debug("Next outcome of {}", outcome);
-      switch (outcome) {
-      case NONE:
-      case NOT_YET:
-      case STOP:
-        return outcome;
-      case OK_NEW_SCHEMA:
-        if (!createAggregator()){
-          done = true;
-          return IterOutcome.STOP;
+    stats.startProcessing();
+    try{
+      // this is only called on the first batch. Beyond this, the aggregator manages batches.
+      if (aggregator == null) {
+        IterOutcome outcome = next(incoming);
+        logger.debug("Next outcome of {}", outcome);
+        switch (outcome) {
+        case NONE:
+        case NOT_YET:
+        case STOP:
+          return outcome;
+        case OK_NEW_SCHEMA:
+          if (!createAggregator()){
+            done = true;
+            return IterOutcome.STOP;
+          }
+          break;
+        case OK:
+          throw new IllegalStateException("You should never get a first batch without a new schema");
+        default:
+          throw new IllegalStateException(String.format("unknown outcome %s", outcome));
         }
-        break;
-      case OK:
-        throw new IllegalStateException("You should never get a first batch without a new schema");
-      default:
-        throw new IllegalStateException(String.format("unknown outcome %s", outcome));
       }
-    }
 
 
-    if (aggregator.allFlushed()) {
-      return IterOutcome.NONE;
-    }
+      if (aggregator.allFlushed()) {
+        return IterOutcome.NONE;
+      }
+
+      logger.debug("Starting aggregator doWork; incoming record count = {} ", incoming.getRecordCount());
 
-    logger.debug("Starting aggregator doWork; incoming record count = {} ", incoming.getRecordCount());
-    
-    while(true){
-      AggOutcome out = aggregator.doWork();
-      logger.debug("Aggregator response {}, records {}", out, aggregator.getOutputCount());
-      switch(out){
-      case CLEANUP_AND_RETURN:
-        container.clear();
-        aggregator.cleanup();
-        done = true;
-        return aggregator.getOutcome();
-      case RETURN_OUTCOME:
-        return aggregator.getOutcome();
-      case UPDATE_AGGREGATOR:
-        aggregator = null;
-        if(!createAggregator()){
-          return IterOutcome.STOP;
+      while(true){
+        AggOutcome out = aggregator.doWork();
+        logger.debug("Aggregator response {}, records {}", out, aggregator.getOutputCount());
+        switch(out){
+        case CLEANUP_AND_RETURN:
+          container.clear();
+          aggregator.cleanup();
+          done = true;
+          return aggregator.getOutcome();
+        case RETURN_OUTCOME:
+          return aggregator.getOutcome();
+        case UPDATE_AGGREGATOR:
+          aggregator = null;
+          if(!createAggregator()){
+            return IterOutcome.STOP;
+          }
+          continue;
+        default:
+          throw new IllegalStateException(String.format("Unknown state %s.", out));
         }
-        continue;
-      default:
-        throw new IllegalStateException(String.format("Unknown state %s.", out));
       }
+      }finally{
+      stats.stopProcessing();
     }
-    
   }
 
   /**
    * Creates a new Aggregator based on the current schema. If setup fails, this method is responsible for cleaning up
    * and informing the context of the failure state, as well is informing the upstream operators.
-   * 
+   *
    * @return true if the aggregator was setup successfully. false if there was a failure.
    */
   private boolean createAggregator() {
     logger.debug("Creating new aggregator.");
     try{
+      stats.startSetup();
       this.aggregator = createAggregatorInternal();
+      stats.stopSetup();
       return true;
     }catch(SchemaChangeException | ClassTransformationException | IOException ex){
+      stats.stopSetup();
       context.fail(ex);
       container.clear();
       incoming.kill();
@@ -181,12 +187,12 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
     container.clear();
     List<VectorAllocator> keyAllocators = Lists.newArrayList();
     List<VectorAllocator> valueAllocators = Lists.newArrayList();
-    
+
     int numGroupByExprs = (popConfig.getGroupByExprs() != null) ? popConfig.getGroupByExprs().length : 0;
     int numAggrExprs = (popConfig.getAggrExprs() != null) ? popConfig.getAggrExprs().length : 0;
     aggrExprs = new LogicalExpression[numAggrExprs];
     groupByOutFieldIds = new TypedFieldId[numGroupByExprs];
-    aggrOutFieldIds = new TypedFieldId[numAggrExprs];    
+    aggrOutFieldIds = new TypedFieldId[numAggrExprs];
 
     ErrorCollector collector = new ErrorCollectorImpl();
 
@@ -201,18 +207,18 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       keyAllocators.add(VectorAllocator.getAllocator(vv, 50));
 
-      // add this group-by vector to the output container 
+      // add this group-by vector to the output container
       groupByOutFieldIds[i] = container.add(vv);
     }
 
     for(i = 0; i < numAggrExprs; i++){
       NamedExpression ne = popConfig.getAggrExprs()[i];
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(ne.getExpr(), incoming, collector, context.getFunctionRegistry() );
-  
+
       if(collector.hasErrors()) throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
 
       if(expr == null) continue;
-      
+
       final MaterializedField outputField = MaterializedField.create(ne.getRef(), expr.getMajorType());
       ValueVector vv = TypeHelper.getNewVector(outputField, oContext.getAllocator());
       valueAllocators.add(VectorAllocator.getAllocator(vv, 50));
@@ -229,10 +235,10 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
     HashAggregator agg = context.getImplementationClass(top);
 
     agg.setup(popConfig, context, oContext.getAllocator(), incoming, this,
-              aggrExprs, 
+              aggrExprs,
               cgInner.getWorkspaceTypes(),
               groupByOutFieldIds,
-              keyAllocators.toArray(new VectorAllocator[keyAllocators.size()]), 
+              keyAllocators.toArray(new VectorAllocator[keyAllocators.size()]),
               valueAllocators.toArray(new VectorAllocator[valueAllocators.size()]));
 
     return agg;
@@ -268,12 +274,12 @@ public class HashAggBatch extends AbstractRecordBatch<HashAggregate> {
       cg.getBlock("getVectorIndex")._return(var.invoke("getIndex").arg(JExpr.direct("recordIndex")));;
       return;
     }
-     
+
     default:
       throw new IllegalStateException();
-      
+
     }
-   
+
   }
 
   @Override


[22/24] git commit: Disable SV2 for Project until wildcard is fixed. Remove spurious test.

Posted by ja...@apache.org.
Disable SV2 for Project until wildcard is fixed.  Remove spurious test.


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/3eb4f751
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/3eb4f751
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/3eb4f751

Branch: refs/heads/diagnostics2
Commit: 3eb4f7514eaf85125b2b01368009cd7150e7f728
Parents: 864ebde
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 17:06:03 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 17:06:03 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/physical/ProjectPrel.java      |  2 +-
 .../apache/drill/exec/server/TestJersey.java    | 65 --------------------
 2 files changed, 1 insertion(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3eb4f751/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
index 55f9f32..88cbb50 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -75,7 +75,7 @@ public class ProjectPrel extends DrillProjectRelBase implements Prel{
 
   @Override
   public SelectionVectorMode[] getSupportedEncodings() {
-    return SelectionVectorMode.NONE_AND_TWO;
+    return SelectionVectorMode.DEFAULT;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3eb4f751/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
deleted file mode 100644
index d47e559..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestJersey.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.server;
-
-import org.apache.drill.exec.client.DrillClient;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.glassfish.hk2.utilities.binding.AbstractBinder;
-import org.glassfish.jersey.jackson.JacksonFeature;
-import org.glassfish.jersey.server.ResourceConfig;
-import org.glassfish.jersey.server.ServerProperties;
-import org.glassfish.jersey.servlet.ServletContainer;
-import org.junit.Test;
-
-public class TestJersey {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJersey.class);
-
-  @Test
-  public void x() throws Exception {
-    Server server = new Server(8112);
-    ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS);
-    context.setContextPath("/");
-    server.setHandler(context);
-    ServletHolder h = new ServletHolder(new ServletContainer(new MyApplication()));
-    h.setInitParameter(ServerProperties.PROVIDER_PACKAGES, "org.apache.drill.exec.server");
-    h.setInitOrder(1);
-    context.addServlet(h, "/*");
-    server.start();
-    server.join();
-
-  }
-
-  public class MyApplication extends ResourceConfig {
-
-    public MyApplication() {
-        registerClasses(HelloResource.class);
-        register(JacksonFeature.class);
-        property(ServerProperties.METAINF_SERVICES_LOOKUP_DISABLE, true);
-        register(new AbstractBinder() {
-            @Override
-            protected void configure() {
-              this.bind(new DrillClient()).to(DrillClient.class);
-            }
-        });
-    }
-}
-
-
-}


[20/24] git commit: Remove references to hive12 Guava.

Posted by ja...@apache.org.
Remove references to hive12 Guava.


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

Branch: refs/heads/diagnostics2
Commit: fbdca8f90c82745218962bddec8c8d55d3ee5c4c
Parents: 79054a8
Author: Jacques Nadeau <ja...@apache.org>
Authored: Wed May 21 13:32:23 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Wed May 21 13:32:23 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/drill/exec/ops/FragmentStats.java     | 2 +-
 .../org/apache/drill/exec/planner/common/DrillWriterRelBase.java   | 2 +-
 .../java/org/apache/drill/exec/planner/logical/DrillJoinRel.java   | 2 +-
 .../apache/drill/exec/planner/physical/PhysicalPlanCreator.java    | 2 +-
 .../java/org/apache/drill/exec/planner/physical/WriterPrel.java    | 2 +-
 .../org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java     | 2 +-
 .../main/java/org/apache/drill/exec/record/MaterializedField.java  | 2 +-
 .../org/apache/drill/exec/store/easy/json/JSONRecordReader2.java   | 2 +-
 .../java/org/apache/drill/exec/vector/BaseDataValueVector.java     | 2 +-
 .../main/java/org/apache/drill/exec/vector/BaseValueVector.java    | 2 +-
 .../main/java/org/apache/drill/exec/vector/complex/MapVector.java  | 2 +-
 .../org/apache/drill/exec/vector/complex/RepeatedListVector.java   | 2 +-
 .../apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java   | 2 +-
 .../src/test/java/org/apache/drill/exec/TestOpSerialization.java   | 2 +-
 .../java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java | 2 +-
 .../apache/drill/exec/vector/complex/writer/TestJsonReader.java    | 2 +-
 16 files changed, 16 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
index 24a02b5..d667794 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentStats.java
@@ -23,7 +23,7 @@ import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
 
 import com.codahale.metrics.MetricRegistry;
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class FragmentStats {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
index 03431d6..7ea98a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillWriterRelBase.java
@@ -29,7 +29,7 @@ import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 /** Base class for logical and physical Writer implemented in Drill. */
 public abstract class DrillWriterRelBase extends SingleRel implements DrillRelNode {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
index a5593e7..e687f5a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
@@ -42,7 +42,7 @@ import org.eigenbase.rex.RexUtil;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.util.Pair;
 
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 /**
  * Join implemented in Drill.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
index f4189e4..35841ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -31,7 +31,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.physical.explain.PrelSequencer.OpId;
 
 import com.google.common.collect.Lists;
-import com.google.hive12.common.collect.Maps;
+import com.google.common.collect.Maps;
 
 
 public class PhysicalPlanCreator {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
index 233b20b..70d0b1b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/WriterPrel.java
@@ -33,7 +33,7 @@ import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.sql.type.SqlTypeName;
 
 import com.google.common.collect.ImmutableList;
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class WriterPrel extends DrillWriterRelBase implements Prel {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
index b074ba0..26f5af2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
@@ -46,7 +46,7 @@ import org.eigenbase.sql.validate.SqlMonotonicity;
 import org.eigenbase.sql.validate.SqlValidator;
 import org.eigenbase.sql.validate.SqlValidatorScope;
 
-import com.google.hive12.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList;
 
 public class DrillSqlAggOperator extends SqlAggFunction {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 439552f..f4d00e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -27,7 +27,7 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class MaterializedField{
   private SchemaPath path;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader2.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader2.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader2.java
index bb52a20..b4f3880 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader2.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader2.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class JSONRecordReader2 implements RecordReader{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JSONRecordReader2.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
index 9641e6a..e70f406 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseDataValueVector.java
@@ -26,7 +26,7 @@ import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.DeadBuf;
 import org.apache.drill.exec.record.MaterializedField;
 
-import com.google.hive12.common.collect.Iterators;
+import com.google.common.collect.Iterators;
 
 public abstract class BaseDataValueVector extends BaseValueVector{
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
index 7a61475..f968435 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BaseValueVector.java
@@ -27,7 +27,7 @@ import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.MaterializedField;
 
-import com.google.hive12.common.collect.Iterators;
+import com.google.common.collect.Iterators;
 
 public abstract class BaseValueVector implements ValueVector{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
index 9b2e119..9d71727 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/MapVector.java
@@ -46,7 +46,7 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import com.carrotsearch.hppc.IntObjectOpenHashMap;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class MapVector extends AbstractContainerVector {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MapVector.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
index 1f65761..ef8aef8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/RepeatedListVector.java
@@ -46,7 +46,7 @@ import org.apache.drill.exec.vector.complex.impl.RepeatedListReaderImpl;
 import org.apache.drill.exec.vector.complex.reader.FieldReader;
 
 import com.google.common.collect.Lists;
-import com.google.hive12.common.base.Preconditions;
+import com.google.common.base.Preconditions;
 
 
 public class RepeatedListVector extends AbstractContainerVector implements RepeatedFixedWidthVector{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
index c6ea75b..0c7378a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/impl/ComplexWriterImpl.java
@@ -24,7 +24,7 @@ import org.apache.drill.exec.vector.complex.StateTool;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter;
 import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
 
-import com.google.hive12.common.base.Preconditions;
+import com.google.common.base.Preconditions;
 
 public class ComplexWriterImpl extends AbstractFieldWriter implements ComplexWriter{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ComplexWriterImpl.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
index 906361a..29502be 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java
@@ -36,7 +36,7 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.store.mock.MockSubScanPOP;
 import org.junit.Test;
 
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class TestOpSerialization {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOpSerialization.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
index 13322f1..9d067da 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/ISpan.java
@@ -38,7 +38,7 @@ import org.infinispan.configuration.global.GlobalConfigurationBuilder;
 import org.infinispan.manager.DefaultCacheManager;
 import org.infinispan.manager.EmbeddedCacheManager;
 
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class ISpan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ISpan.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/fbdca8f9/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
index 7bc4a40..1f3b540 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
@@ -37,7 +37,7 @@ import org.junit.Test;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.base.Charsets;
-import com.google.hive12.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 public class TestJsonReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJsonReader.class);