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 2013/08/23 02:06:44 UTC

[1/8] git commit: DRILL-156 Union (all) pop implementation Fixes to TestSimpleFilter

Updated Branches:
  refs/heads/master c50135256 -> ac8590d2b


DRILL-156 Union (all) pop implementation
Fixes to TestSimpleFilter


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

Branch: refs/heads/master
Commit: acf47818eb545d6f4109aec1189017754552adcf
Parents: c501352
Author: immars <im...@gmail.com>
Authored: Thu Aug 22 15:21:22 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:08:28 2013 -0700

----------------------------------------------------------------------
 .../exec/physical/base/AbstractMultiple.java    |  36 +++++
 .../physical/base/AbstractPhysicalVisitor.java  |  16 +--
 .../exec/physical/base/PhysicalVisitor.java     |  12 +-
 .../drill/exec/physical/config/Union.java       |  42 ++++++
 .../drill/exec/physical/impl/ImplCreator.java   |   8 ++
 .../drill/exec/physical/impl/ScreenCreator.java |   1 +
 .../physical/impl/union/UnionBatchCreator.java  |  23 ++++
 .../physical/impl/union/UnionRecordBatch.java   | 130 +++++++++++++++++++
 .../physical/impl/TestSimpleFragmentRun.java    |   4 +-
 .../physical/impl/filter/TestSimpleFilter.java  |   2 +-
 .../physical/impl/union/TestSimpleUnion.java    |  68 ++++++++++
 .../src/test/resources/filter/test1.json        |   8 +-
 .../src/test/resources/union/test1.json         |  57 ++++++++
 13 files changed, 384 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractMultiple.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractMultiple.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractMultiple.java
new file mode 100644
index 0000000..247595b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractMultiple.java
@@ -0,0 +1,36 @@
+package org.apache.drill.exec.physical.base;
+
+import com.google.common.collect.Iterators;
+
+import java.util.Iterator;
+
+/**
+ * Describes an operator that expects more than one children operators as its input.
+ */
+public abstract class AbstractMultiple extends AbstractBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractMultiple.class);  
+  
+  protected final PhysicalOperator[] children;
+
+  protected AbstractMultiple(PhysicalOperator[] children) {
+    this.children = children;
+  }
+
+  public PhysicalOperator[] getChildren() {
+    return children;
+  }
+  
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.forArray(children);
+  }
+  
+  public Size getSize() {
+    Size size = new Size(0,0);
+    for(PhysicalOperator child:children){
+      size.add(child.getSize());
+    }
+    return size;
+  }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
index 3b58803..3cd0d7c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -17,16 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.base;
 
-import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.HashPartitionSender;
-import org.apache.drill.exec.physical.config.HashToRandomExchange;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.physical.config.RandomReceiver;
-import org.apache.drill.exec.physical.config.RangeSender;
-import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.physical.config.SingleSender;
-import org.apache.drill.exec.physical.config.Sort;
-import org.apache.drill.exec.physical.config.UnionExchange;
+import org.apache.drill.exec.physical.config.*;
 
 public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
@@ -37,6 +28,11 @@ public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> impleme
   }
 
   @Override
+  public T visitUnion(Union union, X value) throws E {
+    return visitOp(union, value);
+  }
+
+  @Override
   public T visitFilter(Filter filter, X value) throws E{
     return visitOp(filter, value);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
index 8e09e3a..39fc105 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -17,16 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.exec.physical.base;
 
-import org.apache.drill.exec.physical.config.Filter;
-import org.apache.drill.exec.physical.config.HashPartitionSender;
-import org.apache.drill.exec.physical.config.HashToRandomExchange;
-import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.physical.config.RandomReceiver;
-import org.apache.drill.exec.physical.config.RangeSender;
-import org.apache.drill.exec.physical.config.Screen;
-import org.apache.drill.exec.physical.config.SingleSender;
-import org.apache.drill.exec.physical.config.Sort;
-import org.apache.drill.exec.physical.config.UnionExchange;
+import org.apache.drill.exec.physical.config.*;
 
 /**
  * Visitor class designed to traversal of a operator tree.  Basis for a number of operator manipulations including fragmentation and materialization.
@@ -44,6 +35,7 @@ public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
   public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
 
   public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
+  public RETURN visitUnion(Union union, EXTRA value) throws EXCEP;
   public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
   public RETURN visitSort(Sort sort, EXTRA value) throws EXCEP;
   public RETURN visitSender(Sender sender, EXTRA value) throws EXCEP;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
new file mode 100644
index 0000000..73ac9b0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Union.java
@@ -0,0 +1,42 @@
+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 java.util.List;
+
+@JsonTypeName("union")
+
+public class Union extends AbstractMultiple {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
+
+  @JsonCreator
+  public Union(@JsonProperty("children") PhysicalOperator[] children) {
+    super(children);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitUnion(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    return new Union(children.toArray(new PhysicalOperator[children.size()]));
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    OperatorCost cost = new OperatorCost(0,0,0,0);
+    for (int i = 0; i < children.length; i++) {
+      PhysicalOperator child = children[i];
+      cost.add(child.getCost());
+    }
+    return cost;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
index 2ce5c28..e69aeae 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -34,11 +34,13 @@ import org.apache.drill.exec.physical.config.Screen;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.physical.config.SingleSender;
 import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.*;
 import org.apache.drill.exec.physical.impl.filter.FilterBatchCreator;
 import org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderCreator;
 import org.apache.drill.exec.physical.impl.project.ProjectBatchCreator;
 import org.apache.drill.exec.physical.impl.sort.SortBatchCreator;
 import org.apache.drill.exec.physical.impl.svremover.SVRemoverCreator;
+import org.apache.drill.exec.physical.impl.union.UnionBatchCreator;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.store.json.JSONScanBatchCreator;
 import org.apache.drill.exec.store.json.JSONSubScan;
@@ -65,6 +67,7 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   private SingleSenderCreator ssc = new SingleSenderCreator();
   private ProjectBatchCreator pbc = new ProjectBatchCreator();
   private FilterBatchCreator fbc = new FilterBatchCreator();
+  private UnionBatchCreator unionbc = new UnionBatchCreator();
   private SVRemoverCreator svc = new SVRemoverCreator();
   private SortBatchCreator sbc = new SortBatchCreator();
   private RootExec root = null;
@@ -130,6 +133,11 @@ public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentCo
   }
 
   @Override
+  public RecordBatch visitUnion(Union union, FragmentContext context) throws ExecutionSetupException {
+    return unionbc.getBatch(context, union, getChildren(union, context));
+  }
+
+  @Override
   public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
     root = ssc.getRoot(context, op, getChildren(op, context));
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
index 9b31407..b66921d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -108,6 +108,7 @@ public class ScreenCreator implements RootCreator<Screen>{
           QueryWritableBatch batch = materializer.convertNext(true);
           connection.sendResult(listener, batch);
         }
+
         return false;
       }
       case OK_NEW_SCHEMA:

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionBatchCreator.java
new file mode 100644
index 0000000..1945139
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionBatchCreator.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.physical.impl.union;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Union;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+
+public class UnionBatchCreator implements BatchCreator<Union>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, Union config, List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.size() >= 1);
+    return new UnionRecordBatch(config, children, context);
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionRecordBatch.java
new file mode 100644
index 0000000..b77d4f4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionRecordBatch.java
@@ -0,0 +1,130 @@
+package org.apache.drill.exec.physical.impl.union;
+
+import com.google.common.collect.Lists;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Union;
+import org.apache.drill.exec.record.*;
+import org.apache.drill.exec.record.selection.SelectionVector2;
+import org.apache.drill.exec.vector.ValueVector;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public class UnionRecordBatch extends AbstractRecordBatch<Union> {
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionRecordBatch.class);
+
+  private final List<RecordBatch> incoming;
+  private SelectionVector2 sv;
+  private Iterator<RecordBatch> incomingIterator = null;
+  private RecordBatch current = null;
+  private ArrayList<TransferPair> transfers;
+  private int outRecordCount;
+
+  public UnionRecordBatch(Union config, List<RecordBatch> children, FragmentContext context) {
+    super(config, context);
+    this.incoming = children;
+    this.incomingIterator = incoming.iterator();
+    current = incomingIterator.next();
+    sv = null;
+  }
+
+
+  @Override
+  public int getRecordCount() {
+    return outRecordCount;
+  }
+
+  @Override
+  public void kill() {
+    if(current != null){
+      current.kill();
+      current = null;
+    }
+    for(;incomingIterator.hasNext();){
+      incomingIterator.next().kill();
+    }
+  }
+
+  @Override
+  protected void killIncoming() {
+    for (int i = 0; i < incoming.size(); i++) {
+      RecordBatch in = incoming.get(i);
+      in.kill();
+    }
+  }
+
+
+  @Override
+  public SelectionVector2 getSelectionVector2() {
+    return sv;
+  }
+
+  @Override
+  public IterOutcome next() {
+    if (current == null) { // end of iteration
+      return IterOutcome.NONE;
+    }
+    IterOutcome upstream = current.next();
+    logger.debug("Upstream... {}", upstream);
+    while (upstream == IterOutcome.NONE) {
+      if (!incomingIterator.hasNext()) {
+        current = null;
+        return IterOutcome.NONE;
+      }
+      current = incomingIterator.next();
+      upstream = current.next();
+    }
+    switch (upstream) {
+      case NONE:
+        throw new IllegalArgumentException("not possible!");
+      case NOT_YET:
+      case STOP:
+        return upstream;
+      case OK_NEW_SCHEMA:
+        setupSchema();
+        // fall through.
+      case OK:
+        doTransfer();
+        return upstream; // change if upstream changed, otherwise normal.
+      default:
+        throw new UnsupportedOperationException();
+    }
+  }
+
+  private void doTransfer() {
+    outRecordCount = current.getRecordCount();
+    if (container.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE) {
+      this.sv = current.getSelectionVector2();
+    }
+    for (TransferPair transfer : transfers) {
+      transfer.transfer();
+    }
+
+    for (VectorWrapper<?> vw : this.container) {
+      ValueVector.Mutator m = vw.getValueVector().getMutator();
+      m.setValueCount(outRecordCount);
+    }
+
+  }
+
+  private void setupSchema() {
+    if (container != null) {
+      container.clear();
+    }
+    transfers = Lists.newArrayList();
+
+    for (VectorWrapper<?> vw : current) {
+      TransferPair pair = vw.getValueVector().getTransferPair();
+      container.add(pair.getTo());
+      transfers.add(pair);
+    }
+    container.buildSchema(current.getSchema().getSelectionVectorMode());
+  }
+
+  @Override
+  public WritableBatch getWritableBatch() {
+    return WritableBatch.get(this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index 6587237..a2612d5 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -59,7 +59,9 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
     // run query.
     bit.run();
     client.connect();
-    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+      String path = "/physical_test2.json";
+//      String path = "/filter/test1.json";
+    List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(path), Charsets.UTF_8));
 
     // look at records
     RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
index afb0b3a..26c8a25 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/filter/TestSimpleFilter.java
@@ -48,7 +48,7 @@ public class TestSimpleFilter {
     FragmentContext context = new FragmentContext(bitContext, FragmentHandle.getDefaultInstance(), connection, null, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
     while(exec.next()){
-      assertEquals(50, exec.getSelectionVector2().getCount());
+      assertEquals(50, exec.getRecordCount());
     }
     
     if(context.getFailureCause() != null){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
new file mode 100644
index 0000000..1dbefb1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/union/TestSimpleUnion.java
@@ -0,0 +1,68 @@
+package org.apache.drill.exec.physical.impl.union;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import com.yammer.metrics.MetricRegistry;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.SimpleRootExec;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.rpc.user.UserServer;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.junit.After;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestSimpleUnion {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleUnion.class);
+  DrillConfig c = DrillConfig.create();
+  
+  
+  @Test
+  public void testUnion(@Injectable final DrillbitContext bitContext, @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+    
+    new NonStrictExpectations(){{
+      bitContext.getMetrics(); result = new MetricRegistry("test");
+      bitContext.getAllocator(); result = BufferAllocator.getAllocator(c);
+    }};
+    
+    
+    PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+    PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/union/test1.json"), Charsets.UTF_8));
+    FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
+    FragmentContext context = new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, null, registry);
+    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+    
+    int[] counts = new int[]{100,50};
+    int i=0;
+    while(exec.next()){
+      System.out.println("iteration count:" + exec.getRecordCount());
+      assertEquals(counts[i++], exec.getRecordCount());
+    }
+    
+    if(context.getFailureCause() != null){
+      throw context.getFailureCause();
+    }
+    assertTrue(!context.isFailed());
+    
+  }
+  
+  @After
+  public void tearDown() throws Exception{
+    // pause to get logger to catch up.
+    Thread.sleep(1000);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
index 4f82145..7d05928 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/filter/test1.json
@@ -26,8 +26,14 @@
             expr: "alternate()"
         },
         {
+          @id:4,
+          child:2,
+          pop: "selection-vector-remover"
+          
+        },
+        {
             @id: 3,
-            child: 2,
+            child: 4,
             pop: "screen"
         }
     ]

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/acf47818/sandbox/prototype/exec/java-exec/src/test/resources/union/test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/union/test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/union/test1.json
new file mode 100644
index 0000000..5b33232
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/union/test1.json
@@ -0,0 +1,57 @@
+{
+    head:{
+        type:"APACHE_DRILL_PHYSICAL",
+        version:"1",
+        generator:{
+            type:"manual"
+        }
+    },
+	graph:[
+        {
+            @id:1,
+            pop:"mock-sub-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },
+        {
+            @id:2,
+            pop:"mock-sub-scan",
+            url: "http://apache.org",
+            entries:[
+            	{records: 100, types: [
+            	  {name: "blue", type: "INT", mode: "REQUIRED"},
+            	  {name: "red", type: "BIGINT", mode: "REQUIRED"},
+            	  {name: "green", type: "INT", mode: "REQUIRED"}
+            	]}
+            ]
+        },        
+        {
+            @id:3,
+            child: 2,
+            pop:"filter",
+            expr: "alternate()"
+        },
+        {
+            @id:4,
+            children: [1,3],
+            pop:"union"
+        },
+        {
+          @id:5,
+          child:4,
+          pop: "selection-vector-remover"
+          
+        },
+        {
+            @id: 6,
+            child: 5,
+            pop: "screen"
+        }
+    ]
+}
\ No newline at end of file


[8/8] git commit: DRILL-166 Update CodeGenerator code to support more generic signatures.

Posted by ja...@apache.org.
DRILL-166 Update CodeGenerator code to support more generic signatures.


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

Branch: refs/heads/master
Commit: ac8590d2b78b4b5f97199af48e425e8f4d205068
Parents: 0fc89a3
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat Aug 10 13:02:59 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:59:25 2013 -0700

----------------------------------------------------------------------
 .../drill/common/expression/IfExpression.java   |  29 +-
 .../common/expression/LogicalExpression.java    |   2 +-
 .../drill/common/expression/SchemaPath.java     |   7 +
 .../common/expression/ValueExpressions.java     |  21 +
 .../expression/visitors/AggregateChecker.java   |   2 +-
 .../expression/visitors/ConstantChecker.java    |   3 +-
 sandbox/prototype/exec/java-exec/pom.xml        |  10 +
 .../exec/compile/TemplateClassDefinition.java   |  36 +-
 .../exec/compile/sig/CodeGeneratorArgument.java |  23 ++
 .../exec/compile/sig/CodeGeneratorMethod.java   |  60 +++
 .../compile/sig/CodeGeneratorSignature.java     |   5 +
 .../sig/ConstantExpressionIdentifier.java       | 114 +++++
 .../compile/sig/DefaultGeneratorSignature.java  |  22 +
 .../exec/compile/sig/GeneratorMapping.java      |  40 ++
 .../drill/exec/compile/sig/MappingSet.java      |  67 +++
 .../drill/exec/compile/sig/Signature.java       |  14 +
 .../drill/exec/compile/sig/SignatureHolder.java |  57 +++
 .../drill/exec/compile/sig/VVReadBatch.java     |  18 +
 .../drill/exec/compile/sig/VVWriteBatch.java    |  18 +
 .../apache/drill/exec/expr/CodeGenerator.java   | 172 ++++----
 .../drill/exec/expr/EvaluationVisitor.java      | 412 ++++++++++++-------
 .../exec/expr/ExpressionTreeMaterializer.java   |   2 +-
 .../exec/expr/HoldingContainerExpression.java   |  46 +++
 .../exec/expr/ValueVectorReadExpression.java    |   9 +
 .../exec/expr/ValueVectorWriteExpression.java   |   9 +
 .../drill/exec/expr/annotations/MethodMap.java  |  21 +
 .../drill/exec/expr/fn/DrillAggFuncHolder.java  |  36 ++
 .../drill/exec/expr/fn/DrillFuncHolder.java     | 102 +++++
 .../apache/drill/exec/expr/fn/FunctionBody.java |  12 +
 .../drill/exec/expr/fn/FunctionConverter.java   |  14 +-
 .../drill/exec/expr/fn/FunctionHolder.java      | 110 +----
 .../apache/drill/exec/ops/FragmentContext.java  |   8 -
 .../physical/impl/filter/FilterSignature.java   |  14 +
 .../exec/physical/impl/filter/Filterer.java     |   3 +-
 .../impl/filter/ReturnValueExpression.java      |  10 +-
 .../PartitionSenderRootExec.java                |  11 +-
 .../impl/partitionsender/Partitioner.java       |   2 +-
 .../PartitionerInnerSignature.java              |  17 +
 .../physical/impl/project/ProjectEvaluator.java |   2 +-
 .../exec/physical/impl/project/Projector.java   |   3 +-
 .../physical/impl/sort/ReadIndexRewriter.java   |   2 +-
 .../exec/physical/impl/sort/SortBatch.java      |  19 +-
 .../exec/physical/impl/sort/SortSignature.java  |  20 +
 .../exec/physical/impl/sort/SortTemplate.java   |   8 +-
 .../drill/exec/physical/impl/sort/Sorter.java   |   2 +-
 .../exec/physical/impl/svremover/Copier.java    |   6 +-
 .../impl/svremover/RemovingRecordBatch.java     |   4 +-
 .../drill/exec/record/NullExpression.java       |  13 +-
 .../exec/compile/TestClassTransformation.java   |   2 +-
 .../physical/impl/TestHashToRandomExchange.java |  13 +-
 .../record/ExpressionTreeMaterializerTest.java  |  58 +--
 51 files changed, 1302 insertions(+), 408 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/IfExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/IfExpression.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/IfExpression.java
index 6f2e4a8..426aa6a 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/IfExpression.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/IfExpression.java
@@ -18,6 +18,7 @@
 package org.apache.drill.common.expression;
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.drill.common.expression.IfExpression.IfCondition;
@@ -28,9 +29,12 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
 import com.google.common.collect.UnmodifiableIterator;
 
-public class IfExpression extends LogicalExpressionBase implements Iterable<IfCondition>{
+public class IfExpression extends LogicalExpressionBase{
 	static final Logger logger = LoggerFactory.getLogger(IfExpression.class);
 	
 	public final ImmutableList<IfCondition> conditions;
@@ -106,9 +110,22 @@ public class IfExpression extends LogicalExpressionBase implements Iterable<IfCo
 	}
 
 
-	@Override
-	public UnmodifiableIterator<IfCondition> iterator() {
-		return conditions.iterator();
-	}
-	
+  public Iterable<IfCondition> conditionIterable(){
+    
+    return ImmutableList.copyOf(conditions);
+  }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    List<LogicalExpression> children = Lists.newLinkedList();
+    
+    for(IfCondition ic : conditions){
+      children.add(ic.condition);
+      children.add(ic.expression);
+    }
+    children.add(this.elseExpression);
+    return children.iterator();
+  }
+
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
index 165f409..ac19e01 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
@@ -43,7 +43,7 @@ import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 
 //@JsonDeserialize(using = LogicalExpression.De.class)  // Excluded as we need to register this with the DrillConfig.
 @JsonSerialize(using = LogicalExpression.Se.class)
-public interface LogicalExpression {
+public interface LogicalExpression extends Iterable<LogicalExpression>{
   static final Logger logger = LoggerFactory.getLogger(LogicalExpression.class);
 
   public abstract MajorType getMajorType();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index 6f1a733..5910c30 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -17,6 +17,7 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
+import java.util.Iterator;
 import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -26,6 +27,7 @@ import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
 
+import com.google.common.collect.Iterators;
 import com.google.protobuf.DescriptorProtos.UninterpretedOption.NamePart;
 
 public class SchemaPath extends LogicalExpressionBase {
@@ -132,6 +134,11 @@ public class SchemaPath extends LogicalExpressionBase {
   }
 
   @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
   public String toString() {
     return "SchemaPath [rootSegment=" + rootSegment + "]";
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index d613ebe..33c4612 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -17,12 +17,16 @@
  ******************************************************************************/
 package org.apache.drill.common.expression;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
 
+import com.google.common.collect.Iterators;
+
 public class ValueExpressions {
 
   public static LogicalExpression getNumericExpression(String s, ExpressionPosition ep) {
@@ -54,6 +58,12 @@ public class ValueExpressions {
     }
 
     protected abstract V parseValue(String s);
+    
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      return Iterators.emptyIterator();
+    }
+
 
   }
 
@@ -109,6 +119,11 @@ public class ValueExpressions {
       return visitor.visitDoubleConstant(this, value);
     }
 
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      return Iterators.emptyIterator();
+    }
+
   }
 
   public static class LongExpression extends LogicalExpressionBase {
@@ -135,6 +150,12 @@ public class ValueExpressions {
     public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
       return visitor.visitLongConstant(this, value);
     }
+    
+    @Override
+    public Iterator<LogicalExpression> iterator() {
+      return Iterators.emptyIterator();
+    }
+
   }
 
   public static class QuotedString extends ValueExpression<String> {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
index 4d99b15..aa44281 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/AggregateChecker.java
@@ -54,7 +54,7 @@ public final class AggregateChecker extends SimpleExprVisitor<Boolean>{
 
   @Override
   public Boolean visitIfExpression(IfExpression ifExpr) {
-    for(IfCondition c : ifExpr){
+    for(IfCondition c : ifExpr.conditions){
       if(c.condition.accept(this, null) || c.expression.accept(this, null)) return true;
     }
     return ifExpr.elseExpression.accept(this, null);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
index 038fb85..f8984e9 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/visitors/ConstantChecker.java
@@ -47,9 +47,10 @@ public final class ConstantChecker extends SimpleExprVisitor<Boolean>{
 
   @Override
   public Boolean visitIfExpression(IfExpression ifExpr) {
-    for(IfCondition c : ifExpr){
+    for(IfCondition c : ifExpr.conditions){
       if(!c.condition.accept(this, null) || !c.expression.accept(this, null)) return false;
     }
+    if(!ifExpr.elseExpression.accept(this, null)) return false;
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/pom.xml b/sandbox/prototype/exec/java-exec/pom.xml
index a1680e8..e7d3f16 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -23,6 +23,11 @@
       <version>4.1</version>
     </dependency>
     <dependency>
+      <groupId>com.thoughtworks.paranamer</groupId>
+      <artifactId>paranamer</artifactId>
+      <version>2.5.6</version>
+    </dependency>
+    <dependency>
       <groupId>com.sun.codemodel</groupId>
       <artifactId>codemodel</artifactId>
       <version>2.6</version>
@@ -59,6 +64,11 @@
       <version>3.0.0-BETA1</version>
     </dependency>
     <dependency>
+      <groupId>javax.inject</groupId>
+      <artifactId>javax.inject</artifactId>
+      <version>1</version>
+    </dependency>    
+    <dependency>
       <groupId>org.apache.drill</groupId>
       <artifactId>common</artifactId>
       <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
index 20ef361..bd23844 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/TemplateClassDefinition.java
@@ -17,21 +17,36 @@
  ******************************************************************************/
 package org.apache.drill.exec.compile;
 
-
+import org.apache.drill.exec.compile.sig.CodeGeneratorSignature;
+import org.apache.drill.exec.compile.sig.DefaultGeneratorSignature;
+import org.apache.drill.exec.compile.sig.SignatureHolder;
 
 public class TemplateClassDefinition<T>{
   
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TemplateClassDefinition.class);
+  
   private final Class<T> externalInterface;
   private final String templateClassName;
   private final Class<?> internalInterface;
-  private final Class<?> evalReturnType;
-
-  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, Class<?> evalReturnType) {
+  private final SignatureHolder signature;
+  
+  public TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface) {
+    this(externalInterface, templateClassName, internalInterface, DefaultGeneratorSignature.class);
+  }
+  
+  public <X extends CodeGeneratorSignature> TemplateClassDefinition(Class<T> externalInterface, String templateClassName, Class<?> internalInterface, Class<X> signature) {
     super();
     this.externalInterface = externalInterface;
     this.templateClassName = templateClassName;
     this.internalInterface = internalInterface;
-    this.evalReturnType = evalReturnType;
+    SignatureHolder holder = null;
+    try{
+      holder = new SignatureHolder(signature);
+    }catch(Exception ex){
+      logger.error("Failure while trying to build signature holder for signature. {}", signature.getSimpleName(), ex);
+    }
+    this.signature = holder;
+
   }
 
   public Class<T> getExternalInterface() {
@@ -46,8 +61,15 @@ public class TemplateClassDefinition<T>{
     return templateClassName;
   }
 
-  public Class<?> getEvalReturnType() {
-    return evalReturnType;
+  public SignatureHolder getSignature(){
+    return signature;
   }
+
+  @Override
+  public String toString() {
+    return "TemplateClassDefinition [externalInterface=" + externalInterface + ", templateClassName="
+        + templateClassName + ", internalInterface=" + internalInterface + ", signature=" + signature + "]";
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorArgument.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorArgument.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorArgument.java
new file mode 100644
index 0000000..f1715b2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorArgument.java
@@ -0,0 +1,23 @@
+package org.apache.drill.exec.compile.sig;
+
+
+public class CodeGeneratorArgument {
+  
+  private final String name;
+  private final Class<?> type;
+  
+  public CodeGeneratorArgument(String name, Class<?> type) {
+    super();
+    this.name = name;
+    this.type = type;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Class<?> getType() {
+    return type;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
new file mode 100644
index 0000000..11dcbe9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorMethod.java
@@ -0,0 +1,60 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Iterator;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterators;
+import com.thoughtworks.paranamer.AnnotationParanamer;
+import com.thoughtworks.paranamer.Paranamer;
+
+public class CodeGeneratorMethod implements Iterable<CodeGeneratorArgument>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeGeneratorMethod.class);
+  
+  private final String methodName;
+  private final Class<?> returnType;
+  private final CodeGeneratorArgument[] arguments;
+  private final Class<?>[] exs;
+  private final Method underlyingMethod;
+  
+  public CodeGeneratorMethod(Method m){
+    this.underlyingMethod = m;
+    this.methodName = m.getName();
+    this.returnType = m.getReturnType();
+//    Paranamer para = new BytecodeReadingParanamer();
+    Paranamer para = new AnnotationParanamer();
+    String[] parameterNames = para.lookupParameterNames(m, true);
+    if(parameterNames == null) throw new RuntimeException(String.format("Unable to read the parameter names for method %s.  This is likely due to the class files not including the appropriate debugging information.  Look up java -g for more information.", m));
+    Class<?>[] types = m.getParameterTypes();
+    if(parameterNames.length != types.length) throw new RuntimeException(String.format("Unexpected number of parameter names %s.  Expected %s on method %s.", Arrays.toString(parameterNames), Arrays.toString(types), m.toGenericString()));
+    arguments = new CodeGeneratorArgument[parameterNames.length];
+    for(int i =0 ; i < parameterNames.length; i++){
+      arguments[i] = new CodeGeneratorArgument(parameterNames[i], types[i]);
+    }
+    exs = m.getExceptionTypes();
+  }
+  
+  public String getMethodName() {
+    return methodName;
+  }
+  public Class<?> getReturnType() {
+    return returnType;
+  }
+
+  public Iterable<Class<?>> getThrowsIterable(){
+    return ImmutableList.copyOf(exs);
+  }
+  
+  @Override
+  public Iterator<CodeGeneratorArgument> iterator() {
+    return Iterators.forArray(arguments);
+  }
+
+  @Override
+  public String toString() {
+    return "CodeGeneratorMethod [" + underlyingMethod.toGenericString() + "]";
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorSignature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorSignature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorSignature.java
new file mode 100644
index 0000000..f294fe7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/CodeGeneratorSignature.java
@@ -0,0 +1,5 @@
+package org.apache.drill.exec.compile.sig;
+
+
+public interface CodeGeneratorSignature {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
new file mode 100644
index 0000000..62e6837
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -0,0 +1,114 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
+import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+
+import com.beust.jcommander.internal.Lists;
+
+public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, IdentityHashMap<LogicalExpression, Object>, RuntimeException>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConstantExpressionIdentifier.class);
+
+  private ConstantExpressionIdentifier(){}
+  
+  /**
+   * Get a list of expressions that mark boundaries into a constant space.
+   * @param e
+   * @return
+   */
+  public static Set<LogicalExpression> getConstantExpressionSet(LogicalExpression e){
+    IdentityHashMap<LogicalExpression, Object> map = new IdentityHashMap<>();
+    ConstantExpressionIdentifier visitor = new ConstantExpressionIdentifier();
+    
+
+    if(e.accept(visitor, map) && map.isEmpty()){
+      // if we receive a constant value here but the map is empty, this means the entire tree is a constant.
+      // note, we can't use a singleton collection here because we need an identity set.
+      map.put(e, true);
+      return map.keySet();
+    }else if(map.isEmpty()){
+      // so we don't continue to carry around a map, we let it go here and simply return an empty set.
+      return Collections.emptySet();
+    }else{
+      return map.keySet();  
+    }
+  }
+
+  private boolean checkChildren(LogicalExpression e, IdentityHashMap<LogicalExpression, Object> value, boolean transmitsConstant){
+    List<LogicalExpression> constants = Lists.newLinkedList();
+    boolean constant = true;
+
+    for(LogicalExpression child : e){
+      if(child.accept(this, value)){
+        constants.add(child);
+      }else{
+        constant = false;
+      }
+    }
+    
+    // if one or more clauses isn't constant, this isn't constant.  this also isn't a constant if it operates on a set.
+    if(!constant || !transmitsConstant){
+      for(LogicalExpression c: constants){
+        value.put(c, true);
+      }
+    }
+    return constant && transmitsConstant;
+  }
+  
+  @Override
+  public Boolean visitFunctionCall(FunctionCall call, IdentityHashMap<LogicalExpression, Object> value){
+    return checkChildren(call, value, !call.getDefinition().isAggregating());
+  }
+
+  
+  @Override
+  public Boolean visitIfExpression(IfExpression ifExpr, IdentityHashMap<LogicalExpression, Object> value){
+    return checkChildren(ifExpr, value, true);
+  }
+
+  @Override
+  public Boolean visitSchemaPath(SchemaPath path, IdentityHashMap<LogicalExpression, Object> value){
+    return false;
+  }
+  
+  @Override
+  public Boolean visitLongConstant(LongExpression intExpr, IdentityHashMap<LogicalExpression, Object> value){
+    value.put(intExpr, true);
+    return true;
+  }
+
+  @Override
+  public Boolean visitDoubleConstant(DoubleExpression dExpr, IdentityHashMap<LogicalExpression, Object> value){
+    value.put(dExpr, true);
+    return true;
+  }
+
+  @Override
+  public Boolean visitBooleanConstant(BooleanExpression e, IdentityHashMap<LogicalExpression, Object> value){
+    value.put(e, true);
+    return true;
+  }
+
+  @Override
+  public Boolean visitQuotedStringConstant(QuotedString e, IdentityHashMap<LogicalExpression, Object> value){
+    value.put(e, true);
+    return true;
+  }
+
+  @Override
+  public Boolean visitUnknown(LogicalExpression e, IdentityHashMap<LogicalExpression, Object> value){
+    return checkChildren(e, value, false);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/DefaultGeneratorSignature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/DefaultGeneratorSignature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/DefaultGeneratorSignature.java
new file mode 100644
index 0000000..dc56206
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/DefaultGeneratorSignature.java
@@ -0,0 +1,22 @@
+package org.apache.drill.exec.compile.sig;
+
+import static org.apache.drill.exec.compile.sig.GeneratorMapping.GM;
+
+import javax.inject.Named;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface DefaultGeneratorSignature extends CodeGeneratorSignature{
+  
+  public static final GeneratorMapping DEFAULT_SCALAR_MAP = GM("doSetup", "doEval", null, null);
+  public static final GeneratorMapping DEFAULT_CONSTANT_MAP = GM("doSetup", "doSetup", null, null);
+  
+  public static final MappingSet DEFAULT_MAPPING = new MappingSet("inIndex", "outIndex", DEFAULT_SCALAR_MAP, DEFAULT_SCALAR_MAP);
+
+  public void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing);
+  public void doEval(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex);
+  
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
new file mode 100644
index 0000000..0633446
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/GeneratorMapping.java
@@ -0,0 +1,40 @@
+package org.apache.drill.exec.compile.sig;
+
+public class GeneratorMapping {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(GeneratorMapping.class);
+
+  private String setup;
+  private String eval;
+  private String reset;
+  private String cleanup;
+  
+  public GeneratorMapping(String setup, String eval, String reset, String cleanup) {
+    super();
+    this.setup = setup;
+    this.eval = eval;
+    this.reset = reset;
+    this.cleanup = cleanup;
+  }
+
+  public static GeneratorMapping GM(String setup, String eval, String reset, String cleanup){
+    return new GeneratorMapping(setup, eval, reset, cleanup);
+  }
+
+  public String getSetup() {
+    return setup;
+  }
+
+  public String getEval() {
+    return eval;
+  }
+
+  public String getReset() {
+    return reset;
+  }
+
+  public String getCleanup() {
+    return cleanup;
+  }
+ 
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java
new file mode 100644
index 0000000..e6dcf4e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/MappingSet.java
@@ -0,0 +1,67 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+
+
+public class MappingSet {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MappingSet.class);
+  
+  private GeneratorMapping constant;
+  private GeneratorMapping[] mappings;
+  private int mappingIndex;
+  private GeneratorMapping current;
+  private JExpression readIndex;
+  private JExpression writeIndex;
+  
+  
+  public MappingSet(String readIndex, String writeIndex, GeneratorMapping... mappings) {
+    this.readIndex = JExpr.direct(readIndex);
+    this.writeIndex = JExpr.direct(writeIndex);
+    Preconditions.checkArgument(mappings.length >= 2);
+    this.constant = mappings[0];
+    this.mappings = Arrays.copyOfRange(mappings, 1, mappings.length);
+    this.current = this.mappings[0];
+  }
+
+  public void enterConstant(){
+//    assert constant != current;
+//    current = constant;
+  }
+  
+  public void exitConstant(){
+//    assert constant == current;
+//    current = mappings[mappingIndex];
+  }
+  
+  
+  public void enterChild(){
+    assert current == mappings[mappingIndex];
+    mappingIndex++;
+    if(mappingIndex >= mappings.length) throw new IllegalStateException("This generator does not support mappings beyond");
+    current = mappings[mappingIndex];
+  }
+  
+  public void exitChild(){
+    assert current == mappings[mappingIndex];
+    mappingIndex--;
+    if(mappingIndex < 0) throw new IllegalStateException("You tried to traverse higher than the provided mapping provides.");
+  }
+  
+  public GeneratorMapping getCurrentMapping(){
+    return current;
+  }
+  
+  public JExpression getValueWriteIndex(){
+    return writeIndex;
+  }
+  
+  public JExpression getValueReadIndex(){
+    return readIndex;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/Signature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/Signature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/Signature.java
new file mode 100644
index 0000000..ea49d96
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/Signature.java
@@ -0,0 +1,14 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Describes the field will provide output from the given function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface Signature {
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
new file mode 100644
index 0000000..68b2498
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/SignatureHolder.java
@@ -0,0 +1,57 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Map;
+
+import com.beust.jcommander.internal.Maps;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterators;
+
+public class SignatureHolder implements Iterable<CodeGeneratorMethod>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SignatureHolder.class);
+  
+  private final CodeGeneratorMethod[] methods;
+  private final Map<String, Integer> methodMap;
+  
+  public <T extends CodeGeneratorSignature> SignatureHolder(Class<T> signature){
+    Method[] reflectMethods = signature.getDeclaredMethods();
+    methods = new CodeGeneratorMethod[reflectMethods.length];
+    Map<String, Integer> newMap = Maps.newHashMap(); 
+    
+    for(int i =0; i < methods.length; i++){
+      methods[i] = new CodeGeneratorMethod(reflectMethods[i]);
+      newMap.put(methods[i].getMethodName(), i);
+    }
+    
+    methodMap = ImmutableMap.copyOf(newMap);
+    
+  }
+
+  @Override
+  public Iterator<CodeGeneratorMethod> iterator() {
+    return Iterators.forArray(methods);
+  }
+  
+  public int size(){
+    return methods.length;
+  }
+  
+  public int get(String method){
+    Integer meth =  methodMap.get(method);
+    if(meth == null){
+      throw new IllegalStateException(String.format("Unknown method requested of name %s.", method));
+    }
+    return meth;
+  }
+
+  @Override
+  public String toString() {
+    final int maxLen = 10;
+    return "SignatureHolder [methods="
+        + (methods != null ? Arrays.asList(methods).subList(0, Math.min(methods.length, maxLen)) : null) + "]";
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVReadBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVReadBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVReadBatch.java
new file mode 100644
index 0000000..c20c6d8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVReadBatch.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Marker annotation to determine which fields should be included as parameters for the function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface VVReadBatch {
+  int value();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVWriteBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVWriteBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVWriteBatch.java
new file mode 100644
index 0000000..be07c6d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/VVWriteBatch.java
@@ -0,0 +1,18 @@
+package org.apache.drill.exec.compile.sig;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Marker annotation to determine which fields should be included as parameters for the function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface VVWriteBatch {
+  int value();
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
index c37e75d..d2e4cf3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/CodeGenerator.java
@@ -8,13 +8,15 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.compile.sig.CodeGeneratorArgument;
+import org.apache.drill.exec.compile.sig.CodeGeneratorMethod;
+import org.apache.drill.exec.compile.sig.DefaultGeneratorSignature;
+import org.apache.drill.exec.compile.sig.GeneratorMapping;
+import org.apache.drill.exec.compile.sig.MappingSet;
+import org.apache.drill.exec.compile.sig.SignatureHolder;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.impl.partitionsender.OutgoingRecordBatch;
-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.TypeHelper;
 
 import com.google.common.base.Preconditions;
@@ -24,22 +26,19 @@ import com.sun.codemodel.JClassAlreadyExistsException;
 import com.sun.codemodel.JCodeModel;
 import com.sun.codemodel.JDefinedClass;
 import com.sun.codemodel.JExpr;
-import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JFieldRef;
 import com.sun.codemodel.JMethod;
 import com.sun.codemodel.JMod;
 import com.sun.codemodel.JType;
 import com.sun.codemodel.JVar;
 
-public class CodeGenerator<T> {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeGenerator.class);
+public class CodeGenerator<T>{
   
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CodeGenerator.class);
+
   public JDefinedClass clazz;
-  private JBlock parentEvalBlock;
-  private JBlock parentSetupBlock;
-  private JBlock currentEvalBlock;
-  private JBlock currentSetupBlock;
-  private final EvaluationVisitor evaluationVisitor;
+  private JBlock[] parentBlocks;
+  private JBlock[] childBlocks;
   private final TemplateClassDefinition<T> definition;
   private JCodeModel model;
   private int index = 0;
@@ -47,25 +46,69 @@ public class CodeGenerator<T> {
   private String className;
   private String fqcn;
   private String packageName = "org.apache.drill.exec.test.generated";
+  private final SignatureHolder sig;
+  private MappingSet mappings;
+  private final EvaluationVisitor evaluationVisitor;
+  
   
   public CodeGenerator(TemplateClassDefinition<T> definition, FunctionImplementationRegistry funcRegistry) {
-    super();
-    className = "Gen" + classCreator.incrementAndGet();
-    fqcn = packageName + "." + className;
+    this(DefaultGeneratorSignature.DEFAULT_MAPPING, definition, funcRegistry);
+  }
+  
+  public CodeGenerator(MappingSet mappingSet, TemplateClassDefinition<T> definition, FunctionImplementationRegistry funcRegistry) {
+    Preconditions.checkNotNull(definition.getSignature(), "The signature for defintion %s was incorrectly initialized.", definition);
+    this.sig = definition.getSignature();
+    this.mappings = mappingSet;
+    this.className = "Gen" + classCreator.incrementAndGet();
+    this.fqcn = packageName + "." + className;
     try{
       this.definition = definition;
       this.model = new JCodeModel();
       this.clazz = model._package(packageName)._class(className);
       clazz._implements(definition.getInternalInterface());
-      this.parentEvalBlock = new JBlock();
-      this.parentSetupBlock = new JBlock();
+      parentBlocks = new JBlock[sig.size()];
+      for(int i =0; i < sig.size(); i++){
+        parentBlocks[i] = new JBlock(false, false);
+      }
+      childBlocks = new JBlock[sig.size()];
       this.evaluationVisitor = new EvaluationVisitor(funcRegistry);
+      rotateBlock();
     } catch (JClassAlreadyExistsException e) {
       throw new IllegalStateException(e);
     }
   }
   
-
+  public MappingSet getMappingSet(){
+    return mappings;
+  }
+  
+  public void setMappingSet(MappingSet mappings){
+    this.mappings = mappings;
+  }
+  
+  private GeneratorMapping getCurrentMapping(){
+    return mappings.getCurrentMapping();
+  }
+  
+  private JBlock getBlock(String methodName){
+    JBlock blk = this.childBlocks[sig.get(methodName)];
+    Preconditions.checkNotNull(blk, "Requested method name of %s was not available for signature %s.",  methodName, this.sig);
+    return blk;
+  }
+  
+  public JBlock getSetupBlock(){
+    return getBlock(getCurrentMapping().getSetup());
+  }
+  public JBlock getEvalBlock(){
+    return getBlock(getCurrentMapping().getEval());
+  }
+//  public JBlock getResetBlock(){
+//    return getBlock(getCurrentMapping().getReset());
+//  }
+//  public JBlock getCleanupBlock(){
+//    return getBlock(getCurrentMapping().getCleanup());
+//  }
+    
   public JVar declareVectorValueSetupAndMember(String batchName, TypedFieldId fieldId){
     Class<?> valueVectorClass = TypeHelper.getValueVectorClass(fieldId.getType().getMinorType(), fieldId.getType().getMode());
     JClass vvClass = model.ref(valueVectorClass);
@@ -78,7 +121,6 @@ public class CodeGenerator<T> {
     
     JVar vv = declareClassField("vv", retClass);
     JClass t = model.ref(SchemaChangeException.class);
-    JType wrapperClass = model.ref(VectorWrapper.class);
     JType objClass = model.ref(Object.class);
     JBlock b = getSetupBlock();
     JVar obj = b.decl( //
@@ -99,87 +141,51 @@ public class CodeGenerator<T> {
   }
 
   public HoldingContainer addExpr(LogicalExpression ex){
+    return addExpr(ex, true);
+  }
+  
+  public HoldingContainer addExpr(LogicalExpression ex, boolean rotate){
     logger.debug("Adding next write {}", ex);
-    rotateBlock();
-    return ex.accept(evaluationVisitor, this);
+    if(rotate) rotateBlock();
+    return evaluationVisitor.addExpr(ex, this);
   }
   
   public void rotateBlock(){
-    currentEvalBlock = new JBlock();
-    parentEvalBlock.add(currentEvalBlock);
-    currentSetupBlock = new JBlock();
-    parentSetupBlock.add(currentSetupBlock);
-
+    for(int i =0; i < childBlocks.length; i++){
+      this.childBlocks[i] = new JBlock(true, true);
+      
+      this.parentBlocks[i].add(childBlocks[i]);
+    }
   }
   
-  public JBlock getBlock() {
-    return currentEvalBlock;
-  }
-
   public String getMaterializedClassName(){
     return fqcn;
   }
-  
-  public JBlock getSetupBlock(){
-    return currentSetupBlock;
-  }
-  
-  
+    
   public TemplateClassDefinition<T> getDefinition() {
     return definition;
   }
 
   public String generate() throws IOException{
+    int i =0;
+    for(CodeGeneratorMethod method : sig){
+      JMethod m = clazz.method(JMod.PUBLIC, model._ref(method.getReturnType()), method.getMethodName());
+      for(CodeGeneratorArgument arg : method){
+        m.param(arg.getType(), arg.getName());
+      }
+      for(Class<?> c : method.getThrowsIterable()){
+        m._throws(model.ref(c));
+      }
 
-    {
-      //setup method
-      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, "doSetup");
-      m.param(model._ref(FragmentContext.class), "context");
-      m.param(model._ref(RecordBatch.class), "incoming");
-      m.param(model._ref(RecordBatch.class), "outgoing");
       m._throws(SchemaChangeException.class);
-      m.body().add(parentSetupBlock);
-    }
-    
-    {
-      // eval method.
-      JType ret = definition.getEvalReturnType() == null ? model.VOID : model._ref(definition.getEvalReturnType());
-      JMethod m = clazz.method(JMod.PUBLIC, ret, "doEval");  
-      m.param(model.INT, "inIndex");
-      m.param(model.INT, "outIndex");
-      m.body().add(parentEvalBlock);
+      m.body().add(this.parentBlocks[i++]);
     }
     
     SingleClassStringWriter w = new SingleClassStringWriter();
     model.build(w);
     return w.getCode().toString();
   }
-
-  public String generateMultipleOutputs() throws IOException{
-
-    {
-      //setup method
-      JMethod m = clazz.method(JMod.PUBLIC, model.VOID, "doSetup");
-      m.param(model._ref(FragmentContext.class), "context");
-      m.param(model._ref(RecordBatch.class), "incoming");
-      m.param(model._ref(OutgoingRecordBatch.class).array(), "outgoing");
-      m._throws(SchemaChangeException.class);
-      m.body().add(parentSetupBlock);
-    }
-
-    {
-      // eval method.
-      JType ret = definition.getEvalReturnType() == null ? model.VOID : model._ref(definition.getEvalReturnType());
-      JMethod m = clazz.method(JMod.PUBLIC, ret, "doEval");
-      m.param(model.INT, "inIndex");
-      m.param(model.INT, "outIndex");
-      m.body().add(parentEvalBlock);
-    }
-
-    SingleClassStringWriter w = new SingleClassStringWriter();
-    model.build(w);
-    return w.getCode().toString();
-  }
+  
   
   public JCodeModel getModel() {
     return model;
@@ -205,9 +211,9 @@ public class CodeGenerator<T> {
     JType holderType = getHolderType(t);
     JVar var;
     if(includeNewInstance){
-      var = currentEvalBlock.decl(holderType, "out" + index, JExpr._new(holderType));
+      var = getEvalBlock().decl(holderType, "out" + index, JExpr._new(holderType));
     }else{
-      var = currentEvalBlock.decl(holderType, "out" + index);
+      var = getEvalBlock().decl(holderType, "out" + index);
     }
     JFieldRef outputSet = null;
     if(t.getMode() == DataMode.OPTIONAL){
@@ -239,6 +245,10 @@ public class CodeGenerator<T> {
     public JFieldRef getValue() {
       return value;
     }
+    
+    public MajorType getMajorType(){
+      return type;
+    }
 
     public JFieldRef getIsSet() {
       Preconditions.checkNotNull(isSet, "You cannot access the isSet variable when operating on a non-nullable output value.");
@@ -261,4 +271,4 @@ public class CodeGenerator<T> {
   public JType getHolderType(MajorType t){
     return TypeHelper.getHolderType(model, t.getMinorType(), t.getMode());
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index 21106c1..0e66fd0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -1,5 +1,7 @@
 package org.apache.drill.exec.expr;
 
+import java.util.Set;
+
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.IfExpression;
 import org.apache.drill.common.expression.IfExpression.IfCondition;
@@ -12,6 +14,7 @@ import org.apache.drill.common.expression.ValueExpressions.QuotedString;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.compile.sig.ConstantExpressionIdentifier;
 import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.fn.FunctionHolder;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -24,190 +27,309 @@ import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JInvocation;
 import com.sun.codemodel.JVar;
 
-public class EvaluationVisitor extends AbstractExprVisitor<HoldingContainer, CodeGenerator<?>, RuntimeException> {
+public class EvaluationVisitor {
 
-  private FunctionImplementationRegistry registry;
-  
+  private final FunctionImplementationRegistry registry;
   
   public EvaluationVisitor(FunctionImplementationRegistry registry) {
+    super();
     this.registry = registry;
   }
 
-  @Override
-  public HoldingContainer visitFunctionCall(FunctionCall call, CodeGenerator<?> generator) throws RuntimeException {
-    HoldingContainer[] args = new HoldingContainer[call.args.size()];
-    for(int i = 0; i < call.args.size(); i++){
-      args[i] = call.args.get(i).accept(this, generator);
-    }
-    FunctionHolder holder = registry.getFunction(call);
+  public HoldingContainer addExpr(LogicalExpression e, CodeGenerator<?> generator){
+    return e.accept(new ConstantFilter(ConstantExpressionIdentifier.getConstantExpressionSet(e)), generator);
     
-    return holder.renderFunction(generator, args);
   }
-  
-  @Override
-  public HoldingContainer visitIfExpression(IfExpression ifExpr, CodeGenerator<?> generator) throws RuntimeException {
-    JBlock local = generator.getBlock();
-    
-    HoldingContainer output = generator.declare(ifExpr.getMajorType());
+
+  private class EvalVisitor extends AbstractExprVisitor<HoldingContainer, CodeGenerator<?>, RuntimeException> {
+
     
-    JConditional jc = null;
-    JBlock conditionalBlock = new JBlock();
-    for (IfCondition c : ifExpr.conditions) {
-      HoldingContainer HoldingContainer = c.condition.accept(this, generator);
-      if (jc == null) {
-        if (HoldingContainer.isOptional()) {
-          jc = conditionalBlock._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
+    @Override
+    public HoldingContainer visitFunctionCall(FunctionCall call, CodeGenerator<?> generator) throws RuntimeException {
+      HoldingContainer[] args = new HoldingContainer[call.args.size()];
+      for (int i = 0; i < call.args.size(); i++) {
+        args[i] = call.args.get(i).accept(this, generator);
+      }
+      FunctionHolder holder = registry.getFunction(call);
+      return holder.renderFunction(generator, args);
+    }
+
+    @Override
+    public HoldingContainer visitIfExpression(IfExpression ifExpr, CodeGenerator<?> generator) throws RuntimeException {
+      JBlock local = generator.getEvalBlock();
+
+      HoldingContainer output = generator.declare(ifExpr.getMajorType());
+
+      JConditional jc = null;
+      JBlock conditionalBlock = new JBlock(false, false);
+      for (IfCondition c : ifExpr.conditions) {
+        HoldingContainer HoldingContainer = c.condition.accept(this, generator);
+        if (jc == null) {
+          if (HoldingContainer.isOptional()) {
+            jc = conditionalBlock._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
+          } else {
+            jc = conditionalBlock._if(HoldingContainer.getValue());
+          }
         } else {
-          jc = conditionalBlock._if(HoldingContainer.getValue());
+          if (HoldingContainer.isOptional()) {
+            jc = jc._else()._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
+          } else {
+            jc = jc._else()._if(HoldingContainer.getValue());
+          }
         }
-      } else {
-        if (HoldingContainer.isOptional()) {
-          jc = jc._else()._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
+
+        HoldingContainer thenExpr = c.expression.accept(this, generator);
+        if (thenExpr.isOptional()) {
+          JConditional newCond = jc._then()._if(thenExpr.getIsSet());
+          JBlock b = newCond._then();
+          b.assign(output.getValue(), thenExpr.getValue());
+          b.assign(output.getIsSet(), thenExpr.getIsSet());
         } else {
-          jc = jc._else()._if(HoldingContainer.getValue());
+          jc._then().assign(output.getValue(), thenExpr.getValue());
         }
+
       }
 
-      HoldingContainer thenExpr = c.expression.accept(this, generator);
-      if (thenExpr.isOptional()) {
-        JConditional newCond = jc._then()._if(thenExpr.getIsSet());
+      HoldingContainer elseExpr = ifExpr.elseExpression.accept(this, generator);
+      if (elseExpr.isOptional()) {
+        JConditional newCond = jc._else()._if(elseExpr.getIsSet());
         JBlock b = newCond._then();
-        b.assign(output.getValue(), thenExpr.getValue());
-        b.assign(output.getIsSet(), thenExpr.getIsSet());
+        b.assign(output.getValue(), elseExpr.getValue());
+        b.assign(output.getIsSet(), elseExpr.getIsSet());
       } else {
-        jc._then().assign(output.getValue(), thenExpr.getValue());
+        jc._else().assign(output.getValue(), elseExpr.getValue());
+
       }
+      local.add(conditionalBlock);
+      return output;
+    }
 
+    @Override
+    public HoldingContainer visitSchemaPath(SchemaPath path, CodeGenerator<?> generator) throws RuntimeException {
+      throw new UnsupportedOperationException("All schema paths should have been replaced with ValueVectorExpressions.");
     }
 
-    HoldingContainer elseExpr = ifExpr.elseExpression.accept(this, generator);
-    if (elseExpr.isOptional()) {
-      JConditional newCond = jc._else()._if(elseExpr.getIsSet());
-      JBlock b = newCond._then();
-      b.assign(output.getValue(), elseExpr.getValue());
-      b.assign(output.getIsSet(), elseExpr.getIsSet());
-    } else {
-      jc._else().assign(output.getValue(), elseExpr.getValue());
+    @Override
+    public HoldingContainer visitLongConstant(LongExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      HoldingContainer out = generator.declare(e.getMajorType());
+      generator.getEvalBlock().assign(out.getValue(), JExpr.lit(e.getLong()));
+      return out;
+    }
 
+    @Override
+    public HoldingContainer visitDoubleConstant(DoubleExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      HoldingContainer out = generator.declare(e.getMajorType());
+      generator.getEvalBlock().assign(out.getValue(), JExpr.lit(e.getDouble()));
+      return out;
     }
-    local.add(conditionalBlock);
-    return output;
-  }
 
-  @Override
-  public HoldingContainer visitSchemaPath(SchemaPath path, CodeGenerator<?> generator) throws RuntimeException {
-    throw new UnsupportedOperationException("All schema paths should have been replaced with ValueVectorExpressions.");
-  }
+    @Override
+    public HoldingContainer visitBooleanConstant(BooleanExpression e, CodeGenerator<?> generator)
+        throws RuntimeException {
+      HoldingContainer out = generator.declare(e.getMajorType());
+      generator.getEvalBlock().assign(out.getValue(), JExpr.lit(e.getBoolean() ? 1 : 0));
+      return out;
+    }
 
-  @Override
-  public HoldingContainer visitLongConstant(LongExpression e, CodeGenerator<?> generator) throws RuntimeException {
-    HoldingContainer out = generator.declare(e.getMajorType());
-    generator.getBlock().assign(out.getValue(), JExpr.lit(e.getLong()));
-    return out;
-  }
+    @Override
+    public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      if (e instanceof ValueVectorReadExpression) {
+        return visitValueVectorReadExpression((ValueVectorReadExpression) e, generator);
+      } else if (e instanceof ValueVectorWriteExpression) {
+        return visitValueVectorWriteExpression((ValueVectorWriteExpression) e, generator);
+      } else if (e instanceof ReturnValueExpression) {
+        return visitReturnValueExpression((ReturnValueExpression) e, generator);
+      }else if(e instanceof HoldingContainerExpression){
+        return ((HoldingContainerExpression) e).getContainer();
+      } else {
+        return super.visitUnknown(e, generator);
+      }
 
-  @Override
-  public HoldingContainer visitDoubleConstant(DoubleExpression e, CodeGenerator<?> generator) throws RuntimeException {
-    HoldingContainer out = generator.declare(e.getMajorType());
-    generator.getBlock().assign(out.getValue(), JExpr.lit(e.getDouble()));
-    return out;
-  }
+    }
 
-  @Override
-  public HoldingContainer visitBooleanConstant(BooleanExpression e, CodeGenerator<?> generator) throws RuntimeException {
-    HoldingContainer out = generator.declare(e.getMajorType());
-    generator.getBlock().assign(out.getValue(), JExpr.lit(e.getBoolean() ? 1 : 0));
-    return out;
-  }
-  
-  @Override
-  public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator<?> generator) throws RuntimeException {
-    if(e instanceof ValueVectorReadExpression){
-      return visitValueVectorReadExpression((ValueVectorReadExpression) e, generator);
-    }else if(e instanceof ValueVectorWriteExpression){
-      return visitValueVectorWriteExpression((ValueVectorWriteExpression) e, generator);
-    }else if(e instanceof ReturnValueExpression){
-      return visitReturnValueExpression((ReturnValueExpression) e, generator);
-    }else{
-      return super.visitUnknown(e, generator);  
+    private HoldingContainer visitValueVectorWriteExpression(ValueVectorWriteExpression e, CodeGenerator<?> generator) {
+      LogicalExpression child = e.getChild();
+      HoldingContainer hc = child.accept(this, generator);
+      JBlock block = generator.getEvalBlock();
+      JExpression outIndex = generator.getMappingSet().getValueWriteIndex();
+      JVar vv = generator.declareVectorValueSetupAndMember("outgoing", e.getFieldId());
+
+      if (hc.isOptional()) {
+        vv.invoke("getMutator").invoke("set").arg(outIndex);
+        JConditional jc = block._if(hc.getIsSet().eq(JExpr.lit(0)).not());
+        block = jc._then();
+      }
+      if (hc.getMinorType() == TypeProtos.MinorType.VARCHAR || hc.getMinorType() == TypeProtos.MinorType.VARBINARY) {
+        block.add(vv.invoke("getMutator").invoke("set").arg(outIndex).arg(hc.getHolder()));
+      } else {
+        block.add(vv.invoke("getMutator").invoke("set").arg(outIndex).arg(hc.getValue()));
+      }
+      return null;
     }
-    
-  }
 
-  private HoldingContainer visitValueVectorWriteExpression(ValueVectorWriteExpression e, CodeGenerator<?> generator){
-    LogicalExpression child = e.getChild();
-    HoldingContainer hc = child.accept(this, generator);
-    JBlock block = generator.getBlock();
-    
-    JVar vv = generator.declareVectorValueSetupAndMember("outgoing", e.getFieldId());
-    
-    if(hc.isOptional()){
-      vv.invoke("getMutator").invoke("set").arg(JExpr.direct("outIndex"));
-      JConditional jc = block._if(hc.getIsSet().eq(JExpr.lit(0)).not());
-      block = jc._then();
+    private HoldingContainer visitValueVectorReadExpression(ValueVectorReadExpression e, CodeGenerator<?> generator)
+        throws RuntimeException {
+      // declare value vector
+      
+      JVar vv1 = generator.declareVectorValueSetupAndMember("incoming", e.getFieldId());
+      JExpression indexVariable = generator.getMappingSet().getValueReadIndex();
+
+      JInvocation getValueAccessor = vv1.invoke("getAccessor").invoke("get");
+      if (e.isSuperReader()) {
+
+        getValueAccessor = ((JExpression) vv1.component(indexVariable.shrz(JExpr.lit(16)))).invoke("getAccessor")
+            .invoke("get");
+        indexVariable = indexVariable.band(JExpr.lit((int) Character.MAX_VALUE));
+      }
+
+      // evaluation work.
+      HoldingContainer out = generator.declare(e.getMajorType());
+
+      if (out.isOptional()) {
+        JBlock blk = generator.getEvalBlock();
+        blk.assign(out.getIsSet(), vv1.invoke("getAccessor").invoke("isSet").arg(indexVariable));
+        JConditional jc = blk._if(out.getIsSet().eq(JExpr.lit(1)));
+        if (Types.usesHolderForGet(e.getMajorType())) {
+          jc._then().add(getValueAccessor.arg(indexVariable).arg(out.getHolder()));
+        } else {
+          jc._then().assign(out.getValue(), getValueAccessor.arg(indexVariable));
+        }
+      } else {
+        if (Types.usesHolderForGet(e.getMajorType())) {
+          generator.getEvalBlock().add(getValueAccessor.arg(indexVariable).arg(out.getHolder()));
+        } else {
+          generator.getEvalBlock().assign(out.getValue(), getValueAccessor.arg(indexVariable));
+        }
+      }
+      return out;
+    }
+
+    private HoldingContainer visitReturnValueExpression(ReturnValueExpression e, CodeGenerator<?> generator) {
+      LogicalExpression child = e.getChild();
+      // Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
+      HoldingContainer hc = child.accept(this, generator);
+      generator.getEvalBlock()._return(hc.getValue().eq(JExpr.lit(1)));
+      return null;
     }
-    if (hc.getMinorType() == TypeProtos.MinorType.VARCHAR || hc.getMinorType() == TypeProtos.MinorType.VARBINARY) {
-      block.add(vv.invoke("getMutator").invoke("set").arg(JExpr.direct("outIndex")).arg(hc.getHolder()));
-    } else {
-      block.add(vv.invoke("getMutator").invoke("set").arg(JExpr.direct("outIndex")).arg(hc.getValue()));
+
+    @Override
+    public HoldingContainer visitQuotedStringConstant(QuotedString e, CodeGenerator<?> generator)
+        throws RuntimeException {
+      throw new UnsupportedOperationException(
+          "We don't yet support string literals as we need to build the string value holders.");
+
+      // JExpr stringLiteral = JExpr.lit(e.value);
+      // CodeGenerator.block.decl(stringLiteral.invoke("getBytes").arg(JExpr.ref(Charsets.UTF_8));
     }
-    return null;
   }
-  
-  private HoldingContainer visitValueVectorReadExpression(ValueVectorReadExpression e, CodeGenerator<?> generator) throws RuntimeException{
-    // declare value vector
+
+  private class ConstantFilter extends EvalVisitor {
+
+    private Set<LogicalExpression> constantBoundaries;
     
-    JVar vv1 = generator.declareVectorValueSetupAndMember("incoming", e.getFieldId());
-    final String indexVariableName = e.isNamed() ? e.getIndexVariableName() : "inIndex";
-    JExpression indexVariable = JExpr.direct(indexVariableName);
     
-    JInvocation getValueAccessor = vv1.invoke("getAccessor").invoke("get");
-    if(e.isSuperReader()){
-      
-      getValueAccessor = ((JExpression) vv1.component(indexVariable.shrz(JExpr.lit(16)))).invoke("getAccessor").invoke("get");
-      indexVariable = indexVariable.band(JExpr.lit((int) Character.MAX_VALUE));
+    public ConstantFilter(Set<LogicalExpression> constantBoundaries) {
+      super();
+      this.constantBoundaries = constantBoundaries;
     }
-    
-    
-    // evaluation work.
-    HoldingContainer out = generator.declare(e.getMajorType());
-    
-    if(out.isOptional()){
-      JBlock blk = generator.getBlock();
-      blk.assign(out.getIsSet(), vv1.invoke("getAccessor").invoke("isSet").arg(indexVariable));
-      JConditional jc = blk._if(out.getIsSet().eq(JExpr.lit(1)));
-      if (Types.usesHolderForGet(e.getMajorType())) {
-        jc._then()
-            .add(getValueAccessor.arg(JExpr.direct("inIndex")).arg(out.getHolder()));
+
+    @Override
+    public HoldingContainer visitFunctionCall(FunctionCall e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitFunctionCall(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
       } else {
-        jc._then()
-            .assign(out.getValue(), getValueAccessor.arg(indexVariable));
+        return super.visitFunctionCall(e, generator);
       }
-    }else{
-      if (Types.usesHolderForGet(e.getMajorType())) {
-        generator.getBlock().add(getValueAccessor.arg(indexVariable).arg(out.getHolder()));
+    }
+
+    @Override
+    public HoldingContainer visitIfExpression(IfExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitIfExpression(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
       } else {
-        generator.getBlock().assign(out.getValue(), getValueAccessor.arg(indexVariable));
+        return super.visitIfExpression(e, generator);
       }
     }
-    return out;
-  }
-  
-  
-  private HoldingContainer visitReturnValueExpression(ReturnValueExpression e, CodeGenerator<?> generator){
-    LogicalExpression child = e.getChild();
-//    Preconditions.checkArgument(child.getMajorType().equals(Types.REQUIRED_BOOLEAN));
-    HoldingContainer hc = child.accept(this, generator);
-    generator.getBlock()._return(hc.getValue().eq(JExpr.lit(1)));
-    return null;
-  }
-  
-  @Override
-  public HoldingContainer visitQuotedStringConstant(QuotedString e, CodeGenerator<?> CodeGenerator) throws RuntimeException {
-    throw new UnsupportedOperationException("We don't yet support string literals as we need to build the string value holders.");
-    
-//    JExpr stringLiteral = JExpr.lit(e.value);
-//    CodeGenerator.block.decl(stringLiteral.invoke("getBytes").arg(JExpr.ref(Charsets.UTF_8));
-  }
 
+    @Override
+    public HoldingContainer visitSchemaPath(SchemaPath e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitSchemaPath(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitSchemaPath(e, generator);
+      }
+    }
+
+    @Override
+    public HoldingContainer visitLongConstant(LongExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitLongConstant(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitLongConstant(e, generator);
+      }
+    }
+
+    @Override
+    public HoldingContainer visitDoubleConstant(DoubleExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitDoubleConstant(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitDoubleConstant(e, generator);
+      }
+    }
+
+    @Override
+    public HoldingContainer visitBooleanConstant(BooleanExpression e, CodeGenerator<?> generator)
+        throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitBooleanConstant(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitBooleanConstant(e, generator);
+      }
+    }
+
+    @Override
+    public HoldingContainer visitUnknown(LogicalExpression e, CodeGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitUnknown(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitUnknown(e, generator);
+      }
+    }
+
+    @Override
+    public HoldingContainer visitQuotedStringConstant(QuotedString e, CodeGenerator<?> generator)
+        throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitQuotedStringConstant(e, generator);
+        generator.getMappingSet().exitConstant();
+        return c;
+      } else {
+        return super.visitQuotedStringConstant(e, generator);
+      }
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 07db72c..cb2be68 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -80,7 +80,7 @@ public class ExpressionTreeMaterializer {
 
     @Override
     public LogicalExpression visitIfExpression(IfExpression ifExpr) {
-      List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
+      List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.conditions);
       LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this, null);
 
       for (int i = 0; i < conditions.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HoldingContainerExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HoldingContainerExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HoldingContainerExpression.java
new file mode 100644
index 0000000..da22bee
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/HoldingContainerExpression.java
@@ -0,0 +1,46 @@
+package org.apache.drill.exec.expr;
+
+import java.util.Iterator;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.visitors.ExprVisitor;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+
+import com.google.common.collect.Iterators;
+
+public class HoldingContainerExpression implements LogicalExpression{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HoldingContainerExpression.class);
+
+  final HoldingContainer container;
+  
+  public HoldingContainerExpression(HoldingContainer container) {
+    this.container = container;
+  }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.emptyIterator();
+  }
+
+  @Override
+  public MajorType getMajorType() {
+    return container.getMajorType();
+  }
+
+  @Override
+  public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
+    return visitor.visitUnknown(this, value);
+  }
+
+  
+  public HoldingContainer getContainer() {
+    return container;
+  }
+
+  @Override
+  public ExpressionPosition getPosition() {
+    return ExpressionPosition.UNKNOWN;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
index f86bd29..6848234 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorReadExpression.java
@@ -1,11 +1,15 @@
 package org.apache.drill.exec.expr;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.exec.record.TypedFieldId;
 
+import com.google.common.collect.Iterators;
+
 public class ValueVectorReadExpression implements LogicalExpression{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorReadExpression.class);
 
@@ -58,6 +62,11 @@ public class ValueVectorReadExpression implements LogicalExpression{
   public ExpressionPosition getPosition() {
     return ExpressionPosition.UNKNOWN;
   }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.emptyIterator();
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
index 58141d8..7f2d847 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ValueVectorWriteExpression.java
@@ -1,5 +1,7 @@
 package org.apache.drill.exec.expr;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
@@ -7,6 +9,8 @@ import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.record.TypedFieldId;
 
+import com.google.common.collect.Iterators;
+
 public class ValueVectorWriteExpression implements LogicalExpression {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ValueVectorWriteExpression.class);
 
@@ -47,5 +51,10 @@ public class ValueVectorWriteExpression implements LogicalExpression {
     return child;
   }
   
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.singletonIterator(child);
+  }
+
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
new file mode 100644
index 0000000..3f3f45d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/MethodMap.java
@@ -0,0 +1,21 @@
+package org.apache.drill.exec.expr.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+
+/**
+ * Marker annotation to determine which fields should be included as parameters for the function.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.METHOD})
+public @interface MethodMap {
+  
+  
+  String parentMethod();
+   
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
new file mode 100644
index 0000000..a77b4b0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillAggFuncHolder.java
@@ -0,0 +1,36 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+public class DrillAggFuncHolder extends DrillFuncHolder{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillAggFuncHolder.class);
+  
+  
+  public DrillAggFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative,
+      String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
+      Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, functionName, parameters, returnValue, workspaceVars, methods, imports);
+    
+  }
+
+  public HoldingContainer renderFunction(CodeGenerator<?> g, HoldingContainer[] inputVariables){
+    g.getMappingSet().enterChild();
+    renderInside(g, inputVariables);
+    g.getMappingSet().exitChild();
+    return renderOutside(g);
+  }
+  
+  private void renderInside(CodeGenerator<?> g, HoldingContainer[] inputVariables){
+    
+  }
+  
+  private HoldingContainer renderOutside(CodeGenerator<?> g){
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
new file mode 100644
index 0000000..8e69609
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
@@ -0,0 +1,102 @@
+package org.apache.drill.exec.expr.fn;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.expr.CodeGenerator;
+import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.sun.codemodel.JBlock;
+import com.sun.codemodel.JConditional;
+import com.sun.codemodel.JExpr;
+import com.sun.codemodel.JExpression;
+import com.sun.codemodel.JMod;
+import com.sun.codemodel.JVar;
+
+public class DrillFuncHolder extends FunctionHolder{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFuncHolder.class);
+  
+  private final String setupBody;
+  private final String evalBody;
+  
+  
+  public DrillFuncHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative,
+      String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars,
+      Map<String, String> methods, List<String> imports) {
+    super(scope, nullHandling, isBinaryCommutative, functionName, parameters, returnValue, workspaceVars, methods, imports);
+    setupBody = methods.get("setup");
+    evalBody = methods.get("eval");
+    Preconditions.checkNotNull(evalBody);
+    
+  }
+
+  private void generateSetupBody(CodeGenerator<?> g){
+    if(!Strings.isNullOrEmpty(setupBody)){
+      JBlock sub = new JBlock(true, true);
+      addProtectedBlock(g, sub, setupBody, null);
+      g.getSetupBlock().directStatement(String.format("/** start setup for function %s **/ ", functionName));
+      g.getSetupBlock().add(sub);
+      g.getSetupBlock().directStatement(String.format("/** end setup for function %s **/ ", functionName));
+    }
+  }
+  
+  
+  public HoldingContainer renderFunction(CodeGenerator<?> g, HoldingContainer[] inputVariables){
+    generateSetupBody(g);
+    return generateEvalBody(g, inputVariables);
+  }
+  
+ private HoldingContainer generateEvalBody(CodeGenerator<?> g, HoldingContainer[] inputVariables){
+    
+    //g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
+    
+    JBlock sub = new JBlock(true, true);
+    JBlock topSub = sub;
+    HoldingContainer out = null;
+    MajorType returnValueType = returnValue.type;
+
+    // add outside null handling if it is defined.
+    if(nullHandling == NullHandling.NULL_IF_NULL){
+      JExpression e = null;
+      for(HoldingContainer v : inputVariables){
+        if(v.isOptional()){
+          if(e == null){
+            e = v.getIsSet();
+          }else{
+            e = e.mul(v.getIsSet());
+          }
+        }
+      }
+      
+      if(e != null){
+        // if at least one expression must be checked, set up the conditional.
+        returnValueType = returnValue.type.toBuilder().setMode(DataMode.OPTIONAL).build();
+        out = g.declare(returnValueType);
+        e = e.eq(JExpr.lit(0));
+        JConditional jc = sub._if(e);
+        jc._then().assign(out.getIsSet(), JExpr.lit(0));
+        sub = jc._else();
+      }
+    }
+    
+    if(out == null) out = g.declare(returnValueType);
+    
+    // add the subblock after the out declaration.
+    g.getEvalBlock().add(topSub);
+    
+    
+    JVar internalOutput = sub.decl(JMod.FINAL, g.getHolderType(returnValueType), returnValue.name, JExpr._new(g.getHolderType(returnValueType)));
+    addProtectedBlock(g, sub, evalBody, inputVariables);
+    if (sub != topSub) sub.assign(internalOutput.ref("isSet"),JExpr.lit(1));// Assign null if NULL_IF_NULL mode
+    sub.assign(out.getHolder(), internalOutput);
+
+    return out;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
new file mode 100644
index 0000000..04d6e4c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionBody.java
@@ -0,0 +1,12 @@
+package org.apache.drill.exec.expr.fn;
+
+public class FunctionBody {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionBody.class);
+  
+  
+  public static enum BodyType{
+    SETUP, EVAL_INNER, EVAL_OUTER, 
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
index 5babc78..24ebeaa 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionConverter.java
@@ -127,8 +127,18 @@ public class FunctionConverter {
     }
     
     try{
-      FunctionHolder fh = new FunctionHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods, imports);
-      return fh;
+      switch(template.scope()){
+      case POINT_AGGREGATE:
+        return new DrillAggFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods, imports);
+      case SIMPLE:
+        FunctionHolder fh = new DrillFuncHolder(template.scope(), template.nulls(), template.isBinaryCommutative(), template.name(), ps, outputField, works, methods, imports);
+        return fh;
+
+      case HOLISTIC_AGGREGATE:
+      case RANGE_AGGREGATE:
+      default:
+        return failure("Unsupported Function Type.", clazz);
+      }
     }catch(Exception ex){
       return failure("Failure while creating function holder.", ex, clazz);
     }


[3/8] git commit: DRILL-169 Correct JSON serialization/deserialization for Project LOP

Posted by ja...@apache.org.
DRILL-169 Correct JSON serialization/deserialization for Project LOP


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

Branch: refs/heads/master
Commit: c568eed28567f65b9e5931f7fea428497d0b5bfd
Parents: acf4781
Author: immars <im...@gmail.com>
Authored: Thu Aug 22 15:28:28 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:08:39 2013 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/drill/common/logical/data/Project.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/c568eed2/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Project.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Project.java
index aca114f..109d218 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Project.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/Project.java
@@ -47,7 +47,7 @@ public class Project extends SingleInputOperator {
     }
   }
 
-  @JsonProperty("exprs")
+  @JsonProperty("projections")
   public NamedExpression[] getSelections() {
     return selections;
   }


Re: [5/8] git commit: DRILL-176: Updates to affinity calculator, fixes for parquet serialization. Fix to ErrorHelper looping

Posted by Tanujit Ghosh <ta...@gmail.com>.
A more detailed output, i just ran the purticular test this time

[tanujit@legolas java-exec]$ mvn -Dtest=ParquetRecordReaderTest test
[INFO] Scanning for projects...
[INFO]

[INFO]
------------------------------------------------------------------------
[INFO] Building java-exec 1.0-SNAPSHOT
[INFO]
------------------------------------------------------------------------
[INFO]
[INFO] --- maven-resources-plugin:2.6:copy-resources (copy-resources) @
java-exec ---
[INFO] Using 'UTF-8' encoding to copy filtered resources.
[INFO] Copying 369 resources
[INFO]
[INFO] --- maven-enforcer-plugin:1.2:enforce (no_commons_logging) @
java-exec ---
[INFO]
[INFO] --- maven-antrun-plugin:1.6:run (generate-sources) @ java-exec ---
[WARNING] Parameter tasks is deprecated, use target instead
[INFO] Executing tasks

main:
[INFO] Executed tasks
[INFO] Registering compile source root
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/generated-sources
[INFO]
[INFO] --- fmpp-maven-plugin:1.0:generate (generate-sources) @ java-exec ---
- Executing: ValueHolders.java
log4j:WARN No appenders could be found for logger (freemarker.cache).
log4j:WARN Please initialize the log4j system properly.
- Executing: VariableLengthVectors.java
- Executing: FixedValueVectors.java
- Executing: TypeHelper.java
- Executing: NullableValueVectors.java
- Executing: RepeatedValueVectors.java
[INFO] Done
[INFO]
[INFO] --- maven-remote-resources-plugin:1.1:process (default) @ java-exec
---
[INFO] Setting property: classpath.resource.loader.class =>
'org.codehaus.plexus.velocity.ContextClassLoaderResourceLoader'.
[INFO] Setting property: velocimacro.messages.on => 'false'.
[INFO] Setting property: resource.loader => 'classpath'.
[INFO] Setting property: resource.manager.logwhenfound => 'false'.
[INFO]
[INFO] --- maven-resources-plugin:2.6:resources (default-resources) @
java-exec ---
[INFO] Using 'UTF-8' encoding to copy filtered resources.
[INFO] Copying 1 resource
[INFO] Copying 3 resources
[INFO]
[INFO] --- maven-compiler-plugin:3.0:compile (default-compile) @ java-exec
---
[INFO] Changes detected - recompiling the module!
[INFO] Compiling 459 source files to
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/classes
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetStorageEngine.java:[20,21]
com.sun.corba.se.impl.interceptors.CodecFactoryImpl is internal proprietary
API and may be removed in a future release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java:
Some input files use or override a deprecated API.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java:
Recompile with -Xlint:deprecation for details.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java:
Some input files use unchecked or unsafe operations.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java:
Recompile with -Xlint:unchecked for details.
[INFO]
[INFO] --- maven-resources-plugin:2.6:testResources (default-testResources)
@ java-exec ---
[INFO] Using 'UTF-8' encoding to copy filtered resources.
[INFO] Copying 36 resources
[INFO] Copying 3 resources
[INFO]
[INFO] --- maven-compiler-plugin:3.0:testCompile (default-testCompile) @
java-exec ---
[INFO] Changes detected - recompiling the module!
[INFO] Compiling 126 source files to
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/test-classes
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[25,16]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[53,17]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[94,12]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[95,14]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[98,26]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExecutionAbstractions.java:[100,27]
sun.misc.Unsafe is internal proprietary API and may be removed in a future
release
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java:
Some input files use or override a deprecated API.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java:
Recompile with -Xlint:deprecation for details.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java:
Some input files use unchecked or unsafe operations.
[WARNING]
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestFileGenerator.java:
Recompile with -Xlint:unchecked for details.
[INFO]
[INFO] --- maven-surefire-plugin:2.15:test (default-test) @ java-exec ---
[INFO] Surefire report directory:
/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/surefire-reports

-------------------------------------------------------
 T E S T S
-------------------------------------------------------
Running org.apache.drill.exec.store.parquet.ParquetRecordReaderTest
08:18:15.598 [main] DEBUG org.reflections.Reflections - going to scan these
urls:
jar:file:/home/tanujit/.m2/repository/org/apache/drill/common/1.0-SNAPSHOT/common-1.0-SNAPSHOT.jar!/
file:/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/test-classes/
file:/backup/java/wip/incubator-drill/sandbox/prototype/exec/java-exec/target/classes/
jar:file:/home/tanujit/.m2/repository/org/apache/drill/common/1.0-SNAPSHOT/common-1.0-SNAPSHOT-tests.jar!/

08:18:16.124 [main] INFO  org.reflections.Reflections - Reflections took
524 ms to scan 4 urls, producing 667 keys and 1620 values
08:18:16.148 [main] DEBUG i.n.u.i.l.InternalLoggerFactory - Using SLF4J as
the default logging framework
08:18:16.150 [main] DEBUG i.n.c.MultithreadEventLoopGroup -
-Dio.netty.eventLoopThreads: 8
08:18:16.165 [main] DEBUG i.n.util.internal.PlatformDependent - UID: 1000
08:18:16.165 [main] DEBUG i.n.util.internal.PlatformDependent - Java
version: 7
08:18:16.166 [main] DEBUG i.n.util.internal.PlatformDependent -
-Dio.netty.noUnsafe: false
08:18:16.167 [main] DEBUG i.n.util.internal.PlatformDependent0 -
java.nio.ByteBuffer.cleaner: available
08:18:16.167 [main] DEBUG i.n.util.internal.PlatformDependent0 -
java.nio.Buffer.address: available
08:18:16.168 [main] DEBUG i.n.util.internal.PlatformDependent0 -
sun.misc.Unsafe.theUnsafe: available
08:18:16.168 [main] DEBUG i.n.util.internal.PlatformDependent0 -
sun.misc.Unsafe.copyMemory: available
08:18:16.168 [main] DEBUG i.n.util.internal.PlatformDependent0 -
java.nio.Bits.unaligned: true
08:18:16.168 [main] DEBUG i.n.util.internal.PlatformDependent -
sun.misc.Unsafe: available
08:18:16.169 [main] DEBUG i.n.util.internal.PlatformDependent -
-Dio.netty.noJavassist: false
08:18:16.227 [main] DEBUG i.n.util.internal.PlatformDependent - Javassist:
available
08:18:16.228 [main] DEBUG i.n.util.internal.PlatformDependent -
-Dio.netty.noPreferDirect: false
08:18:16.249 [main] DEBUG io.netty.channel.nio.NioEventLoop -
-Dio.netty.noKeySetOptimization: false
08:18:16.249 [main] DEBUG io.netty.channel.nio.NioEventLoop -
-Dio.netty.selectorAutoRebuildThreshold: 512
08:18:16.265 [main] DEBUG i.n.buffer.PooledByteBufAllocatorL -
-Dio.netty.allocator.numHeapArenas: 4
08:18:16.265 [main] DEBUG i.n.buffer.PooledByteBufAllocatorL -
-Dio.netty.allocator.numDirectArenas: 4
08:18:16.265 [main] DEBUG i.n.buffer.PooledByteBufAllocatorL -
-Dio.netty.allocator.pageSize: 8192
08:18:16.265 [main] DEBUG i.n.buffer.PooledByteBufAllocatorL -
-Dio.netty.allocator.maxOrder: 11
08:18:16.265 [main] DEBUG i.n.buffer.PooledByteBufAllocatorL -
-Dio.netty.allocator.chunkSize: 16777216
08:18:16.335 [main] DEBUG io.netty.util.NetUtil - Loopback interface: lo
08:18:16.335 [main] DEBUG io.netty.util.NetUtil - Loopback address:
/0:0:0:0:0:0:0:1%1 (primary)
08:18:16.335 [main] DEBUG io.netty.util.NetUtil - Loopback address: /
127.0.0.1
08:18:16.335 [main] DEBUG io.netty.util.NetUtil -
/proc/sys/net/core/somaxconn: 128
08:18:16.344 [UserServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x30e3b962] REGISTERED
08:18:16.347 [UserServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x30e3b962] BIND(0.0.0.0/0.0.0.0:31010)
08:18:16.350 [UserServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x30e3b962, /0:0:0:0:0:0:0:0:31010] ACTIVE
08:18:16.366 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b] REGISTERED
08:18:16.367 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b] BIND(0.0.0.0/0.0.0.0:31011)
08:18:16.367 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] ACTIVE
08:18:16.571 [Client-1] DEBUG
i.n.u.i.JavassistTypeParameterMatcherGenerator - Generated:
io.netty.util.internal.__matchers__.io.netty.buffer.ByteBufMatcher
08:18:16.574 [Client-1] DEBUG
i.n.u.i.JavassistTypeParameterMatcherGenerator - Generated:
io.netty.util.internal.__matchers__.org.apache.drill.exec.rpc.OutboundRpcMessageMatcher
08:18:16.576 [Client-1] DEBUG
i.n.u.i.JavassistTypeParameterMatcherGenerator - Generated:
io.netty.util.internal.__matchers__.org.apache.drill.exec.rpc.InboundRpcMessageMatcher
08:18:16.579 [UserServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x30e3b962, /0:0:0:0:0:0:0:0:31010] RECEIVED: [id: 0xf33275f9, /
192.168.0.102:34995 => /192.168.0.102:31010]
08:18:16.589 [Client-1] DEBUG io.netty.util.ResourceLeakDetector -
-Dio.netty.noResourceLeakDetection: false
08:18:16.770 [WorkManager-1] DEBUG org.apache.hadoop.security.Groups -
 Creating new Groups object
08:18:16.817 [WorkManager-1] DEBUG org.apache.hadoop.security.Groups -
Group mapping impl=org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
cacheTimeout=300000
08:18:16.882 [WorkManager-1] DEBUG o.a.h.security.UserGroupInformation -
hadoop login
08:18:16.882 [WorkManager-1] DEBUG o.a.h.security.UserGroupInformation -
hadoop login commit
08:18:16.898 [WorkManager-1] DEBUG o.a.h.security.UserGroupInformation -
using local user:UnixPrincipal: tanujit
08:18:16.900 [WorkManager-1] DEBUG o.a.h.security.UserGroupInformation -
UGI loginUser:tanujit
08:18:16.940 [WorkManager-1] DEBUG org.apache.hadoop.fs.FileSystem -
Creating filesystem for file:///
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.
08:18:17.922 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0x1f148baa, /
192.168.0.102:45778 => /192.168.0.102:31011]
08:18:17.923 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0xc4ed7542, /
192.168.0.102:45779 => /192.168.0.102:31011]
08:18:17.937 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0x70a4a44c, /
192.168.0.102:45780 => /192.168.0.102:31011]
08:18:17.937 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0xf6360725, /
192.168.0.102:45781 => /192.168.0.102:31011]
08:18:17.937 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0x51467583, /
192.168.0.102:45782 => /192.168.0.102:31011]
08:18:17.938 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0xaf6d72df, /
192.168.0.102:45783 => /192.168.0.102:31011]
08:18:17.938 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0x6a8a1670, /
192.168.0.102:45784 => /192.168.0.102:31011]
08:18:17.938 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0xf18205a3, /
192.168.0.102:45785 => /192.168.0.102:31011]
08:18:17.939 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0xa09abc2c, /
192.168.0.102:45786 => /192.168.0.102:31011]
08:18:17.943 [BitServer-1] INFO  i.n.handler.logging.LoggingHandler - [id:
0x8ebafd5b, /0:0:0:0:0:0:0:0:31011] RECEIVED: [id: 0x3784f526, /
192.168.0.102:45787 => /192.168.0.102:31011]





On Sat, Aug 24, 2013 at 8:15 AM, Tanujit Ghosh <ta...@gmail.com>wrote:

> Hi,
>
> when i try mvn install after these changes the
> org.apache.drill.exec.store.parquet.ParquetRecordReaderTest is hanging.
>
> Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 1.596 sec
> - in org.apache.drill.exec.expr.ExpressionTest
> Running org.apache.drill.exec.store.TestAffinityCalculator
> Took 0.616287 ms to build range map
> Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.137 sec
> - in org.apache.drill.exec.store.TestAffinityCalculator
> Running org.apache.drill.exec.store.parquet.ParquetRecordReaderTest
> SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
> SLF4J: Defaulting to no-operation (NOP) logger implementation
> SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
> details.
>
> Environment is Fedora 18, open jdk 1.7
>
> with skipTests everything is getting compiled fine.
>
> Regards
> Tanujit
>
>
>
> On Fri, Aug 23, 2013 at 5:36 AM, <ja...@apache.org> wrote:
>
>> DRILL-176:  Updates to affinity calculator, fixes for parquet
>> serialization.  Fix to ErrorHelper looping
>>
>>
>> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
>> Commit:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/7edd3617
>> Tree:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/7edd3617
>> Diff:
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/7edd3617
>>
>> Branch: refs/heads/master
>> Commit: 7edd36170a9be291a69e44f6090474193485bf14
>> Parents: d6ae53e
>> Author: Steven Phillips <sp...@maprtech.com>
>> Authored: Thu Aug 22 16:18:55 2013 -0700
>> Committer: Jacques Nadeau <ja...@apache.org>
>> Committed: Thu Aug 22 16:18:55 2013 -0700
>>
>> ----------------------------------------------------------------------
>>  .../drill/exec/planner/fragment/Wrapper.java    |   5 +-
>>  .../drill/exec/store/AffinityCalculator.java    |  91 ++++++----
>>  .../exec/store/parquet/ParquetGroupScan.java    | 177 +++++++++----------
>>  .../exec/store/parquet/ParquetRecordReader.java |   2 +-
>>  .../store/parquet/ParquetScanBatchCreator.java  |  10 +-
>>  .../drill/exec/work/foreman/ErrorHelper.java    |   8 +-
>>  .../exec/store/TestParquetPhysicalPlan.java     |  55 +++++-
>>  .../store/parquet/ParquetRecordReaderTest.java  |  52 +++++-
>>  .../parquet_scan_union_screen_physical.json     |   5 +-
>>  9 files changed, 257 insertions(+), 148 deletions(-)
>> ----------------------------------------------------------------------
>>
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
>> index d5a24b0..8c4b0b4 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
>> @@ -151,15 +151,12 @@ public class Wrapper {
>>        for (int i = start; i < start + width; i++) {
>>          endpoints.add(all.get(i % div));
>>        }
>> -    } else if (values.size() < width) {
>> -      throw new NotImplementedException(
>> -          "Haven't implemented a scenario where we have some node
>> affinity but the affinity list is smaller than the expected width.");
>>      } else {
>>        // get nodes with highest affinity.
>>        Collections.sort(values);
>>        values = Lists.reverse(values);
>>        for (int i = 0; i < width; i++) {
>> -        endpoints.add(values.get(i).getEndpoint());
>> +        endpoints.add(values.get(i%values.size()).getEndpoint());
>>        }
>>      }
>>
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
>> index b4092cc..b341ea4 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
>> @@ -1,6 +1,7 @@
>>  package org.apache.drill.exec.store;
>>
>>
>> +import com.google.common.base.Stopwatch;
>>  import com.google.common.collect.ImmutableRangeMap;
>>  import com.google.common.collect.Range;
>>  import org.apache.drill.exec.store.parquet.ParquetGroupScan;
>> @@ -13,6 +14,7 @@ import org.apache.hadoop.fs.Path;
>>
>>  import java.io.IOException;
>>  import java.util.*;
>> +import java.util.concurrent.TimeUnit;
>>
>>  public class AffinityCalculator {
>>    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(AffinityCalculator.class);
>> @@ -24,6 +26,7 @@ public class AffinityCalculator {
>>    String fileName;
>>    Collection<DrillbitEndpoint> endpoints;
>>    HashMap<String,DrillbitEndpoint> endPointMap;
>> +  Stopwatch watch = new Stopwatch();
>>
>>    public AffinityCalculator(String fileName, FileSystem fs,
>> Collection<DrillbitEndpoint> endpoints) {
>>      this.fs = fs;
>> @@ -33,16 +36,20 @@ public class AffinityCalculator {
>>      buildEndpointMap();
>>    }
>>
>> +  /**
>> +   * Builds a mapping of block locations to file byte range
>> +   */
>>    private void buildBlockMap() {
>>      try {
>> +      watch.start();
>>        FileStatus file = fs.getFileStatus(new Path(fileName));
>> -      long tC = System.nanoTime();
>>        blocks = fs.getFileBlockLocations(file, 0 , file.getLen());
>> -      long tD = System.nanoTime();
>> +      watch.stop();
>>        logger.debug("Block locations: {}", blocks);
>> -      logger.debug("Took {} ms to get Block locations", (float)(tD - tC)
>> / 1e6);
>> +      logger.debug("Took {} ms to get Block locations",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>      } catch (IOException ioe) { throw new RuntimeException(ioe); }
>> -    long tA = System.nanoTime();
>> +    watch.reset();
>> +    watch.start();
>>      ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new
>> ImmutableRangeMap.Builder<Long,BlockLocation>();
>>      for (BlockLocation block : blocks) {
>>        long start = block.getOffset();
>> @@ -51,62 +58,72 @@ public class AffinityCalculator {
>>        blockMapBuilder = blockMapBuilder.put(range, block);
>>      }
>>      blockMap = blockMapBuilder.build();
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to build block map", (float)(tB - tA) /
>> 1e6);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to build block map",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>    }
>>    /**
>> +   * For a given RowGroup, calculate how many bytes are available on
>> each on drillbit endpoint
>>     *
>> -   * @param entry
>> +   * @param rowGroup the RowGroup to calculate endpoint bytes for
>>     */
>> -  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo entry) {
>> -    long tA = System.nanoTime();
>> +  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo rowGroup) {
>> +    watch.reset();
>> +    watch.start();
>>      HashMap<String,Long> hostMap = new HashMap<>();
>> -    long start = entry.getStart();
>> -    long end = start + entry.getLength();
>> -    Range<Long> entryRange = Range.closedOpen(start, end);
>> -    ImmutableRangeMap<Long,BlockLocation> subRangeMap =
>> blockMap.subRangeMap(entryRange);
>> -    for (Map.Entry<Range<Long>,BlockLocation> e :
>> subRangeMap.asMapOfRanges().entrySet()) {
>> -      String[] hosts = null;
>> -      Range<Long> blockRange = e.getKey();
>> +    HashMap<DrillbitEndpoint,Long> endpointByteMap = new HashMap();
>> +    long start = rowGroup.getStart();
>> +    long end = start + rowGroup.getLength();
>> +    Range<Long> rowGroupRange = Range.closedOpen(start, end);
>> +
>> +    // Find submap of ranges that intersect with the rowGroup
>> +    ImmutableRangeMap<Long,BlockLocation> subRangeMap =
>> blockMap.subRangeMap(rowGroupRange);
>> +
>> +    // Iterate through each block in this submap and get the host for
>> the block location
>> +    for (Map.Entry<Range<Long>,BlockLocation> block :
>> subRangeMap.asMapOfRanges().entrySet()) {
>> +      String[] hosts;
>> +      Range<Long> blockRange = block.getKey();
>>        try {
>> -        hosts = e.getValue().getHosts();
>> -      } catch (IOException ioe) { /*TODO Handle this exception */}
>> -      Range<Long> intersection = entryRange.intersection(blockRange);
>> +        hosts = block.getValue().getHosts();
>> +      } catch (IOException ioe) {
>> +        throw new RuntimeException("Failed to get hosts for block
>> location", ioe);
>> +      }
>> +      Range<Long> intersection = rowGroupRange.intersection(blockRange);
>>        long bytes = intersection.upperEndpoint() -
>> intersection.lowerEndpoint();
>> +
>> +      // For each host in the current block location, add the
>> intersecting bytes to the corresponding endpoint
>>        for (String host : hosts) {
>> -        if (hostMap.containsKey(host)) {
>> -          hostMap.put(host, hostMap.get(host) + bytes);
>> +        DrillbitEndpoint endpoint = getDrillBitEndpoint(host);
>> +        if (endpointByteMap.containsKey(endpoint)) {
>> +          endpointByteMap.put(endpoint, endpointByteMap.get(endpoint) +
>> bytes);
>>          } else {
>> -          hostMap.put(host, bytes);
>> +          if (endpoint != null ) endpointByteMap.put(endpoint, bytes);
>>          }
>>        }
>>      }
>> -    HashMap<DrillbitEndpoint,Long> ebs = new HashMap();
>> -    try {
>> -      for (Map.Entry<String,Long> hostEntry : hostMap.entrySet()) {
>> -        String host = hostEntry.getKey();
>> -        Long bytes = hostEntry.getValue();
>> -        DrillbitEndpoint d = getDrillBitEndpoint(host);
>> -        if (d != null ) ebs.put(d, bytes);
>> -      }
>> -    } catch (NullPointerException n) {}
>> -    entry.setEndpointBytes(ebs);
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to set endpoint bytes", (float)(tB - tA) /
>> 1e6);
>> +
>> +    rowGroup.setEndpointBytes(endpointByteMap);
>> +    rowGroup.setMaxBytes(endpointByteMap.size() > 0 ?
>> Collections.max(endpointByteMap.values()) : 0);
>> +    logger.debug("Row group ({},{}) max bytes {}", rowGroup.getPath(),
>> rowGroup.getStart(), rowGroup.getMaxBytes());
>> +    watch.stop();
>> +    logger.debug("Took {} ms to set endpoint bytes",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>    }
>>
>>    private DrillbitEndpoint getDrillBitEndpoint(String hostName) {
>>      return endPointMap.get(hostName);
>>    }
>>
>> +  /**
>> +   * Builds a mapping of drillbit endpoints to hostnames
>> +   */
>>    private void buildEndpointMap() {
>> -    long tA = System.nanoTime();
>> +    watch.reset();
>> +    watch.start();
>>      endPointMap = new HashMap<String, DrillbitEndpoint>();
>>      for (DrillbitEndpoint d : endpoints) {
>>        String hostName = d.getAddress();
>>        endPointMap.put(hostName, d);
>>      }
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to build endpoint map", (float)(tB - tA) /
>> 1e6);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to build endpoint map",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>    }
>>  }
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
>> index 9e48d33..64ced87 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
>> @@ -18,14 +18,13 @@
>>  package org.apache.drill.exec.store.parquet;
>>
>>  import java.io.IOException;
>> -import java.util.ArrayList;
>> -import java.util.Collection;
>> -import java.util.Collections;
>> -import java.util.Comparator;
>> -import java.util.HashMap;
>> -import java.util.LinkedList;
>> -import java.util.List;
>> +import java.util.*;
>> +import java.util.concurrent.TimeUnit;
>>
>> +import com.google.common.base.Stopwatch;
>> +import com.google.common.collect.ArrayListMultimap;
>> +import com.google.common.collect.Lists;
>> +import com.google.common.collect.Multimap;
>>  import org.apache.drill.common.config.DrillConfig;
>>  import org.apache.drill.exec.exception.SetupException;
>>  import org.apache.drill.exec.physical.EndpointAffinity;
>> @@ -59,8 +58,9 @@ import com.google.common.base.Preconditions;
>>  public class ParquetGroupScan extends AbstractGroupScan {
>>    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class);
>>
>> -  private LinkedList<ParquetRowGroupScan.RowGroupReadEntry>[] mappings;
>> +  private ArrayListMultimap<Integer,
>> ParquetRowGroupScan.RowGroupReadEntry> mappings;
>>    private List<RowGroupInfo> rowGroupInfos;
>> +  private Stopwatch watch = new Stopwatch();
>>
>>    public List<ReadEntryWithPath> getEntries() {
>>      return entries;
>> @@ -110,16 +110,14 @@ public class ParquetGroupScan extends
>> AbstractGroupScan {
>>    }
>>
>>    private void readFooter() throws IOException {
>> -    long tA = System.nanoTime();
>> +    watch.reset();
>> +    watch.start();
>>      rowGroupInfos = new ArrayList();
>>      long start = 0, length = 0;
>>      ColumnChunkMetaData columnChunkMetaData;
>>      for (ReadEntryWithPath readEntryWithPath : entries){
>>        Path path = new Path(readEntryWithPath.getPath());
>>        ParquetMetadata footer =
>> ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), path);
>> -//      FileSystem fs =
>> FileSystem.get(this.storageEngine.getHadoopConfig());
>> -//      FileStatus status = fs.getFileStatus(path);
>> -//      ParquetMetadata footer =
>> ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), status);
>>        readEntryWithPath.getPath();
>>
>>        int i = 0;
>> @@ -138,38 +136,21 @@ public class ParquetGroupScan extends
>> AbstractGroupScan {
>>          i++;
>>        }
>>      }
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to get row group infos", (float)(tB - tA) /
>> 1E6);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to get row group infos",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>    }
>>
>>    private void calculateEndpointBytes() {
>> -    long tA = System.nanoTime();
>> +    watch.reset();
>> +    watch.start();
>>      AffinityCalculator ac = new AffinityCalculator(fileName, fs,
>> availableEndpoints);
>>      for (RowGroupInfo e : rowGroupInfos) {
>>        ac.setEndpointBytes(e);
>>        totalBytes += e.getLength();
>>      }
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to calculate EndpointBytes", (float)(tB -
>> tA) / 1E6);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to calculate EndpointBytes",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>    }
>> -/*
>> -  public LinkedList<RowGroupInfo> getRowGroups() {
>> -    return rowGroups;
>> -  }
>> -
>> -  public void setRowGroups(LinkedList<RowGroupInfo> rowGroups) {
>> -    this.rowGroups = rowGroups;
>> -  }
>> -
>> -  public static class ParquetFileReadEntry {
>> -
>> -    String path;
>> -
>> -    public ParquetFileReadEntry(@JsonProperty String path){
>> -      this.path = path;
>> -    }
>> -  }
>> -  */
>>
>>    @JsonIgnore
>>    public FileSystem getFileSystem() {
>> @@ -232,16 +213,22 @@ public class ParquetGroupScan extends
>> AbstractGroupScan {
>>      }
>>    }
>>
>> +  /**
>> +   *Calculates the affinity each endpoint has for this scan, by adding
>> up the affinity each endpoint has for each
>> +   * rowGroup
>> +   * @return a list of EndpointAffinity objects
>> +   */
>>    @Override
>>    public List<EndpointAffinity> getOperatorAffinity() {
>> -    long tA = System.nanoTime();
>> +    watch.reset();
>> +    watch.start();
>>      if (this.endpointAffinities == null) {
>>        HashMap<DrillbitEndpoint, Float> affinities = new HashMap<>();
>>        for (RowGroupInfo entry : rowGroupInfos) {
>>          for (DrillbitEndpoint d : entry.getEndpointBytes().keySet()) {
>>            long bytes = entry.getEndpointBytes().get(d);
>>            float affinity = (float)bytes / (float)totalBytes;
>> -          logger.error("RowGroup: {} Endpoint: {} Bytes: {}",
>> entry.getRowGroupIndex(), d.getAddress(), bytes);
>> +          logger.debug("RowGroup: {} Endpoint: {} Bytes: {}",
>> entry.getRowGroupIndex(), d.getAddress(), bytes);
>>            if (affinities.keySet().contains(d)) {
>>              affinities.put(d, affinities.get(d) + affinity);
>>            } else {
>> @@ -256,83 +243,90 @@ public class ParquetGroupScan extends
>> AbstractGroupScan {
>>        }
>>        this.endpointAffinities = affinityList;
>>      }
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to get operator affinity", (float)(tB - tA)
>> / 1E6);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to get operator affinity",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>      return this.endpointAffinities;
>>    }
>>
>>
>> +  static final double[] ASSIGNMENT_CUTOFFS = {0.99, 0.50, 0.25, 0.01};
>>
>> -
>> +  /**
>> +   *
>> +   * @param incomingEndpoints
>> +   */
>>    @Override
>> -  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
>> -    long tA = System.nanoTime();
>> -    Preconditions.checkArgument(endpoints.size() <=
>> rowGroupInfos.size());
>> -
>> -    int i = 0;
>> -    for (DrillbitEndpoint endpoint : endpoints) {
>> -      logger.debug("Endpoint index {}, endpoint host: {}", i++,
>> endpoint.getAddress());
>> +  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints)
>> {
>> +    watch.reset();
>> +    watch.start();
>> +    Preconditions.checkArgument(incomingEndpoints.size() <=
>> rowGroupInfos.size());
>> +    mappings = ArrayListMultimap.create();
>> +    ArrayList rowGroupList = new ArrayList(rowGroupInfos);
>> +    List<DrillbitEndpoint> endpointLinkedlist =
>> Lists.newLinkedList(incomingEndpoints);
>> +    for(double cutoff : ASSIGNMENT_CUTOFFS ){
>> +      scanAndAssign(mappings, endpointLinkedlist, rowGroupList, cutoff,
>> false);
>>      }
>> -
>> -    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
>> -    mappings = new LinkedList[endpoints.size()];
>> -    LinkedList<RowGroupInfo> unassigned = scanAndAssign(endpoints,
>> rowGroupInfos, 100, true, false);
>> -    LinkedList<RowGroupInfo> unassigned2 = scanAndAssign(endpoints,
>> unassigned, 50, true, false);
>> -    LinkedList<RowGroupInfo> unassigned3 = scanAndAssign(endpoints,
>> unassigned2, 25, true, false);
>> -    LinkedList<RowGroupInfo> unassigned4 = scanAndAssign(endpoints,
>> unassigned3, 0, false, false);
>> -    LinkedList<RowGroupInfo> unassigned5 = scanAndAssign(endpoints,
>> unassigned4, 0, false, true);
>> -    assert unassigned5.size() == 0 : String.format("All readEntries
>> should be assigned by now, but some are still unassigned");
>> -    long tB = System.nanoTime();
>> -    logger.debug("Took {} ms to apply assignments ", (float)(tB - tA) /
>> 1E6);
>> +    scanAndAssign(mappings, endpointLinkedlist, rowGroupList, 0.0, true);
>> +    watch.stop();
>> +    logger.debug("Took {} ms to apply assignments",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>> +    Preconditions.checkArgument(rowGroupList.isEmpty(), "All readEntries
>> should be assigned by now, but some are still unassigned");
>> +    Preconditions.checkArgument(!rowGroupInfos.isEmpty());
>>    }
>>
>> -  private LinkedList<RowGroupInfo> scanAndAssign (List<DrillbitEndpoint>
>> endpoints, List<RowGroupInfo> rowGroups, int requiredPercentage, boolean
>> mustContain, boolean assignAll) {
>> -    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
>> -    LinkedList<RowGroupInfo> unassigned = new LinkedList<>();
>> -
>> -    int maxEntries = (int) (rowGroupInfos.size() / endpoints.size() *
>> 1.5);
>> +  public int fragmentPointer = 0;
>> +
>> +  /**
>> +   *
>> +   * @param endpointAssignments the mapping between fragment/endpoint
>> and rowGroup
>> +   * @param endpoints the list of drillbits, ordered by the
>> corresponding fragment
>> +   * @param rowGroups the list of rowGroups to assign
>> +   * @param requiredPercentage the percentage of max bytes required to
>> make an assignment
>> +   * @param assignAll if true, will assign even if no affinity
>> +   */
>> +  private void scanAndAssign (Multimap<Integer,
>> ParquetRowGroupScan.RowGroupReadEntry> endpointAssignments,
>> List<DrillbitEndpoint> endpoints, List<RowGroupInfo> rowGroups, double
>> requiredPercentage, boolean assignAll) {
>> +    Collections.sort(rowGroups, new ParquetReadEntryComparator());
>> +    final boolean requireAffinity = requiredPercentage > 0;
>> +    int maxAssignments = (int) (rowGroups.size() / endpoints.size());
>> +
>> +    if (maxAssignments < 1) maxAssignments = 1;
>> +
>> +    for(Iterator<RowGroupInfo> iter = rowGroups.iterator();
>> iter.hasNext();){
>> +      RowGroupInfo rowGroupInfo = iter.next();
>> +      for (int i = 0; i < endpoints.size(); i++) {
>> +        int minorFragmentId = (fragmentPointer + i) % endpoints.size();
>> +        DrillbitEndpoint currentEndpoint =
>> endpoints.get(minorFragmentId);
>> +        Map<DrillbitEndpoint, Long> bytesPerEndpoint =
>> rowGroupInfo.getEndpointBytes();
>> +        boolean haveAffinity =
>> bytesPerEndpoint.containsKey(currentEndpoint) ;
>>
>> -    if (maxEntries < 1) maxEntries = 1;
>> -
>> -    int i =0;
>> -    for(RowGroupInfo e : rowGroups) {
>> -      boolean assigned = false;
>> -      for (int j = i; j < i + endpoints.size(); j++) {
>> -        DrillbitEndpoint currentEndpoint =
>> endpoints.get(j%endpoints.size());
>>          if (assignAll ||
>> -                (e.getEndpointBytes().size() > 0 &&
>> -                (e.getEndpointBytes().containsKey(currentEndpoint) ||
>> !mustContain) &&
>> -                (mappings[j%endpoints.size()] == null ||
>> mappings[j%endpoints.size()].size() < maxEntries) &&
>> -                e.getEndpointBytes().get(currentEndpoint) >=
>> e.getMaxBytes() * requiredPercentage / 100)) {
>> -          LinkedList<ParquetRowGroupScan.RowGroupReadEntry> entries =
>> mappings[j%endpoints.size()];
>> -          if(entries == null){
>> -            entries = new
>> LinkedList<ParquetRowGroupScan.RowGroupReadEntry>();
>> -            mappings[j%endpoints.size()] = entries;
>> -          }
>> -          entries.add(e.getRowGroupReadEntry());
>> -          logger.debug("Assigned rowGroup ( {} , {} ) to endpoint {}",
>> e.getPath(), e.getStart(), currentEndpoint.getAddress());
>> -          assigned = true;
>> +                (!bytesPerEndpoint.isEmpty() &&
>> +                        (!requireAffinity || haveAffinity) &&
>> +
>>  (!endpointAssignments.containsKey(minorFragmentId) ||
>> endpointAssignments.get(minorFragmentId).size() < maxAssignments) &&
>> +                        bytesPerEndpoint.get(currentEndpoint) >=
>> rowGroupInfo.getMaxBytes() * requiredPercentage)) {
>> +
>> +          endpointAssignments.put(minorFragmentId,
>> rowGroupInfo.getRowGroupReadEntry());
>> +          logger.debug("Assigned rowGroup {} to minorFragmentId {}
>> endpoint {}", rowGroupInfo.getRowGroupIndex(), minorFragmentId,
>> endpoints.get(minorFragmentId).getAddress());
>> +          iter.remove();
>> +          fragmentPointer = (minorFragmentId + 1) % endpoints.size();
>>            break;
>>          }
>>        }
>> -      if (!assigned) unassigned.add(e);
>> -      i++;
>> +
>>      }
>> -    return unassigned;
>>    }
>>
>>    @Override
>>    public ParquetRowGroupScan getSpecificScan(int minorFragmentId) {
>> -    assert minorFragmentId < mappings.length : String.format("Mappings
>> length [%d] should be longer than minor fragment id [%d] but it isn't.",
>> mappings.length, minorFragmentId);
>> -    for (ParquetRowGroupScan.RowGroupReadEntry rg :
>> mappings[minorFragmentId]) {
>> +    assert minorFragmentId < mappings.size() : String.format("Mappings
>> length [%d] should be longer than minor fragment id [%d] but it isn't.",
>> mappings.size(), minorFragmentId);
>> +    for (ParquetRowGroupScan.RowGroupReadEntry rg :
>> mappings.get(minorFragmentId)) {
>>        logger.debug("minorFragmentId: {} Path: {} RowGroupIndex:
>> {}",minorFragmentId, rg.getPath(),rg.getRowGroupIndex());
>>      }
>> +
>>  Preconditions.checkArgument(!mappings.get(minorFragmentId).isEmpty(),
>> String.format("MinorFragmentId %d has no read entries assigned",
>> minorFragmentId));
>>      try {
>> -      return new ParquetRowGroupScan(storageEngine, engineConfig,
>> mappings[minorFragmentId]);
>> +      return new ParquetRowGroupScan(storageEngine, engineConfig,
>> mappings.get(minorFragmentId));
>>      } catch (SetupException e) {
>> -      e.printStackTrace(); // TODO - fix this
>> +      throw new RuntimeException("Error setting up ParquetRowGroupScan",
>> e);
>>      }
>> -    return null;
>>    }
>>
>>    @Override
>> @@ -342,7 +336,8 @@ public class ParquetGroupScan extends
>> AbstractGroupScan {
>>
>>    @Override
>>    public OperatorCost getCost() {
>> -    return new OperatorCost(1,1,1,1);
>> +    //TODO Figure out how to properly calculate cost
>> +    return new OperatorCost(1,rowGroupInfos.size(),1,1);
>>    }
>>
>>    @Override
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> index 4e46034..3aaa987 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
>> @@ -211,8 +211,8 @@ public class ParquetRecordReader implements
>> RecordReader {
>>        }
>>        for (VarLenBinaryReader.VarLengthColumn r :
>> varLengthReader.columns) {
>>          output.addField(r.valueVecHolder.getValueVector());
>> -        output.setNewSchema();
>>        }
>> +      output.setNewSchema();
>>      }catch(SchemaChangeException e) {
>>        throw new ExecutionSetupException("Error setting up output
>> mutator.", e);
>>      }
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
>> index 03fb4ec..addd288 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
>> @@ -21,7 +21,9 @@ import java.io.IOException;
>>  import java.text.SimpleDateFormat;
>>  import java.util.Date;
>>  import java.util.List;
>> +import java.util.concurrent.TimeUnit;
>>
>> +import com.google.common.base.Stopwatch;
>>  import org.apache.drill.common.exceptions.ExecutionSetupException;
>>  import org.apache.drill.exec.ops.FragmentContext;
>>  import org.apache.drill.exec.physical.impl.BatchCreator;
>> @@ -40,12 +42,12 @@ import parquet.hadoop.ParquetFileReader;
>>  import parquet.hadoop.metadata.ParquetMetadata;
>>
>>  public class ParquetScanBatchCreator implements
>> BatchCreator<ParquetRowGroupScan>{
>> -  static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
>> +  static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ParquetScanBatchCreator.class);
>>
>>    @Override
>>    public RecordBatch getBatch(FragmentContext context,
>> ParquetRowGroupScan rowGroupScan, List<RecordBatch> children) throws
>> ExecutionSetupException {
>> -    long tA = System.nanoTime(), tB;
>> -    System.out.println( new SimpleDateFormat("mm:ss S").format(new
>> Date()) + " :Start of ScanBatCreator.scanBatch");
>> +    Stopwatch watch = new Stopwatch();
>> +    watch.start();
>>      Preconditions.checkArgument(children.isEmpty());
>>      List<RecordReader> readers = Lists.newArrayList();
>>      for(ParquetRowGroupScan.RowGroupReadEntry e :
>> rowGroupScan.getRowGroupReadEntries()){
>> @@ -68,7 +70,7 @@ public class ParquetScanBatchCreator implements
>> BatchCreator<ParquetRowGroupScan
>>          throw new ExecutionSetupException(e1);
>>        }
>>      }
>> -    System.out.println( "Total time in method: " + ((float)
>> (System.nanoTime() - tA) / 1e9));
>> +    logger.debug("total time in ScanBatchCreator.getBatch: {} ms",
>> watch.elapsed(TimeUnit.MILLISECONDS));
>>      return new ScanBatch(context, readers.iterator());
>>    }
>>  }
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
>> index 9a33109..72c5f34 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
>> @@ -35,8 +35,8 @@ public class ErrorHelper {
>>      if(message != null){
>>        sb.append(message);
>>      }
>> -
>> -    do{
>> +
>> +    while (true) {
>>        sb.append(" < ");
>>        sb.append(t.getClass().getSimpleName());
>>        if(t.getMessage() != null){
>> @@ -44,7 +44,9 @@ public class ErrorHelper {
>>          sb.append(t.getMessage());
>>          sb.append(" ]");
>>        }
>> -    }while(t.getCause() != null && t.getCause() != t);
>> +      if (t.getCause() == null || t.getCause() == t) break;
>> +      t = t.getCause();
>> +    }
>>
>>      builder.setMessage(sb.toString());
>>
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
>> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
>> index e2a00f1..18ac294 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
>> @@ -13,10 +13,16 @@ import org.apache.drill.exec.physical.PhysicalPlan;
>>  import org.apache.drill.exec.planner.PhysicalPlanReader;
>>  import org.apache.drill.exec.proto.CoordinationProtos;
>>  import org.apache.drill.exec.proto.UserProtos;
>> +import org.apache.drill.exec.record.RecordBatchLoader;
>> +import org.apache.drill.exec.record.VectorWrapper;
>> +import org.apache.drill.exec.rpc.RpcException;
>>  import org.apache.drill.exec.rpc.user.QueryResultBatch;
>> +import org.apache.drill.exec.rpc.user.UserResultsListener;
>> +import org.apache.drill.exec.server.BootStrapContext;
>>  import org.apache.drill.exec.server.Drillbit;
>>  import org.apache.drill.exec.server.RemoteServiceSet;
>>  import org.apache.drill.exec.store.parquet.ParquetGroupScan;
>> +import org.apache.drill.exec.vector.ValueVector;
>>  import org.apache.hadoop.fs.BlockLocation;
>>  import org.apache.hadoop.fs.FileStatus;
>>  import org.apache.hadoop.fs.FileSystem;
>> @@ -29,6 +35,7 @@ import java.io.IOException;
>>  import java.nio.charset.Charset;
>>  import java.util.LinkedList;
>>  import java.util.List;
>> +import java.util.concurrent.CountDownLatch;
>>
>>  import static junit.framework.Assert.assertNull;
>>  import static org.junit.Assert.assertEquals;
>> @@ -38,6 +45,8 @@ public class TestParquetPhysicalPlan {
>>
>>    //public String fileName = "/physical_test2.json";
>>    public String fileName = "parquet_scan_union_screen_physical.json";
>> +//  public String fileName = "parquet-sample.json";
>> +
>>
>>    @Test
>>    @Ignore
>> @@ -49,7 +58,51 @@ public class TestParquetPhysicalPlan {
>>        bit1.run();
>>        client.connect();
>>        List<QueryResultBatch> results =
>> client.runQuery(UserProtos.QueryType.PHYSICAL,
>> Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
>> -      System.out.println(String.format("Got %d results",
>> results.size()));
>> +      RecordBatchLoader loader = new
>> RecordBatchLoader(bit1.getContext().getAllocator());
>> +      for (QueryResultBatch b : results) {
>> +        System.out.println(String.format("Got %d results",
>> b.getHeader().getRowCount()));
>> +        loader.load(b.getHeader().getDef(), b.getData());
>> +        for (VectorWrapper vw : loader) {
>> +          System.out.println(vw.getValueVector().getField().getName());
>> +          ValueVector vv = vw.getValueVector();
>> +          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
>> +            Object o = vv.getAccessor().getObject(i);
>> +            System.out.println(vv.getAccessor().getObject(i));
>> +          }
>> +        }
>> +      }
>> +      client.close();
>> +    }
>> +  }
>> +
>> +  private class ParquetResultsListener implements UserResultsListener {
>> +    private CountDownLatch latch = new CountDownLatch(1);
>> +    @Override
>> +    public void submissionFailed(RpcException ex) {
>> +      logger.error("submission failed", ex);
>> +      latch.countDown();
>> +    }
>> +
>> +    @Override
>> +    public void resultArrived(QueryResultBatch result) {
>> +      System.out.printf("Result batch arrived. Number of records: %d",
>> result.getHeader().getRowCount());
>> +      if (result.getHeader().getIsLastChunk()) latch.countDown();
>> +    }
>> +
>> +    public void await() throws Exception {
>> +      latch.await();
>> +    }
>> +  }
>> +  @Test
>> +  @Ignore
>> +  public void testParseParquetPhysicalPlanRemote() throws Exception {
>> +    DrillConfig config = DrillConfig.create();
>> +
>> +    try(DrillClient client = new DrillClient(config);){
>> +      client.connect();
>> +      ParquetResultsListener listener = new ParquetResultsListener();
>> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
>> Resources.toString(Resources.getResource(fileName),Charsets.UTF_8),
>> listener);
>> +      listener.await();
>>        client.close();
>>      }
>>    }
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
>> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
>> index 1d91455..7a99c3f 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
>> +++
>> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
>> @@ -48,6 +48,7 @@ import
>> org.apache.drill.exec.store.parquet.TestFileGenerator.FieldInfo;
>>  import org.apache.drill.exec.vector.BaseDataValueVector;
>>  import org.apache.drill.exec.vector.ValueVector;
>>  import org.junit.BeforeClass;
>> +import org.junit.Ignore;
>>  import org.junit.Test;
>>
>>  import parquet.bytes.BytesInput;
>> @@ -68,6 +69,7 @@ public class ParquetRecordReaderTest {
>>    static final org.slf4j.Logger logger =
>> org.slf4j.LoggerFactory.getLogger(ParquetRecordReaderTest.class);
>>
>>    private boolean VERBOSE_DEBUG = false;
>> +  private boolean checkValues = true;
>>
>>    static final int numberRowGroups = 20;
>>    static final int recordsPerRowGroup = 300000;
>> @@ -100,6 +102,20 @@ public class ParquetRecordReaderTest {
>>      testParquetFullEngineLocalText(planText, fileName, i,
>> numberRowGroups, recordsPerRowGroup);
>>    }
>>
>> +  @Test
>> +  @Ignore
>> +  public void testLocalDistributed() throws Exception {
>> +    String planName = "/parquet/parquet_scan_union_screen_physical.json";
>> +    testParquetFullEngineLocalTextDistributed(planName, fileName, 1, 20,
>> 300000);
>> +  }
>> +
>> +  @Test
>> +  @Ignore
>> +  public void testRemoteDistributed() throws Exception {
>> +    String planName = "/parquet/parquet_scan_union_screen_physical.json";
>> +    testParquetFullEngineRemote(planName, fileName, 1, 10, 30000);
>> +  }
>> +
>>
>>    private class ParquetResultListener implements UserResultsListener {
>>      private SettableFuture<Void> future = SettableFuture.create();
>> @@ -155,8 +171,12 @@ public class ParquetRecordReaderTest {
>>            if (VERBOSE_DEBUG){
>>              System.out.print(vv.getAccessor().getObject(j) + ", " + (j %
>> 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
>>            }
>> -          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
>> -              currentField.values[(int) (columnValCounter % 3)],
>> (String) currentField.name + "/");
>> +          if (checkValues) {
>> +            try {
>> +              assertField(vv, j, (TypeProtos.MinorType)
>> currentField.type,
>> +                currentField.values[(int) (columnValCounter % 3)],
>> (String) currentField.name + "/");
>> +            } catch (AssertionError e) { submissionFailed(new
>> RpcException(e)); }
>> +          }
>>            columnValCounter++;
>>          }
>>          if (VERBOSE_DEBUG){
>> @@ -197,7 +217,9 @@ public class ParquetRecordReaderTest {
>>        batchCounter++;
>>        if(result.getHeader().getIsLastChunk()){
>>          for (String s : valuesChecked.keySet()) {
>> +          try {
>>            assertEquals("Record count incorrect for column: " + s,
>> totalRecords, (long) valuesChecked.get(s));
>> +          } catch (AssertionError e) { submissionFailed(new
>> RpcException(e)); }
>>          }
>>
>>          assert valuesChecked.keySet().size() > 0;
>> @@ -222,11 +244,13 @@ public class ParquetRecordReaderTest {
>>
>>      DrillConfig config = DrillConfig.create();
>>
>> +    checkValues = false;
>> +
>>      try(DrillClient client = new DrillClient(config);){
>>        client.connect();
>>        RecordBatchLoader batchLoader = new
>> RecordBatchLoader(client.getAllocator());
>>        ParquetResultListener resultListener = new
>> ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups,
>> numberOfTimesRead);
>> -      client.runQuery(UserProtos.QueryType.LOGICAL,
>> Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8),
>> resultListener);
>> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
>> Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8),
>> resultListener);
>>        resultListener.get();
>>      }
>>
>> @@ -259,6 +283,28 @@ public class ParquetRecordReaderTest {
>>    }
>>
>>
>> +  //use this method to submit physical plan
>> +  public void testParquetFullEngineLocalTextDistributed(String planName,
>> String filename, int numberOfTimesRead /* specified in json plan */, int
>> numberOfRowGroups, int recordsPerRowGroup) throws Exception{
>> +
>> +    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
>> +
>> +    checkValues = false;
>> +
>> +    DrillConfig config = DrillConfig.create();
>> +
>> +    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient
>> client = new DrillClient(config, serviceSet.getCoordinator());){
>> +      bit1.run();
>> +      client.connect();
>> +      RecordBatchLoader batchLoader = new
>> RecordBatchLoader(client.getAllocator());
>> +      ParquetResultListener resultListener = new
>> ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups,
>> numberOfTimesRead);
>> +      Stopwatch watch = new Stopwatch().start();
>> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
>> Files.toString(FileUtils.getResourceAsFile(planName), Charsets.UTF_8),
>> resultListener);
>> +      resultListener.get();
>> +      System.out.println(String.format("Took %d ms to run query",
>> watch.elapsed(TimeUnit.MILLISECONDS)));
>> +
>> +    }
>> +
>> +  }
>>
>>    public String pad(String value, int length) {
>>      return pad(value, length, " ");
>>
>>
>> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
>> ----------------------------------------------------------------------
>> diff --git
>> a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
>> b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
>> index f508d09..5efecaf 100644
>> ---
>> a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
>> +++
>> b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
>> @@ -11,10 +11,7 @@
>>      @id : 1,
>>      entries : [
>>      {
>> -        path : "/tmp/testParquetFile_many_types_3"
>> -    },
>> -    {
>> -        path : "/tmp/testParquetFile_many_types_3"
>> +        path : "/tmp/parquet_test_file_many_types"
>>      }
>>      ],
>>      storageengine:{
>>
>>
>
>
> --
> Regards,
> Tanujit
>



-- 
Regards,
Tanujit

Re: [5/8] git commit: DRILL-176: Updates to affinity calculator, fixes for parquet serialization. Fix to ErrorHelper looping

Posted by Tanujit Ghosh <ta...@gmail.com>.
Hi,

when i try mvn install after these changes the
org.apache.drill.exec.store.parquet.ParquetRecordReaderTest is hanging.

Tests run: 3, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 1.596 sec -
in org.apache.drill.exec.expr.ExpressionTest
Running org.apache.drill.exec.store.TestAffinityCalculator
Took 0.616287 ms to build range map
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.137 sec -
in org.apache.drill.exec.store.TestAffinityCalculator
Running org.apache.drill.exec.store.parquet.ParquetRecordReaderTest
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further
details.

Environment is Fedora 18, open jdk 1.7

with skipTests everything is getting compiled fine.

Regards
Tanujit



On Fri, Aug 23, 2013 at 5:36 AM, <ja...@apache.org> wrote:

> DRILL-176:  Updates to affinity calculator, fixes for parquet
> serialization.  Fix to ErrorHelper looping
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/7edd3617
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/7edd3617
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/7edd3617
>
> Branch: refs/heads/master
> Commit: 7edd36170a9be291a69e44f6090474193485bf14
> Parents: d6ae53e
> Author: Steven Phillips <sp...@maprtech.com>
> Authored: Thu Aug 22 16:18:55 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Thu Aug 22 16:18:55 2013 -0700
>
> ----------------------------------------------------------------------
>  .../drill/exec/planner/fragment/Wrapper.java    |   5 +-
>  .../drill/exec/store/AffinityCalculator.java    |  91 ++++++----
>  .../exec/store/parquet/ParquetGroupScan.java    | 177 +++++++++----------
>  .../exec/store/parquet/ParquetRecordReader.java |   2 +-
>  .../store/parquet/ParquetScanBatchCreator.java  |  10 +-
>  .../drill/exec/work/foreman/ErrorHelper.java    |   8 +-
>  .../exec/store/TestParquetPhysicalPlan.java     |  55 +++++-
>  .../store/parquet/ParquetRecordReaderTest.java  |  52 +++++-
>  .../parquet_scan_union_screen_physical.json     |   5 +-
>  9 files changed, 257 insertions(+), 148 deletions(-)
> ----------------------------------------------------------------------
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
> index d5a24b0..8c4b0b4 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
> @@ -151,15 +151,12 @@ public class Wrapper {
>        for (int i = start; i < start + width; i++) {
>          endpoints.add(all.get(i % div));
>        }
> -    } else if (values.size() < width) {
> -      throw new NotImplementedException(
> -          "Haven't implemented a scenario where we have some node
> affinity but the affinity list is smaller than the expected width.");
>      } else {
>        // get nodes with highest affinity.
>        Collections.sort(values);
>        values = Lists.reverse(values);
>        for (int i = 0; i < width; i++) {
> -        endpoints.add(values.get(i).getEndpoint());
> +        endpoints.add(values.get(i%values.size()).getEndpoint());
>        }
>      }
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
> index b4092cc..b341ea4 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
> @@ -1,6 +1,7 @@
>  package org.apache.drill.exec.store;
>
>
> +import com.google.common.base.Stopwatch;
>  import com.google.common.collect.ImmutableRangeMap;
>  import com.google.common.collect.Range;
>  import org.apache.drill.exec.store.parquet.ParquetGroupScan;
> @@ -13,6 +14,7 @@ import org.apache.hadoop.fs.Path;
>
>  import java.io.IOException;
>  import java.util.*;
> +import java.util.concurrent.TimeUnit;
>
>  public class AffinityCalculator {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(AffinityCalculator.class);
> @@ -24,6 +26,7 @@ public class AffinityCalculator {
>    String fileName;
>    Collection<DrillbitEndpoint> endpoints;
>    HashMap<String,DrillbitEndpoint> endPointMap;
> +  Stopwatch watch = new Stopwatch();
>
>    public AffinityCalculator(String fileName, FileSystem fs,
> Collection<DrillbitEndpoint> endpoints) {
>      this.fs = fs;
> @@ -33,16 +36,20 @@ public class AffinityCalculator {
>      buildEndpointMap();
>    }
>
> +  /**
> +   * Builds a mapping of block locations to file byte range
> +   */
>    private void buildBlockMap() {
>      try {
> +      watch.start();
>        FileStatus file = fs.getFileStatus(new Path(fileName));
> -      long tC = System.nanoTime();
>        blocks = fs.getFileBlockLocations(file, 0 , file.getLen());
> -      long tD = System.nanoTime();
> +      watch.stop();
>        logger.debug("Block locations: {}", blocks);
> -      logger.debug("Took {} ms to get Block locations", (float)(tD - tC)
> / 1e6);
> +      logger.debug("Took {} ms to get Block locations",
> watch.elapsed(TimeUnit.MILLISECONDS));
>      } catch (IOException ioe) { throw new RuntimeException(ioe); }
> -    long tA = System.nanoTime();
> +    watch.reset();
> +    watch.start();
>      ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new
> ImmutableRangeMap.Builder<Long,BlockLocation>();
>      for (BlockLocation block : blocks) {
>        long start = block.getOffset();
> @@ -51,62 +58,72 @@ public class AffinityCalculator {
>        blockMapBuilder = blockMapBuilder.put(range, block);
>      }
>      blockMap = blockMapBuilder.build();
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to build block map", (float)(tB - tA) / 1e6);
> +    watch.stop();
> +    logger.debug("Took {} ms to build block map",
> watch.elapsed(TimeUnit.MILLISECONDS));
>    }
>    /**
> +   * For a given RowGroup, calculate how many bytes are available on each
> on drillbit endpoint
>     *
> -   * @param entry
> +   * @param rowGroup the RowGroup to calculate endpoint bytes for
>     */
> -  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo entry) {
> -    long tA = System.nanoTime();
> +  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo rowGroup) {
> +    watch.reset();
> +    watch.start();
>      HashMap<String,Long> hostMap = new HashMap<>();
> -    long start = entry.getStart();
> -    long end = start + entry.getLength();
> -    Range<Long> entryRange = Range.closedOpen(start, end);
> -    ImmutableRangeMap<Long,BlockLocation> subRangeMap =
> blockMap.subRangeMap(entryRange);
> -    for (Map.Entry<Range<Long>,BlockLocation> e :
> subRangeMap.asMapOfRanges().entrySet()) {
> -      String[] hosts = null;
> -      Range<Long> blockRange = e.getKey();
> +    HashMap<DrillbitEndpoint,Long> endpointByteMap = new HashMap();
> +    long start = rowGroup.getStart();
> +    long end = start + rowGroup.getLength();
> +    Range<Long> rowGroupRange = Range.closedOpen(start, end);
> +
> +    // Find submap of ranges that intersect with the rowGroup
> +    ImmutableRangeMap<Long,BlockLocation> subRangeMap =
> blockMap.subRangeMap(rowGroupRange);
> +
> +    // Iterate through each block in this submap and get the host for the
> block location
> +    for (Map.Entry<Range<Long>,BlockLocation> block :
> subRangeMap.asMapOfRanges().entrySet()) {
> +      String[] hosts;
> +      Range<Long> blockRange = block.getKey();
>        try {
> -        hosts = e.getValue().getHosts();
> -      } catch (IOException ioe) { /*TODO Handle this exception */}
> -      Range<Long> intersection = entryRange.intersection(blockRange);
> +        hosts = block.getValue().getHosts();
> +      } catch (IOException ioe) {
> +        throw new RuntimeException("Failed to get hosts for block
> location", ioe);
> +      }
> +      Range<Long> intersection = rowGroupRange.intersection(blockRange);
>        long bytes = intersection.upperEndpoint() -
> intersection.lowerEndpoint();
> +
> +      // For each host in the current block location, add the
> intersecting bytes to the corresponding endpoint
>        for (String host : hosts) {
> -        if (hostMap.containsKey(host)) {
> -          hostMap.put(host, hostMap.get(host) + bytes);
> +        DrillbitEndpoint endpoint = getDrillBitEndpoint(host);
> +        if (endpointByteMap.containsKey(endpoint)) {
> +          endpointByteMap.put(endpoint, endpointByteMap.get(endpoint) +
> bytes);
>          } else {
> -          hostMap.put(host, bytes);
> +          if (endpoint != null ) endpointByteMap.put(endpoint, bytes);
>          }
>        }
>      }
> -    HashMap<DrillbitEndpoint,Long> ebs = new HashMap();
> -    try {
> -      for (Map.Entry<String,Long> hostEntry : hostMap.entrySet()) {
> -        String host = hostEntry.getKey();
> -        Long bytes = hostEntry.getValue();
> -        DrillbitEndpoint d = getDrillBitEndpoint(host);
> -        if (d != null ) ebs.put(d, bytes);
> -      }
> -    } catch (NullPointerException n) {}
> -    entry.setEndpointBytes(ebs);
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to set endpoint bytes", (float)(tB - tA) /
> 1e6);
> +
> +    rowGroup.setEndpointBytes(endpointByteMap);
> +    rowGroup.setMaxBytes(endpointByteMap.size() > 0 ?
> Collections.max(endpointByteMap.values()) : 0);
> +    logger.debug("Row group ({},{}) max bytes {}", rowGroup.getPath(),
> rowGroup.getStart(), rowGroup.getMaxBytes());
> +    watch.stop();
> +    logger.debug("Took {} ms to set endpoint bytes",
> watch.elapsed(TimeUnit.MILLISECONDS));
>    }
>
>    private DrillbitEndpoint getDrillBitEndpoint(String hostName) {
>      return endPointMap.get(hostName);
>    }
>
> +  /**
> +   * Builds a mapping of drillbit endpoints to hostnames
> +   */
>    private void buildEndpointMap() {
> -    long tA = System.nanoTime();
> +    watch.reset();
> +    watch.start();
>      endPointMap = new HashMap<String, DrillbitEndpoint>();
>      for (DrillbitEndpoint d : endpoints) {
>        String hostName = d.getAddress();
>        endPointMap.put(hostName, d);
>      }
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to build endpoint map", (float)(tB - tA) /
> 1e6);
> +    watch.stop();
> +    logger.debug("Took {} ms to build endpoint map",
> watch.elapsed(TimeUnit.MILLISECONDS));
>    }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
> index 9e48d33..64ced87 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
> @@ -18,14 +18,13 @@
>  package org.apache.drill.exec.store.parquet;
>
>  import java.io.IOException;
> -import java.util.ArrayList;
> -import java.util.Collection;
> -import java.util.Collections;
> -import java.util.Comparator;
> -import java.util.HashMap;
> -import java.util.LinkedList;
> -import java.util.List;
> +import java.util.*;
> +import java.util.concurrent.TimeUnit;
>
> +import com.google.common.base.Stopwatch;
> +import com.google.common.collect.ArrayListMultimap;
> +import com.google.common.collect.Lists;
> +import com.google.common.collect.Multimap;
>  import org.apache.drill.common.config.DrillConfig;
>  import org.apache.drill.exec.exception.SetupException;
>  import org.apache.drill.exec.physical.EndpointAffinity;
> @@ -59,8 +58,9 @@ import com.google.common.base.Preconditions;
>  public class ParquetGroupScan extends AbstractGroupScan {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class);
>
> -  private LinkedList<ParquetRowGroupScan.RowGroupReadEntry>[] mappings;
> +  private ArrayListMultimap<Integer,
> ParquetRowGroupScan.RowGroupReadEntry> mappings;
>    private List<RowGroupInfo> rowGroupInfos;
> +  private Stopwatch watch = new Stopwatch();
>
>    public List<ReadEntryWithPath> getEntries() {
>      return entries;
> @@ -110,16 +110,14 @@ public class ParquetGroupScan extends
> AbstractGroupScan {
>    }
>
>    private void readFooter() throws IOException {
> -    long tA = System.nanoTime();
> +    watch.reset();
> +    watch.start();
>      rowGroupInfos = new ArrayList();
>      long start = 0, length = 0;
>      ColumnChunkMetaData columnChunkMetaData;
>      for (ReadEntryWithPath readEntryWithPath : entries){
>        Path path = new Path(readEntryWithPath.getPath());
>        ParquetMetadata footer =
> ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), path);
> -//      FileSystem fs =
> FileSystem.get(this.storageEngine.getHadoopConfig());
> -//      FileStatus status = fs.getFileStatus(path);
> -//      ParquetMetadata footer =
> ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), status);
>        readEntryWithPath.getPath();
>
>        int i = 0;
> @@ -138,38 +136,21 @@ public class ParquetGroupScan extends
> AbstractGroupScan {
>          i++;
>        }
>      }
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to get row group infos", (float)(tB - tA) /
> 1E6);
> +    watch.stop();
> +    logger.debug("Took {} ms to get row group infos",
> watch.elapsed(TimeUnit.MILLISECONDS));
>    }
>
>    private void calculateEndpointBytes() {
> -    long tA = System.nanoTime();
> +    watch.reset();
> +    watch.start();
>      AffinityCalculator ac = new AffinityCalculator(fileName, fs,
> availableEndpoints);
>      for (RowGroupInfo e : rowGroupInfos) {
>        ac.setEndpointBytes(e);
>        totalBytes += e.getLength();
>      }
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to calculate EndpointBytes", (float)(tB -
> tA) / 1E6);
> +    watch.stop();
> +    logger.debug("Took {} ms to calculate EndpointBytes",
> watch.elapsed(TimeUnit.MILLISECONDS));
>    }
> -/*
> -  public LinkedList<RowGroupInfo> getRowGroups() {
> -    return rowGroups;
> -  }
> -
> -  public void setRowGroups(LinkedList<RowGroupInfo> rowGroups) {
> -    this.rowGroups = rowGroups;
> -  }
> -
> -  public static class ParquetFileReadEntry {
> -
> -    String path;
> -
> -    public ParquetFileReadEntry(@JsonProperty String path){
> -      this.path = path;
> -    }
> -  }
> -  */
>
>    @JsonIgnore
>    public FileSystem getFileSystem() {
> @@ -232,16 +213,22 @@ public class ParquetGroupScan extends
> AbstractGroupScan {
>      }
>    }
>
> +  /**
> +   *Calculates the affinity each endpoint has for this scan, by adding up
> the affinity each endpoint has for each
> +   * rowGroup
> +   * @return a list of EndpointAffinity objects
> +   */
>    @Override
>    public List<EndpointAffinity> getOperatorAffinity() {
> -    long tA = System.nanoTime();
> +    watch.reset();
> +    watch.start();
>      if (this.endpointAffinities == null) {
>        HashMap<DrillbitEndpoint, Float> affinities = new HashMap<>();
>        for (RowGroupInfo entry : rowGroupInfos) {
>          for (DrillbitEndpoint d : entry.getEndpointBytes().keySet()) {
>            long bytes = entry.getEndpointBytes().get(d);
>            float affinity = (float)bytes / (float)totalBytes;
> -          logger.error("RowGroup: {} Endpoint: {} Bytes: {}",
> entry.getRowGroupIndex(), d.getAddress(), bytes);
> +          logger.debug("RowGroup: {} Endpoint: {} Bytes: {}",
> entry.getRowGroupIndex(), d.getAddress(), bytes);
>            if (affinities.keySet().contains(d)) {
>              affinities.put(d, affinities.get(d) + affinity);
>            } else {
> @@ -256,83 +243,90 @@ public class ParquetGroupScan extends
> AbstractGroupScan {
>        }
>        this.endpointAffinities = affinityList;
>      }
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to get operator affinity", (float)(tB - tA)
> / 1E6);
> +    watch.stop();
> +    logger.debug("Took {} ms to get operator affinity",
> watch.elapsed(TimeUnit.MILLISECONDS));
>      return this.endpointAffinities;
>    }
>
>
> +  static final double[] ASSIGNMENT_CUTOFFS = {0.99, 0.50, 0.25, 0.01};
>
> -
> +  /**
> +   *
> +   * @param incomingEndpoints
> +   */
>    @Override
> -  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
> -    long tA = System.nanoTime();
> -    Preconditions.checkArgument(endpoints.size() <= rowGroupInfos.size());
> -
> -    int i = 0;
> -    for (DrillbitEndpoint endpoint : endpoints) {
> -      logger.debug("Endpoint index {}, endpoint host: {}", i++,
> endpoint.getAddress());
> +  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
> +    watch.reset();
> +    watch.start();
> +    Preconditions.checkArgument(incomingEndpoints.size() <=
> rowGroupInfos.size());
> +    mappings = ArrayListMultimap.create();
> +    ArrayList rowGroupList = new ArrayList(rowGroupInfos);
> +    List<DrillbitEndpoint> endpointLinkedlist =
> Lists.newLinkedList(incomingEndpoints);
> +    for(double cutoff : ASSIGNMENT_CUTOFFS ){
> +      scanAndAssign(mappings, endpointLinkedlist, rowGroupList, cutoff,
> false);
>      }
> -
> -    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
> -    mappings = new LinkedList[endpoints.size()];
> -    LinkedList<RowGroupInfo> unassigned = scanAndAssign(endpoints,
> rowGroupInfos, 100, true, false);
> -    LinkedList<RowGroupInfo> unassigned2 = scanAndAssign(endpoints,
> unassigned, 50, true, false);
> -    LinkedList<RowGroupInfo> unassigned3 = scanAndAssign(endpoints,
> unassigned2, 25, true, false);
> -    LinkedList<RowGroupInfo> unassigned4 = scanAndAssign(endpoints,
> unassigned3, 0, false, false);
> -    LinkedList<RowGroupInfo> unassigned5 = scanAndAssign(endpoints,
> unassigned4, 0, false, true);
> -    assert unassigned5.size() == 0 : String.format("All readEntries
> should be assigned by now, but some are still unassigned");
> -    long tB = System.nanoTime();
> -    logger.debug("Took {} ms to apply assignments ", (float)(tB - tA) /
> 1E6);
> +    scanAndAssign(mappings, endpointLinkedlist, rowGroupList, 0.0, true);
> +    watch.stop();
> +    logger.debug("Took {} ms to apply assignments",
> watch.elapsed(TimeUnit.MILLISECONDS));
> +    Preconditions.checkArgument(rowGroupList.isEmpty(), "All readEntries
> should be assigned by now, but some are still unassigned");
> +    Preconditions.checkArgument(!rowGroupInfos.isEmpty());
>    }
>
> -  private LinkedList<RowGroupInfo> scanAndAssign (List<DrillbitEndpoint>
> endpoints, List<RowGroupInfo> rowGroups, int requiredPercentage, boolean
> mustContain, boolean assignAll) {
> -    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
> -    LinkedList<RowGroupInfo> unassigned = new LinkedList<>();
> -
> -    int maxEntries = (int) (rowGroupInfos.size() / endpoints.size() *
> 1.5);
> +  public int fragmentPointer = 0;
> +
> +  /**
> +   *
> +   * @param endpointAssignments the mapping between fragment/endpoint and
> rowGroup
> +   * @param endpoints the list of drillbits, ordered by the corresponding
> fragment
> +   * @param rowGroups the list of rowGroups to assign
> +   * @param requiredPercentage the percentage of max bytes required to
> make an assignment
> +   * @param assignAll if true, will assign even if no affinity
> +   */
> +  private void scanAndAssign (Multimap<Integer,
> ParquetRowGroupScan.RowGroupReadEntry> endpointAssignments,
> List<DrillbitEndpoint> endpoints, List<RowGroupInfo> rowGroups, double
> requiredPercentage, boolean assignAll) {
> +    Collections.sort(rowGroups, new ParquetReadEntryComparator());
> +    final boolean requireAffinity = requiredPercentage > 0;
> +    int maxAssignments = (int) (rowGroups.size() / endpoints.size());
> +
> +    if (maxAssignments < 1) maxAssignments = 1;
> +
> +    for(Iterator<RowGroupInfo> iter = rowGroups.iterator();
> iter.hasNext();){
> +      RowGroupInfo rowGroupInfo = iter.next();
> +      for (int i = 0; i < endpoints.size(); i++) {
> +        int minorFragmentId = (fragmentPointer + i) % endpoints.size();
> +        DrillbitEndpoint currentEndpoint = endpoints.get(minorFragmentId);
> +        Map<DrillbitEndpoint, Long> bytesPerEndpoint =
> rowGroupInfo.getEndpointBytes();
> +        boolean haveAffinity =
> bytesPerEndpoint.containsKey(currentEndpoint) ;
>
> -    if (maxEntries < 1) maxEntries = 1;
> -
> -    int i =0;
> -    for(RowGroupInfo e : rowGroups) {
> -      boolean assigned = false;
> -      for (int j = i; j < i + endpoints.size(); j++) {
> -        DrillbitEndpoint currentEndpoint =
> endpoints.get(j%endpoints.size());
>          if (assignAll ||
> -                (e.getEndpointBytes().size() > 0 &&
> -                (e.getEndpointBytes().containsKey(currentEndpoint) ||
> !mustContain) &&
> -                (mappings[j%endpoints.size()] == null ||
> mappings[j%endpoints.size()].size() < maxEntries) &&
> -                e.getEndpointBytes().get(currentEndpoint) >=
> e.getMaxBytes() * requiredPercentage / 100)) {
> -          LinkedList<ParquetRowGroupScan.RowGroupReadEntry> entries =
> mappings[j%endpoints.size()];
> -          if(entries == null){
> -            entries = new
> LinkedList<ParquetRowGroupScan.RowGroupReadEntry>();
> -            mappings[j%endpoints.size()] = entries;
> -          }
> -          entries.add(e.getRowGroupReadEntry());
> -          logger.debug("Assigned rowGroup ( {} , {} ) to endpoint {}",
> e.getPath(), e.getStart(), currentEndpoint.getAddress());
> -          assigned = true;
> +                (!bytesPerEndpoint.isEmpty() &&
> +                        (!requireAffinity || haveAffinity) &&
> +
>  (!endpointAssignments.containsKey(minorFragmentId) ||
> endpointAssignments.get(minorFragmentId).size() < maxAssignments) &&
> +                        bytesPerEndpoint.get(currentEndpoint) >=
> rowGroupInfo.getMaxBytes() * requiredPercentage)) {
> +
> +          endpointAssignments.put(minorFragmentId,
> rowGroupInfo.getRowGroupReadEntry());
> +          logger.debug("Assigned rowGroup {} to minorFragmentId {}
> endpoint {}", rowGroupInfo.getRowGroupIndex(), minorFragmentId,
> endpoints.get(minorFragmentId).getAddress());
> +          iter.remove();
> +          fragmentPointer = (minorFragmentId + 1) % endpoints.size();
>            break;
>          }
>        }
> -      if (!assigned) unassigned.add(e);
> -      i++;
> +
>      }
> -    return unassigned;
>    }
>
>    @Override
>    public ParquetRowGroupScan getSpecificScan(int minorFragmentId) {
> -    assert minorFragmentId < mappings.length : String.format("Mappings
> length [%d] should be longer than minor fragment id [%d] but it isn't.",
> mappings.length, minorFragmentId);
> -    for (ParquetRowGroupScan.RowGroupReadEntry rg :
> mappings[minorFragmentId]) {
> +    assert minorFragmentId < mappings.size() : String.format("Mappings
> length [%d] should be longer than minor fragment id [%d] but it isn't.",
> mappings.size(), minorFragmentId);
> +    for (ParquetRowGroupScan.RowGroupReadEntry rg :
> mappings.get(minorFragmentId)) {
>        logger.debug("minorFragmentId: {} Path: {} RowGroupIndex:
> {}",minorFragmentId, rg.getPath(),rg.getRowGroupIndex());
>      }
> +    Preconditions.checkArgument(!mappings.get(minorFragmentId).isEmpty(),
> String.format("MinorFragmentId %d has no read entries assigned",
> minorFragmentId));
>      try {
> -      return new ParquetRowGroupScan(storageEngine, engineConfig,
> mappings[minorFragmentId]);
> +      return new ParquetRowGroupScan(storageEngine, engineConfig,
> mappings.get(minorFragmentId));
>      } catch (SetupException e) {
> -      e.printStackTrace(); // TODO - fix this
> +      throw new RuntimeException("Error setting up ParquetRowGroupScan",
> e);
>      }
> -    return null;
>    }
>
>    @Override
> @@ -342,7 +336,8 @@ public class ParquetGroupScan extends
> AbstractGroupScan {
>
>    @Override
>    public OperatorCost getCost() {
> -    return new OperatorCost(1,1,1,1);
> +    //TODO Figure out how to properly calculate cost
> +    return new OperatorCost(1,rowGroupInfos.size(),1,1);
>    }
>
>    @Override
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> index 4e46034..3aaa987 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
> @@ -211,8 +211,8 @@ public class ParquetRecordReader implements
> RecordReader {
>        }
>        for (VarLenBinaryReader.VarLengthColumn r :
> varLengthReader.columns) {
>          output.addField(r.valueVecHolder.getValueVector());
> -        output.setNewSchema();
>        }
> +      output.setNewSchema();
>      }catch(SchemaChangeException e) {
>        throw new ExecutionSetupException("Error setting up output
> mutator.", e);
>      }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
> index 03fb4ec..addd288 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
> @@ -21,7 +21,9 @@ import java.io.IOException;
>  import java.text.SimpleDateFormat;
>  import java.util.Date;
>  import java.util.List;
> +import java.util.concurrent.TimeUnit;
>
> +import com.google.common.base.Stopwatch;
>  import org.apache.drill.common.exceptions.ExecutionSetupException;
>  import org.apache.drill.exec.ops.FragmentContext;
>  import org.apache.drill.exec.physical.impl.BatchCreator;
> @@ -40,12 +42,12 @@ import parquet.hadoop.ParquetFileReader;
>  import parquet.hadoop.metadata.ParquetMetadata;
>
>  public class ParquetScanBatchCreator implements
> BatchCreator<ParquetRowGroupScan>{
> -  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
> +  static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ParquetScanBatchCreator.class);
>
>    @Override
>    public RecordBatch getBatch(FragmentContext context,
> ParquetRowGroupScan rowGroupScan, List<RecordBatch> children) throws
> ExecutionSetupException {
> -    long tA = System.nanoTime(), tB;
> -    System.out.println( new SimpleDateFormat("mm:ss S").format(new
> Date()) + " :Start of ScanBatCreator.scanBatch");
> +    Stopwatch watch = new Stopwatch();
> +    watch.start();
>      Preconditions.checkArgument(children.isEmpty());
>      List<RecordReader> readers = Lists.newArrayList();
>      for(ParquetRowGroupScan.RowGroupReadEntry e :
> rowGroupScan.getRowGroupReadEntries()){
> @@ -68,7 +70,7 @@ public class ParquetScanBatchCreator implements
> BatchCreator<ParquetRowGroupScan
>          throw new ExecutionSetupException(e1);
>        }
>      }
> -    System.out.println( "Total time in method: " + ((float)
> (System.nanoTime() - tA) / 1e9));
> +    logger.debug("total time in ScanBatchCreator.getBatch: {} ms",
> watch.elapsed(TimeUnit.MILLISECONDS));
>      return new ScanBatch(context, readers.iterator());
>    }
>  }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
> index 9a33109..72c5f34 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
> +++
> b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
> @@ -35,8 +35,8 @@ public class ErrorHelper {
>      if(message != null){
>        sb.append(message);
>      }
> -
> -    do{
> +
> +    while (true) {
>        sb.append(" < ");
>        sb.append(t.getClass().getSimpleName());
>        if(t.getMessage() != null){
> @@ -44,7 +44,9 @@ public class ErrorHelper {
>          sb.append(t.getMessage());
>          sb.append(" ]");
>        }
> -    }while(t.getCause() != null && t.getCause() != t);
> +      if (t.getCause() == null || t.getCause() == t) break;
> +      t = t.getCause();
> +    }
>
>      builder.setMessage(sb.toString());
>
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
> index e2a00f1..18ac294 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
> @@ -13,10 +13,16 @@ import org.apache.drill.exec.physical.PhysicalPlan;
>  import org.apache.drill.exec.planner.PhysicalPlanReader;
>  import org.apache.drill.exec.proto.CoordinationProtos;
>  import org.apache.drill.exec.proto.UserProtos;
> +import org.apache.drill.exec.record.RecordBatchLoader;
> +import org.apache.drill.exec.record.VectorWrapper;
> +import org.apache.drill.exec.rpc.RpcException;
>  import org.apache.drill.exec.rpc.user.QueryResultBatch;
> +import org.apache.drill.exec.rpc.user.UserResultsListener;
> +import org.apache.drill.exec.server.BootStrapContext;
>  import org.apache.drill.exec.server.Drillbit;
>  import org.apache.drill.exec.server.RemoteServiceSet;
>  import org.apache.drill.exec.store.parquet.ParquetGroupScan;
> +import org.apache.drill.exec.vector.ValueVector;
>  import org.apache.hadoop.fs.BlockLocation;
>  import org.apache.hadoop.fs.FileStatus;
>  import org.apache.hadoop.fs.FileSystem;
> @@ -29,6 +35,7 @@ import java.io.IOException;
>  import java.nio.charset.Charset;
>  import java.util.LinkedList;
>  import java.util.List;
> +import java.util.concurrent.CountDownLatch;
>
>  import static junit.framework.Assert.assertNull;
>  import static org.junit.Assert.assertEquals;
> @@ -38,6 +45,8 @@ public class TestParquetPhysicalPlan {
>
>    //public String fileName = "/physical_test2.json";
>    public String fileName = "parquet_scan_union_screen_physical.json";
> +//  public String fileName = "parquet-sample.json";
> +
>
>    @Test
>    @Ignore
> @@ -49,7 +58,51 @@ public class TestParquetPhysicalPlan {
>        bit1.run();
>        client.connect();
>        List<QueryResultBatch> results =
> client.runQuery(UserProtos.QueryType.PHYSICAL,
> Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
> -      System.out.println(String.format("Got %d results", results.size()));
> +      RecordBatchLoader loader = new
> RecordBatchLoader(bit1.getContext().getAllocator());
> +      for (QueryResultBatch b : results) {
> +        System.out.println(String.format("Got %d results",
> b.getHeader().getRowCount()));
> +        loader.load(b.getHeader().getDef(), b.getData());
> +        for (VectorWrapper vw : loader) {
> +          System.out.println(vw.getValueVector().getField().getName());
> +          ValueVector vv = vw.getValueVector();
> +          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
> +            Object o = vv.getAccessor().getObject(i);
> +            System.out.println(vv.getAccessor().getObject(i));
> +          }
> +        }
> +      }
> +      client.close();
> +    }
> +  }
> +
> +  private class ParquetResultsListener implements UserResultsListener {
> +    private CountDownLatch latch = new CountDownLatch(1);
> +    @Override
> +    public void submissionFailed(RpcException ex) {
> +      logger.error("submission failed", ex);
> +      latch.countDown();
> +    }
> +
> +    @Override
> +    public void resultArrived(QueryResultBatch result) {
> +      System.out.printf("Result batch arrived. Number of records: %d",
> result.getHeader().getRowCount());
> +      if (result.getHeader().getIsLastChunk()) latch.countDown();
> +    }
> +
> +    public void await() throws Exception {
> +      latch.await();
> +    }
> +  }
> +  @Test
> +  @Ignore
> +  public void testParseParquetPhysicalPlanRemote() throws Exception {
> +    DrillConfig config = DrillConfig.create();
> +
> +    try(DrillClient client = new DrillClient(config);){
> +      client.connect();
> +      ParquetResultsListener listener = new ParquetResultsListener();
> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
> Resources.toString(Resources.getResource(fileName),Charsets.UTF_8),
> listener);
> +      listener.await();
>        client.close();
>      }
>    }
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
> index 1d91455..7a99c3f 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
> +++
> b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
> @@ -48,6 +48,7 @@ import
> org.apache.drill.exec.store.parquet.TestFileGenerator.FieldInfo;
>  import org.apache.drill.exec.vector.BaseDataValueVector;
>  import org.apache.drill.exec.vector.ValueVector;
>  import org.junit.BeforeClass;
> +import org.junit.Ignore;
>  import org.junit.Test;
>
>  import parquet.bytes.BytesInput;
> @@ -68,6 +69,7 @@ public class ParquetRecordReaderTest {
>    static final org.slf4j.Logger logger =
> org.slf4j.LoggerFactory.getLogger(ParquetRecordReaderTest.class);
>
>    private boolean VERBOSE_DEBUG = false;
> +  private boolean checkValues = true;
>
>    static final int numberRowGroups = 20;
>    static final int recordsPerRowGroup = 300000;
> @@ -100,6 +102,20 @@ public class ParquetRecordReaderTest {
>      testParquetFullEngineLocalText(planText, fileName, i,
> numberRowGroups, recordsPerRowGroup);
>    }
>
> +  @Test
> +  @Ignore
> +  public void testLocalDistributed() throws Exception {
> +    String planName = "/parquet/parquet_scan_union_screen_physical.json";
> +    testParquetFullEngineLocalTextDistributed(planName, fileName, 1, 20,
> 300000);
> +  }
> +
> +  @Test
> +  @Ignore
> +  public void testRemoteDistributed() throws Exception {
> +    String planName = "/parquet/parquet_scan_union_screen_physical.json";
> +    testParquetFullEngineRemote(planName, fileName, 1, 10, 30000);
> +  }
> +
>
>    private class ParquetResultListener implements UserResultsListener {
>      private SettableFuture<Void> future = SettableFuture.create();
> @@ -155,8 +171,12 @@ public class ParquetRecordReaderTest {
>            if (VERBOSE_DEBUG){
>              System.out.print(vv.getAccessor().getObject(j) + ", " + (j %
> 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
>            }
> -          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
> -              currentField.values[(int) (columnValCounter % 3)], (String)
> currentField.name + "/");
> +          if (checkValues) {
> +            try {
> +              assertField(vv, j, (TypeProtos.MinorType) currentField.type,
> +                currentField.values[(int) (columnValCounter % 3)],
> (String) currentField.name + "/");
> +            } catch (AssertionError e) { submissionFailed(new
> RpcException(e)); }
> +          }
>            columnValCounter++;
>          }
>          if (VERBOSE_DEBUG){
> @@ -197,7 +217,9 @@ public class ParquetRecordReaderTest {
>        batchCounter++;
>        if(result.getHeader().getIsLastChunk()){
>          for (String s : valuesChecked.keySet()) {
> +          try {
>            assertEquals("Record count incorrect for column: " + s,
> totalRecords, (long) valuesChecked.get(s));
> +          } catch (AssertionError e) { submissionFailed(new
> RpcException(e)); }
>          }
>
>          assert valuesChecked.keySet().size() > 0;
> @@ -222,11 +244,13 @@ public class ParquetRecordReaderTest {
>
>      DrillConfig config = DrillConfig.create();
>
> +    checkValues = false;
> +
>      try(DrillClient client = new DrillClient(config);){
>        client.connect();
>        RecordBatchLoader batchLoader = new
> RecordBatchLoader(client.getAllocator());
>        ParquetResultListener resultListener = new
> ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups,
> numberOfTimesRead);
> -      client.runQuery(UserProtos.QueryType.LOGICAL,
> Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8),
> resultListener);
> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
> Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8),
> resultListener);
>        resultListener.get();
>      }
>
> @@ -259,6 +283,28 @@ public class ParquetRecordReaderTest {
>    }
>
>
> +  //use this method to submit physical plan
> +  public void testParquetFullEngineLocalTextDistributed(String planName,
> String filename, int numberOfTimesRead /* specified in json plan */, int
> numberOfRowGroups, int recordsPerRowGroup) throws Exception{
> +
> +    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
> +
> +    checkValues = false;
> +
> +    DrillConfig config = DrillConfig.create();
> +
> +    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient
> client = new DrillClient(config, serviceSet.getCoordinator());){
> +      bit1.run();
> +      client.connect();
> +      RecordBatchLoader batchLoader = new
> RecordBatchLoader(client.getAllocator());
> +      ParquetResultListener resultListener = new
> ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups,
> numberOfTimesRead);
> +      Stopwatch watch = new Stopwatch().start();
> +      client.runQuery(UserProtos.QueryType.PHYSICAL,
> Files.toString(FileUtils.getResourceAsFile(planName), Charsets.UTF_8),
> resultListener);
> +      resultListener.get();
> +      System.out.println(String.format("Took %d ms to run query",
> watch.elapsed(TimeUnit.MILLISECONDS)));
> +
> +    }
> +
> +  }
>
>    public String pad(String value, int length) {
>      return pad(value, length, " ");
>
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
> ----------------------------------------------------------------------
> diff --git
> a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
> b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
> index f508d09..5efecaf 100644
> ---
> a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
> +++
> b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
> @@ -11,10 +11,7 @@
>      @id : 1,
>      entries : [
>      {
> -        path : "/tmp/testParquetFile_many_types_3"
> -    },
> -    {
> -        path : "/tmp/testParquetFile_many_types_3"
> +        path : "/tmp/parquet_test_file_many_types"
>      }
>      ],
>      storageengine:{
>
>


-- 
Regards,
Tanujit

[5/8] git commit: DRILL-176: Updates to affinity calculator, fixes for parquet serialization. Fix to ErrorHelper looping

Posted by ja...@apache.org.
DRILL-176:  Updates to affinity calculator, fixes for parquet serialization.  Fix to ErrorHelper looping


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

Branch: refs/heads/master
Commit: 7edd36170a9be291a69e44f6090474193485bf14
Parents: d6ae53e
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Aug 22 16:18:55 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:18:55 2013 -0700

----------------------------------------------------------------------
 .../drill/exec/planner/fragment/Wrapper.java    |   5 +-
 .../drill/exec/store/AffinityCalculator.java    |  91 ++++++----
 .../exec/store/parquet/ParquetGroupScan.java    | 177 +++++++++----------
 .../exec/store/parquet/ParquetRecordReader.java |   2 +-
 .../store/parquet/ParquetScanBatchCreator.java  |  10 +-
 .../drill/exec/work/foreman/ErrorHelper.java    |   8 +-
 .../exec/store/TestParquetPhysicalPlan.java     |  55 +++++-
 .../store/parquet/ParquetRecordReaderTest.java  |  52 +++++-
 .../parquet_scan_union_screen_physical.json     |   5 +-
 9 files changed, 257 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
index d5a24b0..8c4b0b4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
@@ -151,15 +151,12 @@ public class Wrapper {
       for (int i = start; i < start + width; i++) {
         endpoints.add(all.get(i % div));
       }
-    } else if (values.size() < width) {
-      throw new NotImplementedException(
-          "Haven't implemented a scenario where we have some node affinity but the affinity list is smaller than the expected width.");
     } else {
       // get nodes with highest affinity.
       Collections.sort(values);
       values = Lists.reverse(values);
       for (int i = 0; i < width; i++) {
-        endpoints.add(values.get(i).getEndpoint());
+        endpoints.add(values.get(i%values.size()).getEndpoint());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
index b4092cc..b341ea4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AffinityCalculator.java
@@ -1,6 +1,7 @@
 package org.apache.drill.exec.store;
 
 
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableRangeMap;
 import com.google.common.collect.Range;
 import org.apache.drill.exec.store.parquet.ParquetGroupScan;
@@ -13,6 +14,7 @@ import org.apache.hadoop.fs.Path;
 
 import java.io.IOException;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 public class AffinityCalculator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AffinityCalculator.class);
@@ -24,6 +26,7 @@ public class AffinityCalculator {
   String fileName;
   Collection<DrillbitEndpoint> endpoints;
   HashMap<String,DrillbitEndpoint> endPointMap;
+  Stopwatch watch = new Stopwatch();
 
   public AffinityCalculator(String fileName, FileSystem fs, Collection<DrillbitEndpoint> endpoints) {
     this.fs = fs;
@@ -33,16 +36,20 @@ public class AffinityCalculator {
     buildEndpointMap();
   }
 
+  /**
+   * Builds a mapping of block locations to file byte range
+   */
   private void buildBlockMap() {
     try {
+      watch.start();
       FileStatus file = fs.getFileStatus(new Path(fileName));
-      long tC = System.nanoTime();
       blocks = fs.getFileBlockLocations(file, 0 , file.getLen());
-      long tD = System.nanoTime();
+      watch.stop();
       logger.debug("Block locations: {}", blocks);
-      logger.debug("Took {} ms to get Block locations", (float)(tD - tC) / 1e6);
+      logger.debug("Took {} ms to get Block locations", watch.elapsed(TimeUnit.MILLISECONDS));
     } catch (IOException ioe) { throw new RuntimeException(ioe); }
-    long tA = System.nanoTime();
+    watch.reset();
+    watch.start();
     ImmutableRangeMap.Builder<Long, BlockLocation> blockMapBuilder = new ImmutableRangeMap.Builder<Long,BlockLocation>();
     for (BlockLocation block : blocks) {
       long start = block.getOffset();
@@ -51,62 +58,72 @@ public class AffinityCalculator {
       blockMapBuilder = blockMapBuilder.put(range, block);
     }
     blockMap = blockMapBuilder.build();
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to build block map", (float)(tB - tA) / 1e6);
+    watch.stop();
+    logger.debug("Took {} ms to build block map", watch.elapsed(TimeUnit.MILLISECONDS));
   }
   /**
+   * For a given RowGroup, calculate how many bytes are available on each on drillbit endpoint
    *
-   * @param entry
+   * @param rowGroup the RowGroup to calculate endpoint bytes for
    */
-  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo entry) {
-    long tA = System.nanoTime();
+  public void setEndpointBytes(ParquetGroupScan.RowGroupInfo rowGroup) {
+    watch.reset();
+    watch.start();
     HashMap<String,Long> hostMap = new HashMap<>();
-    long start = entry.getStart();
-    long end = start + entry.getLength();
-    Range<Long> entryRange = Range.closedOpen(start, end);
-    ImmutableRangeMap<Long,BlockLocation> subRangeMap = blockMap.subRangeMap(entryRange);
-    for (Map.Entry<Range<Long>,BlockLocation> e : subRangeMap.asMapOfRanges().entrySet()) {
-      String[] hosts = null;
-      Range<Long> blockRange = e.getKey();
+    HashMap<DrillbitEndpoint,Long> endpointByteMap = new HashMap();
+    long start = rowGroup.getStart();
+    long end = start + rowGroup.getLength();
+    Range<Long> rowGroupRange = Range.closedOpen(start, end);
+
+    // Find submap of ranges that intersect with the rowGroup
+    ImmutableRangeMap<Long,BlockLocation> subRangeMap = blockMap.subRangeMap(rowGroupRange);
+
+    // Iterate through each block in this submap and get the host for the block location
+    for (Map.Entry<Range<Long>,BlockLocation> block : subRangeMap.asMapOfRanges().entrySet()) {
+      String[] hosts;
+      Range<Long> blockRange = block.getKey();
       try {
-        hosts = e.getValue().getHosts();
-      } catch (IOException ioe) { /*TODO Handle this exception */}
-      Range<Long> intersection = entryRange.intersection(blockRange);
+        hosts = block.getValue().getHosts();
+      } catch (IOException ioe) {
+        throw new RuntimeException("Failed to get hosts for block location", ioe);
+      }
+      Range<Long> intersection = rowGroupRange.intersection(blockRange);
       long bytes = intersection.upperEndpoint() - intersection.lowerEndpoint();
+
+      // For each host in the current block location, add the intersecting bytes to the corresponding endpoint
       for (String host : hosts) {
-        if (hostMap.containsKey(host)) {
-          hostMap.put(host, hostMap.get(host) + bytes);
+        DrillbitEndpoint endpoint = getDrillBitEndpoint(host);
+        if (endpointByteMap.containsKey(endpoint)) {
+          endpointByteMap.put(endpoint, endpointByteMap.get(endpoint) + bytes);
         } else {
-          hostMap.put(host, bytes);
+          if (endpoint != null ) endpointByteMap.put(endpoint, bytes);
         }
       }
     }
-    HashMap<DrillbitEndpoint,Long> ebs = new HashMap();
-    try {
-      for (Map.Entry<String,Long> hostEntry : hostMap.entrySet()) {
-        String host = hostEntry.getKey();
-        Long bytes = hostEntry.getValue();
-        DrillbitEndpoint d = getDrillBitEndpoint(host);
-        if (d != null ) ebs.put(d, bytes);
-      }
-    } catch (NullPointerException n) {}
-    entry.setEndpointBytes(ebs);
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to set endpoint bytes", (float)(tB - tA) / 1e6);
+
+    rowGroup.setEndpointBytes(endpointByteMap);
+    rowGroup.setMaxBytes(endpointByteMap.size() > 0 ? Collections.max(endpointByteMap.values()) : 0);
+    logger.debug("Row group ({},{}) max bytes {}", rowGroup.getPath(), rowGroup.getStart(), rowGroup.getMaxBytes());
+    watch.stop();
+    logger.debug("Took {} ms to set endpoint bytes", watch.elapsed(TimeUnit.MILLISECONDS));
   }
 
   private DrillbitEndpoint getDrillBitEndpoint(String hostName) {
     return endPointMap.get(hostName);
   }
 
+  /**
+   * Builds a mapping of drillbit endpoints to hostnames
+   */
   private void buildEndpointMap() {
-    long tA = System.nanoTime();
+    watch.reset();
+    watch.start();
     endPointMap = new HashMap<String, DrillbitEndpoint>();
     for (DrillbitEndpoint d : endpoints) {
       String hostName = d.getAddress();
       endPointMap.put(hostName, d);
     }
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to build endpoint map", (float)(tB - tA) / 1e6);
+    watch.stop();
+    logger.debug("Took {} ms to build endpoint map", watch.elapsed(TimeUnit.MILLISECONDS));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 9e48d33..64ced87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -18,14 +18,13 @@
 package org.apache.drill.exec.store.parquet;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.exception.SetupException;
 import org.apache.drill.exec.physical.EndpointAffinity;
@@ -59,8 +58,9 @@ import com.google.common.base.Preconditions;
 public class ParquetGroupScan extends AbstractGroupScan {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetGroupScan.class);
 
-  private LinkedList<ParquetRowGroupScan.RowGroupReadEntry>[] mappings;
+  private ArrayListMultimap<Integer, ParquetRowGroupScan.RowGroupReadEntry> mappings;
   private List<RowGroupInfo> rowGroupInfos;
+  private Stopwatch watch = new Stopwatch();
 
   public List<ReadEntryWithPath> getEntries() {
     return entries;
@@ -110,16 +110,14 @@ public class ParquetGroupScan extends AbstractGroupScan {
   }
 
   private void readFooter() throws IOException {
-    long tA = System.nanoTime();
+    watch.reset();
+    watch.start();
     rowGroupInfos = new ArrayList();
     long start = 0, length = 0;
     ColumnChunkMetaData columnChunkMetaData;
     for (ReadEntryWithPath readEntryWithPath : entries){
       Path path = new Path(readEntryWithPath.getPath());
       ParquetMetadata footer = ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), path);
-//      FileSystem fs = FileSystem.get(this.storageEngine.getHadoopConfig());
-//      FileStatus status = fs.getFileStatus(path);
-//      ParquetMetadata footer = ParquetFileReader.readFooter(this.storageEngine.getHadoopConfig(), status);
       readEntryWithPath.getPath();
 
       int i = 0;
@@ -138,38 +136,21 @@ public class ParquetGroupScan extends AbstractGroupScan {
         i++;
       }
     }
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to get row group infos", (float)(tB - tA) / 1E6);
+    watch.stop();
+    logger.debug("Took {} ms to get row group infos", watch.elapsed(TimeUnit.MILLISECONDS));
   }
 
   private void calculateEndpointBytes() {
-    long tA = System.nanoTime();
+    watch.reset();
+    watch.start();
     AffinityCalculator ac = new AffinityCalculator(fileName, fs, availableEndpoints);
     for (RowGroupInfo e : rowGroupInfos) {
       ac.setEndpointBytes(e);
       totalBytes += e.getLength();
     }
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to calculate EndpointBytes", (float)(tB - tA) / 1E6);
+    watch.stop();
+    logger.debug("Took {} ms to calculate EndpointBytes", watch.elapsed(TimeUnit.MILLISECONDS));
   }
-/*
-  public LinkedList<RowGroupInfo> getRowGroups() {
-    return rowGroups;
-  }
-
-  public void setRowGroups(LinkedList<RowGroupInfo> rowGroups) {
-    this.rowGroups = rowGroups;
-  }
-
-  public static class ParquetFileReadEntry {
-
-    String path;
-
-    public ParquetFileReadEntry(@JsonProperty String path){
-      this.path = path;
-    }
-  }
-  */
 
   @JsonIgnore
   public FileSystem getFileSystem() {
@@ -232,16 +213,22 @@ public class ParquetGroupScan extends AbstractGroupScan {
     }
   }
 
+  /**
+   *Calculates the affinity each endpoint has for this scan, by adding up the affinity each endpoint has for each
+   * rowGroup
+   * @return a list of EndpointAffinity objects
+   */
   @Override
   public List<EndpointAffinity> getOperatorAffinity() {
-    long tA = System.nanoTime();
+    watch.reset();
+    watch.start();
     if (this.endpointAffinities == null) {
       HashMap<DrillbitEndpoint, Float> affinities = new HashMap<>();
       for (RowGroupInfo entry : rowGroupInfos) {
         for (DrillbitEndpoint d : entry.getEndpointBytes().keySet()) {
           long bytes = entry.getEndpointBytes().get(d);
           float affinity = (float)bytes / (float)totalBytes;
-          logger.error("RowGroup: {} Endpoint: {} Bytes: {}", entry.getRowGroupIndex(), d.getAddress(), bytes);
+          logger.debug("RowGroup: {} Endpoint: {} Bytes: {}", entry.getRowGroupIndex(), d.getAddress(), bytes);
           if (affinities.keySet().contains(d)) {
             affinities.put(d, affinities.get(d) + affinity);
           } else {
@@ -256,83 +243,90 @@ public class ParquetGroupScan extends AbstractGroupScan {
       }
       this.endpointAffinities = affinityList;
     }
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to get operator affinity", (float)(tB - tA) / 1E6);
+    watch.stop();
+    logger.debug("Took {} ms to get operator affinity", watch.elapsed(TimeUnit.MILLISECONDS));
     return this.endpointAffinities;
   }
 
 
+  static final double[] ASSIGNMENT_CUTOFFS = {0.99, 0.50, 0.25, 0.01};
 
-
+  /**
+   *
+   * @param incomingEndpoints
+   */
   @Override
-  public void applyAssignments(List<DrillbitEndpoint> endpoints) {
-    long tA = System.nanoTime();
-    Preconditions.checkArgument(endpoints.size() <= rowGroupInfos.size());
-
-    int i = 0;
-    for (DrillbitEndpoint endpoint : endpoints) {
-      logger.debug("Endpoint index {}, endpoint host: {}", i++, endpoint.getAddress());
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    watch.reset();
+    watch.start();
+    Preconditions.checkArgument(incomingEndpoints.size() <= rowGroupInfos.size());
+    mappings = ArrayListMultimap.create();
+    ArrayList rowGroupList = new ArrayList(rowGroupInfos);
+    List<DrillbitEndpoint> endpointLinkedlist = Lists.newLinkedList(incomingEndpoints);
+    for(double cutoff : ASSIGNMENT_CUTOFFS ){
+      scanAndAssign(mappings, endpointLinkedlist, rowGroupList, cutoff, false);
     }
-
-    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
-    mappings = new LinkedList[endpoints.size()];
-    LinkedList<RowGroupInfo> unassigned = scanAndAssign(endpoints, rowGroupInfos, 100, true, false);
-    LinkedList<RowGroupInfo> unassigned2 = scanAndAssign(endpoints, unassigned, 50, true, false);
-    LinkedList<RowGroupInfo> unassigned3 = scanAndAssign(endpoints, unassigned2, 25, true, false);
-    LinkedList<RowGroupInfo> unassigned4 = scanAndAssign(endpoints, unassigned3, 0, false, false);
-    LinkedList<RowGroupInfo> unassigned5 = scanAndAssign(endpoints, unassigned4, 0, false, true);
-    assert unassigned5.size() == 0 : String.format("All readEntries should be assigned by now, but some are still unassigned");
-    long tB = System.nanoTime();
-    logger.debug("Took {} ms to apply assignments ", (float)(tB - tA) / 1E6);
+    scanAndAssign(mappings, endpointLinkedlist, rowGroupList, 0.0, true);
+    watch.stop();
+    logger.debug("Took {} ms to apply assignments", watch.elapsed(TimeUnit.MILLISECONDS));
+    Preconditions.checkArgument(rowGroupList.isEmpty(), "All readEntries should be assigned by now, but some are still unassigned");
+    Preconditions.checkArgument(!rowGroupInfos.isEmpty());
   }
 
-  private LinkedList<RowGroupInfo> scanAndAssign (List<DrillbitEndpoint> endpoints, List<RowGroupInfo> rowGroups, int requiredPercentage, boolean mustContain, boolean assignAll) {
-    Collections.sort(rowGroupInfos, new ParquetReadEntryComparator());
-    LinkedList<RowGroupInfo> unassigned = new LinkedList<>();
-
-    int maxEntries = (int) (rowGroupInfos.size() / endpoints.size() * 1.5);
+  public int fragmentPointer = 0;
+
+  /**
+   *
+   * @param endpointAssignments the mapping between fragment/endpoint and rowGroup
+   * @param endpoints the list of drillbits, ordered by the corresponding fragment
+   * @param rowGroups the list of rowGroups to assign
+   * @param requiredPercentage the percentage of max bytes required to make an assignment
+   * @param assignAll if true, will assign even if no affinity
+   */
+  private void scanAndAssign (Multimap<Integer, ParquetRowGroupScan.RowGroupReadEntry> endpointAssignments, List<DrillbitEndpoint> endpoints, List<RowGroupInfo> rowGroups, double requiredPercentage, boolean assignAll) {
+    Collections.sort(rowGroups, new ParquetReadEntryComparator());
+    final boolean requireAffinity = requiredPercentage > 0;
+    int maxAssignments = (int) (rowGroups.size() / endpoints.size());
+
+    if (maxAssignments < 1) maxAssignments = 1;
+
+    for(Iterator<RowGroupInfo> iter = rowGroups.iterator(); iter.hasNext();){
+      RowGroupInfo rowGroupInfo = iter.next();
+      for (int i = 0; i < endpoints.size(); i++) {
+        int minorFragmentId = (fragmentPointer + i) % endpoints.size();
+        DrillbitEndpoint currentEndpoint = endpoints.get(minorFragmentId);
+        Map<DrillbitEndpoint, Long> bytesPerEndpoint = rowGroupInfo.getEndpointBytes();
+        boolean haveAffinity = bytesPerEndpoint.containsKey(currentEndpoint) ;
 
-    if (maxEntries < 1) maxEntries = 1;
-
-    int i =0;
-    for(RowGroupInfo e : rowGroups) {
-      boolean assigned = false;
-      for (int j = i; j < i + endpoints.size(); j++) {
-        DrillbitEndpoint currentEndpoint = endpoints.get(j%endpoints.size());
         if (assignAll ||
-                (e.getEndpointBytes().size() > 0 &&
-                (e.getEndpointBytes().containsKey(currentEndpoint) || !mustContain) &&
-                (mappings[j%endpoints.size()] == null || mappings[j%endpoints.size()].size() < maxEntries) &&
-                e.getEndpointBytes().get(currentEndpoint) >= e.getMaxBytes() * requiredPercentage / 100)) {
-          LinkedList<ParquetRowGroupScan.RowGroupReadEntry> entries = mappings[j%endpoints.size()];
-          if(entries == null){
-            entries = new LinkedList<ParquetRowGroupScan.RowGroupReadEntry>();
-            mappings[j%endpoints.size()] = entries;
-          }
-          entries.add(e.getRowGroupReadEntry());
-          logger.debug("Assigned rowGroup ( {} , {} ) to endpoint {}", e.getPath(), e.getStart(), currentEndpoint.getAddress());
-          assigned = true;
+                (!bytesPerEndpoint.isEmpty() &&
+                        (!requireAffinity || haveAffinity) &&
+                        (!endpointAssignments.containsKey(minorFragmentId) || endpointAssignments.get(minorFragmentId).size() < maxAssignments) &&
+                        bytesPerEndpoint.get(currentEndpoint) >= rowGroupInfo.getMaxBytes() * requiredPercentage)) {
+
+          endpointAssignments.put(minorFragmentId, rowGroupInfo.getRowGroupReadEntry());
+          logger.debug("Assigned rowGroup {} to minorFragmentId {} endpoint {}", rowGroupInfo.getRowGroupIndex(), minorFragmentId, endpoints.get(minorFragmentId).getAddress());
+          iter.remove();
+          fragmentPointer = (minorFragmentId + 1) % endpoints.size();
           break;
         }
       }
-      if (!assigned) unassigned.add(e);
-      i++;
+
     }
-    return unassigned;
   }
 
   @Override
   public ParquetRowGroupScan getSpecificScan(int minorFragmentId) {
-    assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId);
-    for (ParquetRowGroupScan.RowGroupReadEntry rg : mappings[minorFragmentId]) {
+    assert minorFragmentId < mappings.size() : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.size(), minorFragmentId);
+    for (ParquetRowGroupScan.RowGroupReadEntry rg : mappings.get(minorFragmentId)) {
       logger.debug("minorFragmentId: {} Path: {} RowGroupIndex: {}",minorFragmentId, rg.getPath(),rg.getRowGroupIndex());
     }
+    Preconditions.checkArgument(!mappings.get(minorFragmentId).isEmpty(), String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId));
     try {
-      return new ParquetRowGroupScan(storageEngine, engineConfig, mappings[minorFragmentId]);
+      return new ParquetRowGroupScan(storageEngine, engineConfig, mappings.get(minorFragmentId));
     } catch (SetupException e) {
-      e.printStackTrace(); // TODO - fix this
+      throw new RuntimeException("Error setting up ParquetRowGroupScan", e);
     }
-    return null;
   }
 
   @Override
@@ -342,7 +336,8 @@ public class ParquetGroupScan extends AbstractGroupScan {
 
   @Override
   public OperatorCost getCost() {
-    return new OperatorCost(1,1,1,1);
+    //TODO Figure out how to properly calculate cost
+    return new OperatorCost(1,rowGroupInfos.size(),1,1);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index 4e46034..3aaa987 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -211,8 +211,8 @@ public class ParquetRecordReader implements RecordReader {
       }
       for (VarLenBinaryReader.VarLengthColumn r : varLengthReader.columns) {
         output.addField(r.valueVecHolder.getValueVector());
-        output.setNewSchema();
       }
+      output.setNewSchema();
     }catch(SchemaChangeException e) {
       throw new ExecutionSetupException("Error setting up output mutator.", e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index 03fb4ec..addd288 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -21,7 +21,9 @@ import java.io.IOException;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Stopwatch;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.BatchCreator;
@@ -40,12 +42,12 @@ import parquet.hadoop.ParquetFileReader;
 import parquet.hadoop.metadata.ParquetMetadata;
 
 public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan>{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetScanBatchCreator.class);
 
   @Override
   public RecordBatch getBatch(FragmentContext context, ParquetRowGroupScan rowGroupScan, List<RecordBatch> children) throws ExecutionSetupException {
-    long tA = System.nanoTime(), tB;
-    System.out.println( new SimpleDateFormat("mm:ss S").format(new Date()) + " :Start of ScanBatCreator.scanBatch");
+    Stopwatch watch = new Stopwatch();
+    watch.start();
     Preconditions.checkArgument(children.isEmpty());
     List<RecordReader> readers = Lists.newArrayList();
     for(ParquetRowGroupScan.RowGroupReadEntry e : rowGroupScan.getRowGroupReadEntries()){
@@ -68,7 +70,7 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
         throw new ExecutionSetupException(e1);
       }
     }
-    System.out.println( "Total time in method: " + ((float) (System.nanoTime() - tA) / 1e9));
+    logger.debug("total time in ScanBatchCreator.getBatch: {} ms", watch.elapsed(TimeUnit.MILLISECONDS));
     return new ScanBatch(context, readers.iterator());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
index 9a33109..72c5f34 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
@@ -35,8 +35,8 @@ public class ErrorHelper {
     if(message != null){
       sb.append(message);
     }
-      
-    do{
+
+    while (true) {
       sb.append(" < ");
       sb.append(t.getClass().getSimpleName());
       if(t.getMessage() != null){
@@ -44,7 +44,9 @@ public class ErrorHelper {
         sb.append(t.getMessage());
         sb.append(" ]");
       }
-    }while(t.getCause() != null && t.getCause() != t);
+      if (t.getCause() == null || t.getCause() == t) break;
+      t = t.getCause();
+    }
     
     builder.setMessage(sb.toString());
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
index e2a00f1..18ac294 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestParquetPhysicalPlan.java
@@ -13,10 +13,16 @@ import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.parquet.ParquetGroupScan;
+import org.apache.drill.exec.vector.ValueVector;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -29,6 +35,7 @@ import java.io.IOException;
 import java.nio.charset.Charset;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
 
 import static junit.framework.Assert.assertNull;
 import static org.junit.Assert.assertEquals;
@@ -38,6 +45,8 @@ public class TestParquetPhysicalPlan {
 
   //public String fileName = "/physical_test2.json";
   public String fileName = "parquet_scan_union_screen_physical.json";
+//  public String fileName = "parquet-sample.json";
+
 
   @Test
   @Ignore
@@ -49,7 +58,51 @@ public class TestParquetPhysicalPlan {
       bit1.run();
       client.connect();
       List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8));
-      System.out.println(String.format("Got %d results", results.size()));
+      RecordBatchLoader loader = new RecordBatchLoader(bit1.getContext().getAllocator());
+      for (QueryResultBatch b : results) {
+        System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
+        loader.load(b.getHeader().getDef(), b.getData());
+        for (VectorWrapper vw : loader) {
+          System.out.println(vw.getValueVector().getField().getName());
+          ValueVector vv = vw.getValueVector();
+          for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
+            Object o = vv.getAccessor().getObject(i);
+            System.out.println(vv.getAccessor().getObject(i));
+          }
+        }
+      }
+      client.close();
+    }
+  }
+
+  private class ParquetResultsListener implements UserResultsListener {
+    private CountDownLatch latch = new CountDownLatch(1);
+    @Override
+    public void submissionFailed(RpcException ex) {
+      logger.error("submission failed", ex);
+      latch.countDown();
+    }
+
+    @Override
+    public void resultArrived(QueryResultBatch result) {
+      System.out.printf("Result batch arrived. Number of records: %d", result.getHeader().getRowCount());
+      if (result.getHeader().getIsLastChunk()) latch.countDown();
+    }
+
+    public void await() throws Exception {
+      latch.await();
+    }
+  }
+  @Test
+  @Ignore
+  public void testParseParquetPhysicalPlanRemote() throws Exception {
+    DrillConfig config = DrillConfig.create();
+
+    try(DrillClient client = new DrillClient(config);){
+      client.connect();
+      ParquetResultsListener listener = new ParquetResultsListener();
+      client.runQuery(UserProtos.QueryType.PHYSICAL, Resources.toString(Resources.getResource(fileName),Charsets.UTF_8), listener);
+      listener.await();
       client.close();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 1d91455..7a99c3f 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -48,6 +48,7 @@ import org.apache.drill.exec.store.parquet.TestFileGenerator.FieldInfo;
 import org.apache.drill.exec.vector.BaseDataValueVector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import parquet.bytes.BytesInput;
@@ -68,6 +69,7 @@ public class ParquetRecordReaderTest {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReaderTest.class);
 
   private boolean VERBOSE_DEBUG = false;
+  private boolean checkValues = true;
 
   static final int numberRowGroups = 20;
   static final int recordsPerRowGroup = 300000;
@@ -100,6 +102,20 @@ public class ParquetRecordReaderTest {
     testParquetFullEngineLocalText(planText, fileName, i, numberRowGroups, recordsPerRowGroup);
   }
 
+  @Test
+  @Ignore
+  public void testLocalDistributed() throws Exception {
+    String planName = "/parquet/parquet_scan_union_screen_physical.json";
+    testParquetFullEngineLocalTextDistributed(planName, fileName, 1, 20, 300000);
+  }
+
+  @Test
+  @Ignore
+  public void testRemoteDistributed() throws Exception {
+    String planName = "/parquet/parquet_scan_union_screen_physical.json";
+    testParquetFullEngineRemote(planName, fileName, 1, 10, 30000);
+  }
+
 
   private class ParquetResultListener implements UserResultsListener {
     private SettableFuture<Void> future = SettableFuture.create();
@@ -155,8 +171,12 @@ public class ParquetRecordReaderTest {
           if (VERBOSE_DEBUG){
             System.out.print(vv.getAccessor().getObject(j) + ", " + (j % 25 == 0 ? "\n batch:" + batchCounter + " v:" + j + " - " : ""));
           }
-          assertField(vv, j, (TypeProtos.MinorType) currentField.type,
-              currentField.values[(int) (columnValCounter % 3)], (String) currentField.name + "/");
+          if (checkValues) {
+            try {
+              assertField(vv, j, (TypeProtos.MinorType) currentField.type,
+                currentField.values[(int) (columnValCounter % 3)], (String) currentField.name + "/");
+            } catch (AssertionError e) { submissionFailed(new RpcException(e)); }
+          }
           columnValCounter++;
         }
         if (VERBOSE_DEBUG){
@@ -197,7 +217,9 @@ public class ParquetRecordReaderTest {
       batchCounter++;
       if(result.getHeader().getIsLastChunk()){
         for (String s : valuesChecked.keySet()) {
+          try {
           assertEquals("Record count incorrect for column: " + s, totalRecords, (long) valuesChecked.get(s));
+          } catch (AssertionError e) { submissionFailed(new RpcException(e)); }
         }
         
         assert valuesChecked.keySet().size() > 0;
@@ -222,11 +244,13 @@ public class ParquetRecordReaderTest {
     
     DrillConfig config = DrillConfig.create();
 
+    checkValues = false;
+
     try(DrillClient client = new DrillClient(config);){
       client.connect();
       RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
       ParquetResultListener resultListener = new ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups, numberOfTimesRead);
-      client.runQuery(UserProtos.QueryType.LOGICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
       resultListener.get();
     }
     
@@ -259,6 +283,28 @@ public class ParquetRecordReaderTest {
   }
 
 
+  //use this method to submit physical plan
+  public void testParquetFullEngineLocalTextDistributed(String planName, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
+
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    checkValues = false;
+
+    DrillConfig config = DrillConfig.create();
+
+    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
+      bit1.run();
+      client.connect();
+      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
+      ParquetResultListener resultListener = new ParquetResultListener(recordsPerRowGroup, batchLoader, numberOfRowGroups, numberOfTimesRead);
+      Stopwatch watch = new Stopwatch().start();
+      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(planName), Charsets.UTF_8), resultListener);
+      resultListener.get();
+      System.out.println(String.format("Took %d ms to run query", watch.elapsed(TimeUnit.MILLISECONDS)));
+
+    }
+
+  }
 
   public String pad(String value, int length) {
     return pad(value, length, " ");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/7edd3617/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
index f508d09..5efecaf 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/parquet/parquet_scan_union_screen_physical.json
@@ -11,10 +11,7 @@
     @id : 1,
     entries : [
     {
-        path : "/tmp/testParquetFile_many_types_3"
-    },
-    {
-        path : "/tmp/testParquetFile_many_types_3"
+        path : "/tmp/parquet_test_file_many_types"
     }
     ],
     storageengine:{


[6/8] git commit: DRILL-180: propagate exceptions back to client. fail all whenever any query without queryId is received. fix RunningFragmentManager and Foreman to include queryId when building results @Ignore ConstantRopTest.testRefInterp() until plan

Posted by ja...@apache.org.
DRILL-180: propagate exceptions back to client.
fail all whenever any query without queryId is received.
fix RunningFragmentManager and Foreman to include queryId when building results
@Ignore ConstantRopTest.testRefInterp() until plan is updated.


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

Branch: refs/heads/master
Commit: 0fc89a317251e684c797ad0e5e5c68c3842ab3b3
Parents: 7edd361
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Aug 22 16:53:19 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:53:19 2013 -0700

----------------------------------------------------------------------
 .../drill/exec/rpc/RpcExceptionHandler.java     |  2 +-
 .../drill/exec/rpc/user/QueryResultHandler.java | 10 ++++++++
 .../exec/work/batch/BitComHandlerImpl.java      |  9 +++++++
 .../apache/drill/exec/work/foreman/Foreman.java |  1 +
 .../work/foreman/RunningFragmentManager.java    |  5 +++-
 .../store/parquet/ParquetRecordReaderTest.java  |  5 ++--
 .../drill/exec/ref/rops/ConstantROPTest.java    | 26 +++++++++-----------
 .../exec/ref/src/test/resources/constant2.json  |  2 +-
 8 files changed, 40 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
index a0aed94..0123cad 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
@@ -30,7 +30,7 @@ public class RpcExceptionHandler implements ChannelHandler{
   @Override
   public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
     if(!ctx.channel().isOpen()) return;
-    logger.info("Exception in pipeline.  Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
+    logger.error("Exception in pipeline.  Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
     ctx.close();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
index b2283a2..50f8c5a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultHandler.java
@@ -63,6 +63,9 @@ public class QueryResultHandler {
       l = resultsListener.putIfAbsent(result.getQueryId(), bl);
       // if we had a succesful insert, use that reference.  Otherwise, just throw away the new bufering listener.
       if (l == null) l = bl;
+      if (result.getQueryId().toString().equals("")) {
+        failAll();
+      }
     }
       
     if(failed){
@@ -80,6 +83,13 @@ public class QueryResultHandler {
       resultsListener.remove(result.getQueryId(), l);
     }
 
+
+  }
+
+  private void failAll() {
+    for (UserResultsListener l : resultsListener.values()) {
+      l.submissionFailed(new RpcException("Received result without QueryId"));
+    }
   }
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
index 5807c87..8cba493 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
@@ -124,6 +124,15 @@ public class BitComHandlerImpl implements BitComHandler {
       listener.fail(fragment.getHandle(), "Failure while parsing fragment execution plan.", e);
     }catch(ExecutionSetupException e){
       listener.fail(fragment.getHandle(), "Failure while setting up execution plan.", e);
+    } catch (Exception e) {
+      listener.fail(fragment.getHandle(), "Failure due to uncaught exception", e);
+    } catch (OutOfMemoryError t) {
+      if(t.getMessage().startsWith("Direct buffer")){
+        listener.fail(fragment.getHandle(), "Failure due to error", t);  
+      }else{
+        throw t;
+      }
+      
     }
     
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index c1fd9e5..bd64938 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -106,6 +106,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
         .addError(error) //
         .setIsLastChunk(true) //
         .setQueryState(QueryState.FAILED) //
+        .setQueryId(queryId) //
         .build();
     cleanupAndSendResult(result);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
index da2f7c1..9d9aca6 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
@@ -147,7 +147,10 @@ class RunningFragmentManager implements FragmentStatusListener{
     updateStatus(status);
     int remaining = remainingFragmentCount.decrementAndGet();
     if(remaining == 0){
-      QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.COMPLETED).build();
+      QueryResult result = QueryResult.newBuilder() //
+              .setQueryState(QueryState.COMPLETED) //
+              .setQueryId(queryId) //
+              .build();
       foreman.cleanupAndSendResult(result);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index 7a99c3f..fc5bc81 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -103,17 +103,16 @@ public class ParquetRecordReaderTest {
   }
 
   @Test
-  @Ignore
   public void testLocalDistributed() throws Exception {
     String planName = "/parquet/parquet_scan_union_screen_physical.json";
-    testParquetFullEngineLocalTextDistributed(planName, fileName, 1, 20, 300000);
+    testParquetFullEngineLocalTextDistributed(planName, fileName, 1, numberRowGroups, recordsPerRowGroup);
   }
 
   @Test
   @Ignore
   public void testRemoteDistributed() throws Exception {
     String planName = "/parquet/parquet_scan_union_screen_physical.json";
-    testParquetFullEngineRemote(planName, fileName, 1, 10, 30000);
+    testParquetFullEngineRemote(planName, fileName, 1, numberRowGroups, recordsPerRowGroup);
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
index 9aea930..353e66d 100644
--- a/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
+++ b/sandbox/prototype/exec/ref/src/test/java/org/apache/drill/exec/ref/rops/ConstantROPTest.java
@@ -1,24 +1,25 @@
 package org.apache.drill.exec.ref.rops;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
+import java.util.Collection;
+
 import org.apache.drill.common.config.DrillConfig;
 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.data.Constant;
 import org.apache.drill.common.util.FileUtils;
-import org.apache.drill.exec.ref.*;
+import org.apache.drill.exec.ref.IteratorRegistry;
+import org.apache.drill.exec.ref.RecordIterator;
+import org.apache.drill.exec.ref.RecordPointer;
+import org.apache.drill.exec.ref.ReferenceInterpreter;
+import org.apache.drill.exec.ref.RunOutcome;
 import org.apache.drill.exec.ref.eval.BasicEvaluatorFactory;
 import org.apache.drill.exec.ref.rse.RSERegistry;
 import org.apache.drill.exec.ref.values.ScalarValues;
+import org.junit.Ignore;
 import org.junit.Test;
 
-
-import java.io.File;
-import java.util.Collection;
-import java.util.Iterator;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 /**
  * Created with IntelliJ IDEA.
@@ -64,9 +65,8 @@ public class ConstantROPTest {
     // not sure if we want to keep this as a test and check the results. Now that the internals of the ConstantROP work
     // it might now be worth running the reference intepreter with every build
     @Test
-    public void testRefInterp(){
-
-        try{
+    @Ignore // this plan needs to be updated.
+    public void testRefInterp() throws Exception{
             DrillConfig config = DrillConfig.create();
             final String jsonFile = "/constant2.json";
             LogicalPlan plan = LogicalPlan.parse(config, FileUtils.getResourceAsString(jsonFile));
@@ -89,8 +89,6 @@ public class ConstantROPTest {
                     outcome.exception.printStackTrace();
                 }
             }
-        } catch (Exception e) {
-            e.printStackTrace();  //To change body of catch statement use File | Settings | File Templates.
-        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0fc89a31/sandbox/prototype/exec/ref/src/test/resources/constant2.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/constant2.json b/sandbox/prototype/exec/ref/src/test/resources/constant2.json
index 31fed5d..bad1aa3 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/constant2.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/constant2.json
@@ -1,6 +1,6 @@
 {
   head:{
-    type:"apache_drill_logical_plan",
+    type:"APACHE_DRILL_LOGICAL",
     version:"1",
     generator:{
       type:"manual",


[4/8] git commit: DRILL-171: Fix graph creation

Posted by ja...@apache.org.
DRILL-171: Fix graph creation


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

Branch: refs/heads/master
Commit: d3b68d13886b1ffc527789a7538e0c3091d2345a
Parents: c568eed
Author: immars <im...@gmail.com>
Authored: Thu Aug 22 16:00:19 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:08:39 2013 -0700

----------------------------------------------------------------------
 .../apache/drill/common/logical/data/LogicalOperatorBase.java | 7 +------
 1 file changed, 1 insertion(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3b68d13/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java
index 23d3c6e..970be76 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/logical/data/LogicalOperatorBase.java
@@ -49,12 +49,7 @@ public abstract class LogicalOperatorBase implements LogicalOperator{
 
   @Override
   public void accept(GraphVisitor<LogicalOperator> visitor) {
-    if(visitor.enter(this)){
-        for(LogicalOperator o : children){
-            o.accept(visitor);
-        }
-    }
-    visitor.leave(this);
+    if(visitor.enter(this)) visitor.leave(this);
   }
 
     @Override


[2/8] git commit: DRILL-161 Add rebuffed jars for Hadoop2 protobuf conflicts

Posted by ja...@apache.org.
DRILL-161 Add rebuffed jars for Hadoop2 protobuf conflicts


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

Branch: refs/heads/master
Commit: d6ae53ee04f4902a2d13e144b3582a49b6afd586
Parents: d3b68d1
Author: immars <im...@gmail.com>
Authored: Thu Aug 22 16:07:12 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Aug 22 16:08:39 2013 -0700

----------------------------------------------------------------------
 sandbox/prototype/common/pom.xml         | 32 ++++++++++++++++++++++
 sandbox/prototype/exec/java-exec/pom.xml | 38 ++++++++++++++++++++++++++-
 2 files changed, 69 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d6ae53ee/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index a33ced9..a8ad008 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -120,6 +120,38 @@
           </execution>
         </executions>
       </plugin>
+	 
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.1</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <artifactSet>
+                 <includes>
+                    <include>com.google.protobuf:protobuf-java</include>                     
+                 </includes>                 
+              </artifactSet>
+              <shadedArtifactAttached>true</shadedArtifactAttached>
+              <shadedClassifierName>rebuffed</shadedClassifierName>
+             
+              <relocations>
+                <relocation>
+                  <pattern>com.google.protobuf</pattern>
+                  <shadedPattern>250.com.google.protobuf</shadedPattern>
+                </relocation>
+              </relocations>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+
     </plugins>
 
   </build>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d6ae53ee/sandbox/prototype/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/pom.xml b/sandbox/prototype/exec/java-exec/pom.xml
index b36208c..a1680e8 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -274,7 +274,43 @@
           </execution>
         </executions>
       </plugin>
-
+        <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>2.1</version>
+                <executions>
+                  <execution>
+                    <phase>package</phase>
+                    <goals>
+                      <goal>shade</goal>
+                    </goals>
+                    <configuration>
+                      <artifactSet>
+                         <includes>
+                            <include>com.google.protobuf:protobuf-java</include>                     
+                         </includes>                 
+                      </artifactSet>
+                      <shadedArtifactAttached>true</shadedArtifactAttached>
+                      <shadedClassifierName>rebuffed</shadedClassifierName>
+                      <filters>
+                        <filter>
+                            <artifact>com.google.protobuf:protobuf-java</artifact>
+                            <excludes>
+                                <exclude>**</exclude>
+                            </excludes>
+                        </filter>
+                    </filters>
+                      <relocations>
+                        <relocation>
+                          <pattern>com.google.protobuf</pattern>
+                          <shadedPattern>250.com.google.protobuf</shadedPattern>
+                        </relocation>
+                      </relocations>
+                    </configuration>
+                  </execution>
+                </executions>
+              </plugin>
+        
       <!-- <plugin> -->
       <!-- <groupId>com.github.igor-petruk.protobuf</groupId> -->
       <!-- <artifactId>protobuf-maven-plugin</artifactId> -->


[7/8] DRILL-166 Update CodeGenerator code to support more generic signatures.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
index e91b904..0c9b979 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionHolder.java
@@ -1,12 +1,10 @@
 package org.apache.drill.exec.expr.fn;
 
-import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
-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.expr.CodeGenerator;
@@ -16,29 +14,25 @@ import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
 import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.sun.codemodel.JBlock;
-import com.sun.codemodel.JConditional;
-import com.sun.codemodel.JExpr;
-import com.sun.codemodel.JExpression;
 import com.sun.codemodel.JMod;
 import com.sun.codemodel.JVar;
 
-public class FunctionHolder {
+public abstract class FunctionHolder {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FunctionImplementationRegistry.class);
   
-  private FunctionTemplate.FunctionScope scope;
-  private FunctionTemplate.NullHandling nullHandling;
-  private boolean isBinaryCommutative;
-  private String functionName;
-  private String evalBody;
-  private String addBody;
-  private String setupBody;
-  private List<String> imports;
-  private WorkspaceReference[] workspaceVars;
-  private ValueReference[] parameters;
-  private ValueReference returnValue;
+  protected final FunctionTemplate.FunctionScope scope;
+  protected final FunctionTemplate.NullHandling nullHandling;
+  protected final boolean isBinaryCommutative;
+  protected final String functionName;
+  protected final ImmutableList<String> imports;
+  protected final WorkspaceReference[] workspaceVars;
+  protected final ValueReference[] parameters;
+  protected final ValueReference returnValue;
+  protected final ImmutableMap<String, String> methodMap; 
   
   public FunctionHolder(FunctionScope scope, NullHandling nullHandling, boolean isBinaryCommutative, String functionName, ValueReference[] parameters, ValueReference returnValue, WorkspaceReference[] workspaceVars, Map<String, String> methods, List<String> imports) {
     super();
@@ -47,14 +41,10 @@ public class FunctionHolder {
     this.workspaceVars = workspaceVars;
     this.isBinaryCommutative = isBinaryCommutative;
     this.functionName = functionName;
-    this.setupBody = methods.get("setup");
-    this.addBody = methods.get("add");
-    this.evalBody = methods.get("eval");
-    Preconditions.checkNotNull(evalBody);
-    Preconditions.checkArgument(!evalBody.isEmpty());
+    this.methodMap = ImmutableMap.copyOf(methods);
     this.parameters = parameters;
     this.returnValue = returnValue;
-    this.imports = imports;
+    this.imports = ImmutableList.copyOf(imports);
     
   }
   
@@ -62,15 +52,7 @@ public class FunctionHolder {
     return imports;
   }
 
-  private void generateSetupBody(CodeGenerator<?> g){
-    if(!Strings.isNullOrEmpty(setupBody)){
-      JBlock sub = new JBlock(true, true);
-      addProtectedBlock(g, sub, setupBody, null);
-      g.getSetupBlock().directStatement(String.format("/** start setup for function %s **/", functionName));
-      g.getSetupBlock().add(sub);
-      g.getSetupBlock().directStatement(String.format("/** end setup for function %s **/", functionName));
-    }
-  }
+  public abstract HoldingContainer renderFunction(CodeGenerator<?> g, HoldingContainer[] inputVariables);
   
   public void addProtectedBlock(CodeGenerator<?> g, JBlock sub, String body, HoldingContainer[] inputVariables){
     
@@ -102,57 +84,7 @@ public class FunctionHolder {
     }
   }
 
-  public HoldingContainer renderFunction(CodeGenerator<?> g, HoldingContainer[] inputVariables){
-    generateSetupBody(g);
-    return generateEvalBody(g, inputVariables);
-  }
-  
-  private HoldingContainer generateEvalBody(CodeGenerator<?> g, HoldingContainer[] inputVariables){
-    
-    //g.getBlock().directStatement(String.format("//---- start of eval portion of %s function. ----//", functionName));
-    
-    JBlock sub = new JBlock(true, true);
-    JBlock topSub = sub;
-    HoldingContainer out = null;
-
-    // add outside null handling if it is defined.
-    if(nullHandling == NullHandling.NULL_IF_NULL){
-      JExpression e = null;
-      for(HoldingContainer v : inputVariables){
-        if(v.isOptional()){
-          if(e == null){
-            e = v.getIsSet();
-          }else{
-            e = e.mul(v.getIsSet());
-          }
-        }
-      }
-      
-      if(e != null){
-        // if at least one expression must be checked, set up the conditional.
-        returnValue.type = returnValue.type.toBuilder().setMode(DataMode.OPTIONAL).build();
-        out = g.declare(returnValue.type);
-        e = e.eq(JExpr.lit(0));
-        JConditional jc = sub._if(e);
-        jc._then().assign(out.getIsSet(), JExpr.lit(0));
-        sub = jc._else();
-      }
-    }
-    
-    if(out == null) out = g.declare(returnValue.type);
-    
-    // add the subblock after the out declaration.
-    g.getBlock().add(topSub);
-    
-    
-    JVar internalOutput = sub.decl(JMod.FINAL, g.getHolderType(returnValue.type), returnValue.name, JExpr._new(g.getHolderType(returnValue.type)));
-    addProtectedBlock(g, sub, evalBody, inputVariables);
-    if (sub != topSub) sub.assign(internalOutput.ref("isSet"),JExpr.lit(1));// Assign null if NULL_IF_NULL mode
-    sub.assign(out.getHolder(), internalOutput);
-
-    return out;
-  }
-  
+ 
   
   
   public boolean matches(FunctionCall call){
@@ -215,15 +147,7 @@ public class FunctionHolder {
     }
     
   }
-  @Override
-  public String toString() {
-    final int maxLen = 10;
-    return "FunctionHolder [scope=" + scope + ", isBinaryCommutative=" + isBinaryCommutative + ", functionName="
-        + functionName + ", evalBody=" + evalBody + ", addBody=" + addBody + ", setupBody=" + setupBody
-        + ", parameters="
-        + (parameters != null ? Arrays.asList(parameters).subList(0, Math.min(parameters.length, maxLen)) : null)
-        + ", returnValue=" + returnValue + "]";
-  }
+
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index c2dfca4..bbe4cfb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -107,14 +107,6 @@ public class FragmentContext {
     return t;
     
   }
-
-  public <T> T getImplementationClassMultipleOutput(CodeGenerator<T> cg) throws ClassTransformationException, IOException{
-    long t1 = System.nanoTime();
-    T t = transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generateMultipleOutputs(), cg.getMaterializedClassName());
-    logger.debug("Compile time: {} millis.", (System.nanoTime() - t1)/1000/1000 );
-    return t;
-
-  }
   
   public void addMetricsToStatus(FragmentStatus.Builder stats){
     stats.setBatchesCompleted(batchesCompleted.get());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterSignature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterSignature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterSignature.java
new file mode 100644
index 0000000..f085cd3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/FilterSignature.java
@@ -0,0 +1,14 @@
+package org.apache.drill.exec.physical.impl.filter;
+
+import javax.inject.Named;
+
+import org.apache.drill.exec.compile.sig.CodeGeneratorSignature;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface FilterSignature  extends CodeGeneratorSignature{
+  
+  public void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing);
+  public boolean doEval(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
index b270869..fb08ef3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/Filterer.java
@@ -3,7 +3,6 @@ package org.apache.drill.exec.physical.impl.filter;
 import org.apache.drill.exec.compile.TemplateClassDefinition;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.impl.project.Projector;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.TransferPair;
 
@@ -14,6 +13,6 @@ public interface Filterer {
   public void filterBatch(int recordCount);
   
   public static TemplateClassDefinition<Filterer> TEMPLATE_DEFINITION = new TemplateClassDefinition<Filterer>( //
-      Filterer.class, "org.apache.drill.exec.physical.impl.filter.FilterTemplate", FilterEvaluator.class, boolean.class);
+      Filterer.class, "org.apache.drill.exec.physical.impl.filter.FilterTemplate", FilterEvaluator.class, FilterSignature.class);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
index a794d63..cfe520d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/filter/ReturnValueExpression.java
@@ -1,11 +1,15 @@
 package org.apache.drill.exec.physical.impl.filter;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.Types;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 
+import com.google.common.collect.Iterators;
+
 public class ReturnValueExpression implements LogicalExpression{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ReturnValueExpression.class);
 
@@ -34,6 +38,10 @@ public class ReturnValueExpression implements LogicalExpression{
     return ExpressionPosition.UNKNOWN;
   }
   
-  
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.singletonIterator(child);
+  }
+
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 6d24e0b..f96e3cf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -171,7 +171,7 @@ class PartitionSenderRootExec implements RootExec {
 
     // generate evaluate expression to determine the hash
     CodeGenerator.HoldingContainer exprHolder = cg.addExpr(materializedExpr);
-    cg.getBlock().decl(JType.parse(cg.getModel(), "int"), "bucket", exprHolder.getValue().mod(JExpr.lit(outgoing.length)));
+    cg.getEvalBlock().decl(JType.parse(cg.getModel(), "int"), "bucket", exprHolder.getValue().mod(JExpr.lit(outgoing.length)));
 
     // declare and assign the array of outgoing record batches
     JVar outgoingBatches = cg.clazz.field(JMod.NONE,
@@ -230,7 +230,7 @@ class PartitionSenderRootExec implements RootExec {
       // ((IntVector) outgoingVectors[bucket][0]).copyFrom(inIndex,
       //                                                     outgoingBatches[bucket].getRecordCount(),
       //                                                     vv1);
-      cg.getBlock().add(
+      cg.getEvalBlock().add(
         ((JExpression) JExpr.cast(vvClass,
               ((JExpression)
                      outgoingVectors
@@ -244,11 +244,12 @@ class PartitionSenderRootExec implements RootExec {
       ++fieldId;
     }
     // generate the OutgoingRecordBatch helper invocations
-    cg.getBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("incRecordCount"));
-    cg.getBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("flushIfNecessary"));
+    cg.getEvalBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("incRecordCount"));
+    cg.getEvalBlock().add(((JExpression) outgoingBatches.component(bucket)).invoke("flushIfNecessary"));
     try {
       // compile and setup generated code
-      partitioner = context.getImplementationClassMultipleOutput(cg);
+//      partitioner = context.getImplementationClassMultipleOutput(cg);
+      partitioner = context.getImplementationClass(cg);
       partitioner.setup(context, incoming, outgoing);
 
     } catch (ClassTransformationException | IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
index e8f2ca7..992ffdf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/Partitioner.java
@@ -35,5 +35,5 @@ public interface Partitioner {
       new TemplateClassDefinition<>(Partitioner.class,
                                     "org.apache.drill.exec.physical.impl.partitionsender.PartitionerTemplate",
                                     PartitionerEvaluator.class,
-                                    null);
+                                    PartitionerInnerSignature.class);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerInnerSignature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerInnerSignature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerInnerSignature.java
new file mode 100644
index 0000000..be209a9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionerInnerSignature.java
@@ -0,0 +1,17 @@
+package org.apache.drill.exec.physical.impl.partitionsender;
+
+import javax.inject.Named;
+
+import org.apache.drill.exec.compile.sig.CodeGeneratorSignature;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface PartitionerInnerSignature  extends CodeGeneratorSignature{
+  
+  public void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") OutgoingRecordBatch[] outgoing) throws SchemaChangeException;
+  public void doEval(@Named("inIndex") int inIndex, @Named("outIndex") int outIndex);
+  
+  
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java
index 5fd1fb4..75632e7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectEvaluator.java
@@ -8,5 +8,5 @@ public interface ProjectEvaluator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectEvaluator.class);
   
   public abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
-  public abstract void doEval(int inIndex, int outIndex);
+  public abstract void doEval(int inIndex, int outIndex) throws SchemaChangeException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
index 0d1e201..ba83e61 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/Projector.java
@@ -3,6 +3,7 @@ package org.apache.drill.exec.physical.impl.project;
 import java.util.List;
 
 import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.compile.sig.DefaultGeneratorSignature;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.RecordBatch;
@@ -16,6 +17,6 @@ public interface Projector {
   public abstract int projectRecords(int recordCount, int firstOutputIndex);
 
   public static TemplateClassDefinition<Projector> TEMPLATE_DEFINITION = new TemplateClassDefinition<Projector>( //
-      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectorTemplate", ProjectEvaluator.class, null);
+      Projector.class, "org.apache.drill.exec.physical.impl.project.ProjectorTemplate", ProjectEvaluator.class);
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/ReadIndexRewriter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/ReadIndexRewriter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/ReadIndexRewriter.java
index 83d43b2..02fffa5 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/ReadIndexRewriter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/ReadIndexRewriter.java
@@ -45,7 +45,7 @@ public class ReadIndexRewriter implements ExprVisitor<LogicalExpression, String,
 
   @Override
   public LogicalExpression visitIfExpression(IfExpression ifExpr, String newIndexName) {
-    List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.iterator());
+    List<IfExpression.IfCondition> conditions = Lists.newArrayList(ifExpr.conditions);
     LogicalExpression newElseExpr = ifExpr.elseExpression.accept(this, null);
 
     for (int i = 0; i < conditions.size(); ++i) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
index e361e38..c9bd55d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
@@ -14,6 +14,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.CodeGenerator.HoldingContainer;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
+import org.apache.drill.exec.expr.HoldingContainerExpression;
 import org.apache.drill.exec.expr.fn.impl.ComparatorFunctions;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.Sort;
@@ -123,22 +124,26 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
   }
 
   
+  
   private Sorter createNewSorter() throws ClassTransformationException, IOException, SchemaChangeException{
     CodeGenerator<Sorter> g = new CodeGenerator<Sorter>(Sorter.TEMPLATE_DEFINITION, context.getFunctionRegistry());
+    g.setMappingSet(SortSignature.MAIN_MAPPING);
     
     for(OrderDef od : popConfig.getOrderings()){
       // first, we rewrite the evaluation stack for each side of the comparison.
       ErrorCollector collector = new ErrorCollectorImpl(); 
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), this, collector);
       if(collector.hasErrors()) throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
-      ReadIndexRewriter rewriter = new ReadIndexRewriter();
-      LogicalExpression left = expr.accept(rewriter, "inIndex");
-      LogicalExpression right = expr.accept(rewriter, "outIndex");
+      g.setMappingSet(SortSignature.LEFT_MAPPING);
+      HoldingContainer left = g.addExpr(expr, false);
+      g.setMappingSet(SortSignature.RIGHT_MAPPING);
+      HoldingContainer right = g.addExpr(expr, false);
+      g.setMappingSet(SortSignature.MAIN_MAPPING);
       
       // next we wrap the two comparison sides and add the expression block for the comparison.
-      FunctionCall f = new FunctionCall(ComparatorFunctions.COMPARE_TO, ImmutableList.of(left, right), ExpressionPosition.UNKNOWN);
-      HoldingContainer out = g.addExpr(f);
-      JConditional jc = g.getBlock()._if(out.getValue().ne(JExpr.lit(0)));
+      FunctionCall f = new FunctionCall(ComparatorFunctions.COMPARE_TO, ImmutableList.of((LogicalExpression) new HoldingContainerExpression(left), new HoldingContainerExpression(right)), ExpressionPosition.UNKNOWN);
+      HoldingContainer out = g.addExpr(f, false);
+      JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
       
       //TODO: is this the right order...
       if(od.getDirection() == Direction.ASC){
@@ -148,7 +153,7 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
       }
     }
     
-    g.getBlock()._return(JExpr.lit(0));
+    g.getEvalBlock()._return(JExpr.lit(0));
     
     return context.getImplementationClass(g);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortSignature.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortSignature.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortSignature.java
new file mode 100644
index 0000000..7614f3e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortSignature.java
@@ -0,0 +1,20 @@
+package org.apache.drill.exec.physical.impl.sort;
+
+import javax.inject.Named;
+
+import org.apache.drill.exec.compile.sig.CodeGeneratorSignature;
+import org.apache.drill.exec.compile.sig.DefaultGeneratorSignature;
+import org.apache.drill.exec.compile.sig.MappingSet;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface SortSignature extends CodeGeneratorSignature{
+  
+  public static final MappingSet MAIN_MAPPING = new MappingSet("null", "null", DefaultGeneratorSignature.DEFAULT_SCALAR_MAP, DefaultGeneratorSignature.DEFAULT_SCALAR_MAP);
+  public static final MappingSet LEFT_MAPPING = new MappingSet("leftIndex", "null", DefaultGeneratorSignature.DEFAULT_SCALAR_MAP, DefaultGeneratorSignature.DEFAULT_SCALAR_MAP);
+  public static final MappingSet RIGHT_MAPPING = new MappingSet("rightIndex", "null", DefaultGeneratorSignature.DEFAULT_SCALAR_MAP, DefaultGeneratorSignature.DEFAULT_SCALAR_MAP);
+
+  public void doSetup(@Named("context") FragmentContext context, @Named("incoming") RecordBatch incoming, @Named("outgoing") RecordBatch outgoing);
+  public int doEval(@Named("leftIndex") int leftIndex, @Named("rightIndex") int rightIndex);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortTemplate.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortTemplate.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortTemplate.java
index c45f500..d312fb4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortTemplate.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortTemplate.java
@@ -34,12 +34,12 @@ public abstract class SortTemplate implements Sorter, IndexedSortable{
   }
   
   @Override
-  public int compare(int inIndex, int outIndex) {
-    int sv1 = vector4.get(inIndex);
-    int sv2 = vector4.get(outIndex);
+  public int compare(int leftIndex, int rightIndex) {
+    int sv1 = vector4.get(leftIndex);
+    int sv2 = vector4.get(rightIndex);
     return doEval(sv1, sv2);
   }
 
   public abstract void doSetup(FragmentContext context, RecordBatch incoming, RecordBatch outgoing) throws SchemaChangeException;
-  public abstract int doEval(int inIndex, int outIndex);
+  public abstract int doEval(int leftIndex, int rightIndex);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/Sorter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/Sorter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/Sorter.java
index bc4fae5..1a76423 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/Sorter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/Sorter.java
@@ -12,6 +12,6 @@ public interface Sorter {
   public void sort(SelectionVector4 vector4, VectorContainer container);
   
   public static TemplateClassDefinition<Sorter> TEMPLATE_DEFINITION = new TemplateClassDefinition<Sorter>( //
-      Sorter.class, "org.apache.drill.exec.physical.impl.sort.SortTemplate", Comparator.class, int.class);
+      Sorter.class, "org.apache.drill.exec.physical.impl.sort.SortTemplate", Comparator.class, SortSignature.class);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
index ce17a2b..363bbee 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/Copier.java
@@ -1,18 +1,18 @@
 package org.apache.drill.exec.physical.impl.svremover;
 
 import org.apache.drill.exec.compile.TemplateClassDefinition;
+import org.apache.drill.exec.compile.sig.DefaultGeneratorSignature;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.svremover.RemovingRecordBatch.VectorAllocator;
 import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.selection.SelectionVector2;
 
 public interface Copier {
   public static TemplateClassDefinition<Copier> TEMPLATE_DEFINITION2 = new TemplateClassDefinition<Copier>( //
-      Copier.class, "org.apache.drill.exec.physical.impl.svremover.CopierTemplate2", CopyEvaluator.class, null);
+      Copier.class, "org.apache.drill.exec.physical.impl.svremover.CopierTemplate2", CopyEvaluator.class);
 
   public static TemplateClassDefinition<Copier> TEMPLATE_DEFINITION4 = new TemplateClassDefinition<Copier>( //
-      Copier.class, "org.apache.drill.exec.physical.impl.svremover.CopierTemplate4", CopyEvaluator.class, null);
+      Copier.class, "org.apache.drill.exec.physical.impl.svremover.CopierTemplate4", CopyEvaluator.class);
 
   public void setupRemover(FragmentContext context, RecordBatch incoming, RecordBatch outgoing, VectorAllocator[] allocators) throws SchemaChangeException;
   public abstract void copyRecords();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
index 64e89ee..e4fd9a0 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/svremover/RemovingRecordBatch.java
@@ -167,7 +167,7 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch<SelectionVect
 
       if(hyper){
         
-        g.getBlock().add( 
+        g.getEvalBlock().add( 
             outVV
             .invoke("copyFrom")
             .arg(
@@ -178,7 +178,7 @@ public class RemovingRecordBatch extends AbstractSingleRecordBatch<SelectionVect
                 )
             );  
       }else{
-        g.getBlock().add(outVV.invoke("copyFrom").arg(inIndex).arg(outIndex).arg(inVV));
+        g.getEvalBlock().add(outVV.invoke("copyFrom").arg(inIndex).arg(outIndex).arg(inVV));
       }
       
       

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
index c2f2a69..821a4b4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/NullExpression.java
@@ -1,12 +1,15 @@
 package org.apache.drill.exec.record;
 
+import java.util.Iterator;
+
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
-import org.apache.drill.common.types.Types;
-import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+
+import com.google.common.collect.Iterators;
 
 public class NullExpression implements LogicalExpression{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullExpression.class);
@@ -29,5 +32,11 @@ public class NullExpression implements LogicalExpression{
   public ExpressionPosition getPosition() {
     return ExpressionPosition.UNKNOWN;
   }
+
+  @Override
+  public Iterator<LogicalExpression> iterator() {
+    return Iterators.emptyIterator();
+  }
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
index d2889ed..16c993d 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassTransformation.java
@@ -40,7 +40,7 @@ public class TestClassTransformation {
 
     TemplateClassDefinition<ExampleExternalInterface> def = new TemplateClassDefinition<ExampleExternalInterface>(
         ExampleExternalInterface.class, "org.apache.drill.exec.compile.ExampleTemplate",
-        ExampleInternalInterface.class, null);
+        ExampleInternalInterface.class);
     
     
     ClassTransformer ct = new ClassTransformer();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
index 4129079..a25e234 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
@@ -18,8 +18,10 @@
 
 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.pop.PopUnitTestBase;
@@ -27,11 +29,12 @@ import org.apache.drill.exec.proto.UserProtos;
 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.nio.charset.Charset;
-import java.util.List;
-import static org.junit.Assert.assertEquals;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
 
 public class TestHashToRandomExchange extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHashToRandomExchange.class);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ac8590d2/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
index 8a1736c..f0d9901 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
@@ -30,7 +30,7 @@ import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Range;
 
 public class ExpressionTreeMaterializerTest {
@@ -42,16 +42,16 @@ public class ExpressionTreeMaterializerTest {
   final MajorType intType = MajorType.newBuilder().setMode(DataMode.REQUIRED).setMinorType(MinorType.INT).build();
 
   private MaterializedField getField(int fieldId, String name, MajorType type) {
-    return new MaterializedField(FieldDef.newBuilder().setMajorType(type)
-        .addName(NamePart.newBuilder().setName(name)).build());
+    return new MaterializedField(FieldDef.newBuilder().setMajorType(type).addName(NamePart.newBuilder().setName(name))
+        .build());
   }
 
-
   @Test
   public void testMaterializingConstantTree(@Injectable RecordBatch batch) throws SchemaChangeException {
-    
+
     ErrorCollector ec = new ErrorCollectorImpl();
-    LogicalExpression expr = ExpressionTreeMaterializer.materialize(new ValueExpressions.LongExpression(1L, ExpressionPosition.UNKNOWN), batch, ec);
+    LogicalExpression expr = ExpressionTreeMaterializer.materialize(new ValueExpressions.LongExpression(1L,
+        ExpressionPosition.UNKNOWN), batch, ec);
     assertTrue(expr instanceof ValueExpressions.LongExpression);
     assertEquals(1L, ValueExpressions.LongExpression.class.cast(expr).getLong());
     assertFalse(ec.hasErrors());
@@ -62,13 +62,14 @@ public class ExpressionTreeMaterializerTest {
     final SchemaBuilder builder = BatchSchema.newBuilder();
     builder.addField(getField(2, "test", bigIntType));
     final BatchSchema schema = builder.build();
-    
+
     new NonStrictExpectations() {
       {
-        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
+        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN));
+        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
       }
     };
-    
+
     ErrorCollector ec = new ErrorCollectorImpl();
     LogicalExpression expr = ExpressionTreeMaterializer.materialize(new FieldReference("test",
         ExpressionPosition.UNKNOWN), batch, ec);
@@ -80,23 +81,26 @@ public class ExpressionTreeMaterializerTest {
   public void testMaterializingLateboundTree(final @Injectable RecordBatch batch) throws SchemaChangeException {
     new NonStrictExpectations() {
       {
-        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIT), -4);
-        batch.getValueVectorId(new FieldReference("test1", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
+        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN));
+        result = new TypedFieldId(Types.required(MinorType.BIT), -4);
+        batch.getValueVectorId(new FieldReference("test1", ExpressionPosition.UNKNOWN));
+        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
       }
     };
-    
+
     ErrorCollector ec = new ErrorCollectorImpl();
 
-    
-      LogicalExpression expr = new IfExpression.Builder()
+    LogicalExpression expr = new IfExpression.Builder()
         .addCondition(
             new IfExpression.IfCondition( //
                 new FieldReference("test", ExpressionPosition.UNKNOWN), //
-                new IfExpression.Builder() //
+                new IfExpression.Builder()
+                    //
                     .addCondition( //
-                        new IfExpression.IfCondition( //
-                            new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN), new FieldReference(
-                                "test1", ExpressionPosition.UNKNOWN)))
+                        new IfExpression.IfCondition(
+                            //
+                            new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN),
+                            new FieldReference("test1", ExpressionPosition.UNKNOWN)))
                     .setElse(new ValueExpressions.LongExpression(1L, ExpressionPosition.UNKNOWN)).build()) //
         ) //
         .setElse(new ValueExpressions.LongExpression(1L, ExpressionPosition.UNKNOWN)).build();
@@ -111,7 +115,8 @@ public class ExpressionTreeMaterializerTest {
     ifCondition = newIfExpr.conditions.get(0);
     assertEquals(bigIntType, ifCondition.expression.getMajorType());
     assertEquals(true, ((ValueExpressions.BooleanExpression) ifCondition.condition).value);
-    if (ec.hasErrors()) System.out.println(ec.toErrorString());
+    if (ec.hasErrors())
+      System.out.println(ec.toErrorString());
     assertFalse(ec.hasErrors());
   }
 
@@ -126,8 +131,8 @@ public class ExpressionTreeMaterializerTest {
       }
 
       @Override
-      public void addUnexpectedArgumentType(ExpressionPosition expr, String name, MajorType actual, MajorType[] expected,
-          int argumentIndex) {
+      public void addUnexpectedArgumentType(ExpressionPosition expr, String name, MajorType actual,
+          MajorType[] expected, int argumentIndex) {
         errorCount++;
       }
 
@@ -174,14 +179,17 @@ public class ExpressionTreeMaterializerTest {
 
     new NonStrictExpectations() {
       {
-        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN)); result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
+        batch.getValueVectorId(new FieldReference("test", ExpressionPosition.UNKNOWN));
+        result = new TypedFieldId(Types.required(MinorType.BIGINT), -5);
       }
     };
+
     
     LogicalExpression functionCallExpr = new FunctionCall(FunctionDefinition.simple("testFunc",
         new ArgumentValidator() {
           @Override
-          public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions, ErrorCollector errors) {
+          public void validateArguments(ExpressionPosition expr, List<LogicalExpression> expressions,
+              ErrorCollector errors) {
             errors.addGeneralError(expr, "Error!");
           }
 
@@ -189,8 +197,8 @@ public class ExpressionTreeMaterializerTest {
           public String[] getArgumentNamesByPosition() {
             return new String[0];
           }
-        }, OutputTypeDeterminer.FIXED_BIT), Lists.newArrayList((LogicalExpression) new FieldReference("test",
-        ExpressionPosition.UNKNOWN)), ExpressionPosition.UNKNOWN);
+        }, OutputTypeDeterminer.FIXED_BIT), ImmutableList.of((LogicalExpression) // 
+            new FieldReference("test", ExpressionPosition.UNKNOWN) ), ExpressionPosition.UNKNOWN);
     LogicalExpression newExpr = ExpressionTreeMaterializer.materialize(functionCallExpr, batch, ec);
     assertTrue(newExpr instanceof FunctionCall);
     FunctionCall funcExpr = (FunctionCall) newExpr;